Multiple remote connection strategy support (#48496)

* Extract remote "sniffing" to connection strategy (#47253)

Currently the connection strategy used by the remote cluster service is
implemented as a multi-step sniffing process in the
RemoteClusterConnection. We intend to introduce a new connection strategy
that will operate in a different manner. This commit extracts the
sniffing logic to a dedicated strategy class. Additionally, it implements
dedicated tests for this class.

Additionally, in previous commits we moved away from a world where the
remote cluster connection was mutable. Instead, when setting updates are
made, the connection is torn down and rebuilt. We still had methods and
tests hanging around for the mutable behavior. This commit removes those.

* Introduce simple remote connection strategy (#47480)

This commit introduces a simple remote connection strategy which will
open remote connections to a configurable list of user supplied
addresses. These addresses can be remote Elasticsearch nodes or
intermediate proxies. We will perform normal clustername and version
validation, but otherwise rely on the remote cluster to route requests
to the appropriate remote node.

* Make remote setting updates support diff strategies (#47891)

Currently the entire remote cluster settings infrastructure is designed
around the sniff strategy. As we introduce an additional conneciton
strategy this infrastructure needs to be modified to support it. This
commit modifies the code so that the strategy implementations will tell
the service if the connection needs to be torn down and rebuilt.

As part of this commit, we will wait 10 seconds for new clusters to
connect when they are added through the "update" settings
infrastructure.

* Make remote setting updates support diff strategies (#47891)

Currently the entire remote cluster settings infrastructure is designed
around the sniff strategy. As we introduce an additional conneciton
strategy this infrastructure needs to be modified to support it. This
commit modifies the code so that the strategy implementations will tell
the service if the connection needs to be torn down and rebuilt.

As part of this commit, we will wait 10 seconds for new clusters to
connect when they are added through the "update" settings
infrastructure.
This commit is contained in:
Tim Brooks 2019-10-25 09:29:41 -06:00 committed by GitHub
parent d6d2edf324
commit f5f1072824
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 2156 additions and 1638 deletions

View File

@ -31,8 +31,10 @@ import org.elasticsearch.common.util.concurrent.RunOnce;
import org.elasticsearch.core.internal.io.IOUtils;
import java.io.Closeable;
import java.util.Collections;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
@ -216,6 +218,10 @@ public class ConnectionManager implements Closeable {
return connectedNodes.size();
}
public Set<DiscoveryNode> getAllConnectedNodes() {
return Collections.unmodifiableSet(connectedNodes.keySet());
}
@Override
public void close() {
internalClose(true);

View File

@ -31,7 +31,6 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.SettingUpgrader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import java.net.InetAddress;
import java.net.InetSocketAddress;
@ -62,19 +61,19 @@ public abstract class RemoteClusterAware {
}
public static final Setting.AffixSetting<List<String>> SEARCH_REMOTE_CLUSTERS_SEEDS =
Setting.affixKeySetting(
"search.remote.",
"seeds",
key -> Setting.listSetting(
key,
Collections.emptyList(),
s -> {
parsePort(s);
return s;
},
Setting.Property.Deprecated,
Setting.Property.Dynamic,
Setting.Property.NodeScope));
Setting.affixKeySetting(
"search.remote.",
"seeds",
key -> Setting.listSetting(
key,
Collections.emptyList(),
s -> {
parsePort(s);
return s;
},
Setting.Property.Deprecated,
Setting.Property.Dynamic,
Setting.Property.NodeScope));
public static final SettingUpgrader<List<String>> SEARCH_REMOTE_CLUSTER_SEEDS_UPGRADER = new SettingUpgrader<List<String>>() {
@ -94,39 +93,39 @@ public abstract class RemoteClusterAware {
* A list of initial seed nodes to discover eligible nodes from the remote cluster
*/
public static final Setting.AffixSetting<List<String>> REMOTE_CLUSTERS_SEEDS = Setting.affixKeySetting(
"cluster.remote.",
"seeds",
key -> Setting.listSetting(
key,
// the default needs to be emptyList() when fallback is removed
"_na_".equals(key)
? SEARCH_REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(key)
: SEARCH_REMOTE_CLUSTERS_SEEDS.getConcreteSetting(key.replaceAll("^cluster", "search")),
s -> {
// validate seed address
parsePort(s);
return s;
},
Setting.Property.Dynamic,
Setting.Property.NodeScope));
"cluster.remote.",
"seeds",
key -> Setting.listSetting(
key,
// the default needs to be emptyList() when fallback is removed
"_na_".equals(key)
? SEARCH_REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(key)
: SEARCH_REMOTE_CLUSTERS_SEEDS.getConcreteSetting(key.replaceAll("^cluster", "search")),
s -> {
// validate seed address
parsePort(s);
return s;
},
Setting.Property.Dynamic,
Setting.Property.NodeScope));
public static final char REMOTE_CLUSTER_INDEX_SEPARATOR = ':';
public static final String LOCAL_CLUSTER_GROUP_KEY = "";
public static final Setting.AffixSetting<String> SEARCH_REMOTE_CLUSTERS_PROXY = Setting.affixKeySetting(
"search.remote.",
"proxy",
key -> Setting.simpleString(
key,
s -> {
if (Strings.hasLength(s)) {
parsePort(s);
}
},
Setting.Property.Deprecated,
Setting.Property.Dynamic,
Setting.Property.NodeScope),
REMOTE_CLUSTERS_SEEDS);
"search.remote.",
"proxy",
key -> Setting.simpleString(
key,
s -> {
if (Strings.hasLength(s)) {
parsePort(s);
}
},
Setting.Property.Deprecated,
Setting.Property.Dynamic,
Setting.Property.NodeScope),
REMOTE_CLUSTERS_SEEDS);
public static final SettingUpgrader<String> SEARCH_REMOTE_CLUSTERS_PROXY_UPGRADER = new SettingUpgrader<String>() {
@ -149,29 +148,30 @@ public abstract class RemoteClusterAware {
* undocumented as it does not work well with all proxies.
*/
public static final Setting.AffixSetting<String> REMOTE_CLUSTERS_PROXY = Setting.affixKeySetting(
"cluster.remote.",
"proxy",
key -> Setting.simpleString(
key,
// no default is needed when fallback is removed, use simple string which gives empty
"_na_".equals(key)
? SEARCH_REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(key)
: SEARCH_REMOTE_CLUSTERS_PROXY.getConcreteSetting(key.replaceAll("^cluster", "search")),
s -> {
if (Strings.hasLength(s)) {
parsePort(s);
}
return s;
},
Setting.Property.Dynamic,
Setting.Property.NodeScope),
REMOTE_CLUSTERS_SEEDS);
"cluster.remote.",
"proxy",
key -> Setting.simpleString(
key,
// no default is needed when fallback is removed, use simple string which gives empty
"_na_".equals(key)
? SEARCH_REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(key)
: SEARCH_REMOTE_CLUSTERS_PROXY.getConcreteSetting(key.replaceAll("^cluster", "search")),
s -> {
if (Strings.hasLength(s)) {
parsePort(s);
}
return s;
},
Setting.Property.Dynamic,
Setting.Property.NodeScope),
REMOTE_CLUSTERS_SEEDS);
protected final Settings settings;
private final ClusterNameExpressionResolver clusterNameResolver;
/**
* Creates a new {@link RemoteClusterAware} instance
*
* @param settings the nodes level settings
*/
protected RemoteClusterAware(Settings settings) {
@ -179,49 +179,60 @@ public abstract class RemoteClusterAware {
this.clusterNameResolver = new ClusterNameExpressionResolver();
}
/**
* Returns remote clusters that are enabled in these settings
*/
protected static Set<String> getEnabledRemoteClusters(final Settings settings) {
final Stream<Setting<List<String>>> allConcreteSettings = REMOTE_CLUSTERS_SEEDS.getAllConcreteSettings(settings);
return allConcreteSettings
.map(REMOTE_CLUSTERS_SEEDS::getNamespace)
.filter(clusterAlias -> RemoteConnectionStrategy.isConnectionEnabled(clusterAlias, settings))
.collect(Collectors.toSet());
}
/**
* Builds the dynamic per-cluster config from the given settings. This is a map keyed by the cluster alias that points to a tuple
* (ProxyAddresss, [SeedNodeSuppliers]). If a cluster is configured with a proxy address all seed nodes will point to
* {@link TransportAddress#META_ADDRESS} and their configured address will be used as the hostname for the generated discovery node.
*/
protected static Map<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> buildRemoteClustersDynamicConfig(
final Settings settings) {
final Settings settings) {
final Map<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> remoteSeeds =
buildRemoteClustersDynamicConfig(settings, REMOTE_CLUSTERS_SEEDS);
buildRemoteClustersDynamicConfig(settings, REMOTE_CLUSTERS_SEEDS);
final Map<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> searchRemoteSeeds =
buildRemoteClustersDynamicConfig(settings, SEARCH_REMOTE_CLUSTERS_SEEDS);
buildRemoteClustersDynamicConfig(settings, SEARCH_REMOTE_CLUSTERS_SEEDS);
// sort the intersection for predictable output order
final NavigableSet<String> intersection =
new TreeSet<>(Arrays.asList(
searchRemoteSeeds.keySet().stream().filter(s -> remoteSeeds.keySet().contains(s)).sorted().toArray(String[]::new)));
new TreeSet<>(Arrays.asList(
searchRemoteSeeds.keySet().stream().filter(s -> remoteSeeds.keySet().contains(s)).sorted().toArray(String[]::new)));
if (intersection.isEmpty() == false) {
final String message = String.format(
Locale.ROOT,
"found duplicate remote cluster configurations for cluster alias%s [%s]",
intersection.size() == 1 ? "" : "es",
String.join(",", intersection));
Locale.ROOT,
"found duplicate remote cluster configurations for cluster alias%s [%s]",
intersection.size() == 1 ? "" : "es",
String.join(",", intersection));
throw new IllegalArgumentException(message);
}
return Stream
.concat(remoteSeeds.entrySet().stream(), searchRemoteSeeds.entrySet().stream())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
.concat(remoteSeeds.entrySet().stream(), searchRemoteSeeds.entrySet().stream())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
}
private static Map<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> buildRemoteClustersDynamicConfig(
final Settings settings, final Setting.AffixSetting<List<String>> seedsSetting) {
final Settings settings, final Setting.AffixSetting<List<String>> seedsSetting) {
final Stream<Setting<List<String>>> allConcreteSettings = seedsSetting.getAllConcreteSettings(settings);
return allConcreteSettings.collect(
Collectors.toMap(seedsSetting::getNamespace, concreteSetting -> {
String clusterName = seedsSetting.getNamespace(concreteSetting);
List<String> addresses = concreteSetting.get(settings);
final boolean proxyMode =
REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterName).existsOrFallbackExists(settings);
List<Tuple<String, Supplier<DiscoveryNode>>> nodes = new ArrayList<>(addresses.size());
for (String address : addresses) {
nodes.add(Tuple.tuple(address, () -> buildSeedNode(clusterName, address, proxyMode)));
}
return new Tuple<>(REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterName).get(settings), nodes);
}));
Collectors.toMap(seedsSetting::getNamespace, concreteSetting -> {
String clusterName = seedsSetting.getNamespace(concreteSetting);
List<String> addresses = concreteSetting.get(settings);
final boolean proxyMode =
REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterName).existsOrFallbackExists(settings);
List<Tuple<String, Supplier<DiscoveryNode>>> nodes = new ArrayList<>(addresses.size());
for (String address : addresses) {
nodes.add(Tuple.tuple(address, () -> buildSeedNode(clusterName, address, proxyMode)));
}
return new Tuple<>(REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterName).get(settings), nodes);
}));
}
static DiscoveryNode buildSeedNode(String clusterName, String address, boolean proxyMode) {
@ -229,13 +240,13 @@ public abstract class RemoteClusterAware {
TransportAddress transportAddress = new TransportAddress(TransportAddress.META_ADDRESS, 0);
String hostName = address.substring(0, indexOfPortSeparator(address));
return new DiscoveryNode("", clusterName + "#" + address, UUIDs.randomBase64UUID(), hostName, address,
transportAddress, Collections.singletonMap("server_name", hostName), DiscoveryNodeRole.BUILT_IN_ROLES,
Version.CURRENT.minimumCompatibilityVersion());
transportAddress, Collections.singletonMap("server_name", hostName), DiscoveryNodeRole.BUILT_IN_ROLES,
Version.CURRENT.minimumCompatibilityVersion());
} else {
TransportAddress transportAddress = new TransportAddress(RemoteClusterAware.parseSeedAddress(address));
return new DiscoveryNode(clusterName + "#" + transportAddress.toString(),
transportAddress,
Version.CURRENT.minimumCompatibilityVersion());
transportAddress,
Version.CURRENT.minimumCompatibilityVersion());
}
}
@ -245,9 +256,8 @@ public abstract class RemoteClusterAware {
* {@link #LOCAL_CLUSTER_GROUP_KEY}. The returned map is mutable.
*
* @param remoteClusterNames the remote cluster names
* @param requestIndices the indices in the search request to filter
* @param indexExists a predicate that can test if a certain index or alias exists in the local cluster
*
* @param requestIndices the indices in the search request to filter
* @param indexExists a predicate that can test if a certain index or alias exists in the local cluster
* @return a map of grouped remote and local indices
*/
protected Map<String, List<String>> groupClusterIndices(Set<String> remoteClusterNames, String[] requestIndices,
@ -265,7 +275,7 @@ public abstract class RemoteClusterAware {
//if that happens. Note that indices and aliases can be created with ":" in their names names up to 6.last, which
//means such names need to be supported until 7.last. It will be possible to remove this check from 8.0 on.
throw new IllegalArgumentException("Can not filter indices; index " + index +
" exists but there is also a remote cluster named: " + remoteClusterName);
" exists but there is also a remote cluster named: " + remoteClusterName);
}
String indexName = index.substring(i + 1);
for (String clusterName : clusters) {
@ -284,29 +294,17 @@ public abstract class RemoteClusterAware {
return perClusterIndices;
}
void updateRemoteCluster(String clusterAlias, List<String> addresses, String proxy) {
Boolean compress = TransportSettings.TRANSPORT_COMPRESS.get(settings);
TimeValue pingSchedule = TransportSettings.PING_SCHEDULE.get(settings);
updateRemoteCluster(clusterAlias, addresses, proxy, compress, pingSchedule);
}
void updateRemoteCluster(String clusterAlias, Settings settings) {
String proxy = REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterAlias).get(settings);
List<String> addresses = REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).get(settings);
Boolean compress = RemoteClusterService.REMOTE_CLUSTER_COMPRESS.getConcreteSettingForNamespace(clusterAlias).get(settings);
TimeValue pingSchedule = RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE
.getConcreteSettingForNamespace(clusterAlias)
.get(settings);
updateRemoteCluster(clusterAlias, addresses, proxy, compress, pingSchedule);
void validateAndUpdateRemoteCluster(String clusterAlias, Settings settings) {
if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias)) {
throw new IllegalArgumentException("remote clusters must not have the empty string as its key");
}
updateRemoteCluster(clusterAlias, settings);
}
/**
* Subclasses must implement this to receive information about updated cluster aliases. If the given address list is
* empty the cluster alias is unregistered and should be removed.
* Subclasses must implement this to receive information about updated cluster aliases.
*/
protected abstract void updateRemoteCluster(String clusterAlias, List<String> addresses, String proxy, boolean compressionEnabled,
TimeValue pingSchedule);
protected abstract void updateRemoteCluster(String clusterAlias, Settings settings);
/**
* Registers this instance to listen to updates on the cluster settings.
@ -314,13 +312,9 @@ public abstract class RemoteClusterAware {
public void listenForUpdates(ClusterSettings clusterSettings) {
List<Setting.AffixSetting<?>> remoteClusterSettings = Arrays.asList(RemoteClusterAware.REMOTE_CLUSTERS_PROXY,
RemoteClusterAware.REMOTE_CLUSTERS_SEEDS, RemoteClusterService.REMOTE_CLUSTER_COMPRESS,
RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE);
clusterSettings.addAffixGroupUpdateConsumer(remoteClusterSettings, this::updateRemoteCluster);
clusterSettings.addAffixUpdateConsumer(
RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_PROXY,
RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_SEEDS,
(key, value) -> updateRemoteCluster(key, value.v2(), value.v1()),
(namespace, value) -> {});
RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE, RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_PROXY,
RemoteClusterAware.SEARCH_REMOTE_CLUSTERS_SEEDS);
clusterSettings.addAffixGroupUpdateConsumer(remoteClusterSettings, this::validateAndUpdateRemoteCluster);
}
static InetSocketAddress parseSeedAddress(String remoteHost) {

View File

@ -18,45 +18,24 @@
*/
package org.elasticsearch.transport;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.StepListener;
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
/**
* Represents a connection to a single remote cluster. In contrast to a local cluster a remote cluster is not joined such that the
@ -71,86 +50,41 @@ import java.util.stream.Collectors;
* {@link RemoteClusterService#REMOTE_CONNECTIONS_PER_CLUSTER} until either all eligible nodes are exhausted or the maximum number of
* connections per cluster has been reached.
*/
final class RemoteClusterConnection implements TransportConnectionListener, Closeable {
private static final Logger logger = LogManager.getLogger(RemoteClusterConnection.class);
final class RemoteClusterConnection implements Closeable {
private final TransportService transportService;
private final RemoteConnectionManager remoteConnectionManager;
private final RemoteConnectionStrategy connectionStrategy;
private final String clusterAlias;
private final int maxNumRemoteConnections;
private final Predicate<DiscoveryNode> nodePredicate;
private final ThreadPool threadPool;
private volatile String proxyAddress;
private volatile List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes;
private volatile boolean skipUnavailable;
private final ConnectHandler connectHandler;
private final TimeValue initialConnectionTimeout;
private final SetOnce<ClusterName> remoteClusterName = new SetOnce<>();
/**
* Creates a new {@link RemoteClusterConnection}
* @param settings the nodes settings object
* @param clusterAlias the configured alias of the cluster to connect to
* @param seedNodes a list of seed nodes to discover eligible nodes from
* @param transportService the local nodes transport service
* @param maxNumRemoteConnections the maximum number of connections to the remote cluster
* @param nodePredicate a predicate to filter eligible remote nodes to connect to
* @param proxyAddress the proxy address
* @param connectionProfile the connection profile to use
*/
RemoteClusterConnection(Settings settings, String clusterAlias, List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes,
TransportService transportService, int maxNumRemoteConnections, Predicate<DiscoveryNode> nodePredicate,
String proxyAddress, ConnectionProfile connectionProfile) {
this(settings, clusterAlias, seedNodes, transportService, maxNumRemoteConnections, nodePredicate, proxyAddress,
createConnectionManager(connectionProfile, transportService));
RemoteClusterConnection(Settings settings, String clusterAlias, TransportService transportService) {
this(settings, clusterAlias, transportService,
createConnectionManager(buildConnectionProfileFromSettings(settings, clusterAlias), transportService));
}
// Public for tests to pass a StubbableConnectionManager
RemoteClusterConnection(Settings settings, String clusterAlias, List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes,
TransportService transportService, int maxNumRemoteConnections, Predicate<DiscoveryNode> nodePredicate,
String proxyAddress, ConnectionManager connectionManager) {
RemoteClusterConnection(Settings settings, String clusterAlias, TransportService transportService,
ConnectionManager connectionManager) {
this.transportService = transportService;
this.maxNumRemoteConnections = maxNumRemoteConnections;
this.nodePredicate = nodePredicate;
this.clusterAlias = clusterAlias;
this.remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
this.seedNodes = Collections.unmodifiableList(seedNodes);
this.skipUnavailable = RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE
.getConcreteSettingForNamespace(clusterAlias).get(settings);
this.connectHandler = new ConnectHandler();
this.threadPool = transportService.threadPool;
connectionManager.addListener(this);
this.connectionStrategy = new SniffConnectionStrategy(clusterAlias, transportService, remoteConnectionManager, settings);
// we register the transport service here as a listener to make sure we notify handlers on disconnect etc.
connectionManager.addListener(transportService);
this.proxyAddress = proxyAddress;
this.skipUnavailable = RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE
.getConcreteSettingForNamespace(clusterAlias).get(settings);
this.threadPool = transportService.threadPool;
initialConnectionTimeout = RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING.get(settings);
}
private static DiscoveryNode maybeAddProxyAddress(String proxyAddress, DiscoveryNode node) {
if (proxyAddress == null || proxyAddress.isEmpty()) {
return node;
} else {
// resolve proxy address lazy here
InetSocketAddress proxyInetAddress = RemoteClusterAware.parseSeedAddress(proxyAddress);
return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node
.getHostAddress(), new TransportAddress(proxyInetAddress), node.getAttributes(), node.getRoles(), node.getVersion());
}
}
/**
* Updates the list of seed nodes for this cluster connection
*/
synchronized void updateSeedNodes(
final String proxyAddress,
final List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes,
final ActionListener<Void> connectListener) {
this.seedNodes = Collections.unmodifiableList(new ArrayList<>(seedNodes));
this.proxyAddress = proxyAddress;
connectHandler.connect(connectListener);
}
/**
* Updates the skipUnavailable flag that can be dynamically set for each remote cluster
*/
@ -165,25 +99,15 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
return skipUnavailable;
}
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
if (remoteConnectionManager.size() < maxNumRemoteConnections) {
// try to reconnect and fill up the slot of the disconnected node
connectHandler.connect(ActionListener.wrap(
ignore -> logger.trace("successfully connected after disconnect of {}", node),
e -> logger.trace(() -> new ParameterizedMessage("failed to connect after disconnect of {}", node), e)));
}
}
/**
* Ensures that this cluster is connected. If the cluster is connected this operation
* will invoke the listener immediately.
*/
void ensureConnected(ActionListener<Void> voidActionListener) {
void ensureConnected(ActionListener<Void> listener) {
if (remoteConnectionManager.size() == 0) {
connectHandler.connect(voidActionListener);
connectionStrategy.connect(listener);
} else {
voidActionListener.onResponse(null);
listener.onResponse(null);
}
}
@ -256,316 +180,22 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
return remoteConnectionManager.getRemoteConnection(remoteClusterNode);
}
private Predicate<ClusterName> getRemoteClusterNamePredicate() {
return
new Predicate<ClusterName>() {
@Override
public boolean test(ClusterName c) {
return remoteClusterName.get() == null || c.equals(remoteClusterName.get());
}
@Override
public String toString() {
return remoteClusterName.get() == null ? "any cluster name"
: "expected remote cluster name [" + remoteClusterName.get().value() + "]";
}
};
}
Transport.Connection getConnection() {
return remoteConnectionManager.getAnyRemoteConnection();
}
@Override
public void close() throws IOException {
IOUtils.close(connectHandler, remoteConnectionManager);
IOUtils.close(connectionStrategy, remoteConnectionManager);
}
public boolean isClosed() {
return connectHandler.isClosed();
return connectionStrategy.isClosed();
}
public List<Tuple<String, Supplier<DiscoveryNode>>> getSeedNodes() {
return seedNodes;
}
/**
* The connect handler manages node discovery and the actual connect to the remote cluster.
* There is at most one connect job running at any time. If such a connect job is triggered
* while another job is running the provided listeners are queued and batched up until the current running job returns.
*
* The handler has a built-in queue that can hold up to 100 connect attempts and will reject requests once the queue is full.
* In a scenario when a remote cluster becomes unavailable we will queue requests up but if we can't connect quick enough
* we will just reject the connect trigger which will lead to failing searches.
*/
private class ConnectHandler implements Closeable {
private static final int MAX_LISTENERS = 100;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Object mutex = new Object();
private List<ActionListener<Void>> listeners = new ArrayList<>();
/**
* Triggers a connect round unless there is one running already. If there is a connect round running, the listener will either
* be queued or rejected and failed.
*/
void connect(ActionListener<Void> connectListener) {
boolean runConnect = false;
final ActionListener<Void> listener =
ContextPreservingActionListener.wrapPreservingContext(connectListener, threadPool.getThreadContext());
boolean closed;
synchronized (mutex) {
closed = this.closed.get();
if (closed) {
assert listeners.isEmpty();
} else {
if (listeners.size() >= MAX_LISTENERS) {
assert listeners.size() == MAX_LISTENERS;
listener.onFailure(new RejectedExecutionException("connect queue is full"));
return;
} else {
listeners.add(listener);
}
runConnect = listeners.size() == 1;
}
}
if (closed) {
connectListener.onFailure(new AlreadyClosedException("connect handler is already closed"));
return;
}
if (runConnect) {
ExecutorService executor = threadPool.executor(ThreadPool.Names.MANAGEMENT);
executor.submit(new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
ActionListener.onFailure(getAndClearListeners(), e);
}
@Override
protected void doRun() {
collectRemoteNodes(seedNodes.stream().map(Tuple::v2).iterator(),
new ActionListener<Void>() {
@Override
public void onResponse(Void aVoid) {
ActionListener.onResponse(getAndClearListeners(), aVoid);
}
@Override
public void onFailure(Exception e) {
ActionListener.onFailure(getAndClearListeners(), e);
}
});
}
});
}
}
private List<ActionListener<Void>> getAndClearListeners() {
final List<ActionListener<Void>> result;
synchronized (mutex) {
if (listeners.isEmpty()) {
result = Collections.emptyList();
} else {
result = listeners;
listeners = new ArrayList<>();
}
}
return result;
}
private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes, ActionListener<Void> listener) {
if (Thread.currentThread().isInterrupted()) {
listener.onFailure(new InterruptedException("remote connect thread got interrupted"));
}
if (seedNodes.hasNext()) {
final Consumer<Exception> onFailure = e -> {
if (e instanceof ConnectTransportException ||
e instanceof IOException ||
e instanceof IllegalStateException) {
// ISE if we fail the handshake with an version incompatible node
if (seedNodes.hasNext()) {
logger.debug(() -> new ParameterizedMessage(
"fetching nodes from external cluster [{}] failed moving to next node", clusterAlias), e);
collectRemoteNodes(seedNodes, listener);
return;
}
}
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster [{}] failed", clusterAlias), e);
listener.onFailure(e);
};
final DiscoveryNode seedNode = maybeAddProxyAddress(proxyAddress, seedNodes.next().get());
logger.debug("[{}] opening connection to seed node: [{}] proxy address: [{}]", clusterAlias, seedNode,
proxyAddress);
final ConnectionProfile profile = ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG);
final StepListener<Transport.Connection> openConnectionStep = new StepListener<>();
try {
remoteConnectionManager.openConnection(seedNode, profile, openConnectionStep);
} catch (Exception e) {
onFailure.accept(e);
}
final StepListener<TransportService.HandshakeResponse> handShakeStep = new StepListener<>();
openConnectionStep.whenComplete(connection -> {
ConnectionProfile connectionProfile = remoteConnectionManager.getConnectionManager().getConnectionProfile();
transportService.handshake(connection, connectionProfile.getHandshakeTimeout().millis(),
getRemoteClusterNamePredicate(), handShakeStep);
}, onFailure);
final StepListener<Void> fullConnectionStep = new StepListener<>();
handShakeStep.whenComplete(handshakeResponse -> {
final DiscoveryNode handshakeNode = maybeAddProxyAddress(proxyAddress, handshakeResponse.getDiscoveryNode());
if (nodePredicate.test(handshakeNode) && remoteConnectionManager.size() < maxNumRemoteConnections) {
remoteConnectionManager.connectToNode(handshakeNode, null,
transportService.connectionValidator(handshakeNode), fullConnectionStep);
} else {
fullConnectionStep.onResponse(null);
}
}, e -> {
final Transport.Connection connection = openConnectionStep.result();
logger.warn(new ParameterizedMessage("failed to connect to seed node [{}]", connection.getNode()), e);
IOUtils.closeWhileHandlingException(connection);
onFailure.accept(e);
});
fullConnectionStep.whenComplete(aVoid -> {
if (remoteClusterName.get() == null) {
TransportService.HandshakeResponse handshakeResponse = handShakeStep.result();
assert handshakeResponse.getClusterName().value() != null;
remoteClusterName.set(handshakeResponse.getClusterName());
}
final Transport.Connection connection = openConnectionStep.result();
ClusterStateRequest request = new ClusterStateRequest();
request.clear();
request.nodes(true);
// here we pass on the connection since we can only close it once the sendRequest returns otherwise
// due to the async nature (it will return before it's actually sent) this can cause the request to fail
// due to an already closed connection.
ThreadPool threadPool = transportService.getThreadPool();
ThreadContext threadContext = threadPool.getThreadContext();
TransportService.ContextRestoreResponseHandler<ClusterStateResponse> responseHandler = new TransportService
.ContextRestoreResponseHandler<>(threadContext.newRestorableContext(false),
new SniffClusterStateResponseHandler(connection, listener, seedNodes));
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
// we stash any context here since this is an internal execution and should not leak any
// existing context information.
threadContext.markAsSystemContext();
transportService.sendRequest(connection, ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY,
responseHandler);
}
}, e -> {
IOUtils.closeWhileHandlingException(openConnectionStep.result());
onFailure.accept(e);
});
} else {
listener.onFailure(new IllegalStateException("no seed node left"));
}
}
@Override
public void close() throws IOException {
final List<ActionListener<Void>> toNotify;
synchronized (mutex) {
if (closed.compareAndSet(false, true)) {
toNotify = listeners;
listeners = Collections.emptyList();
} else {
toNotify = Collections.emptyList();
}
}
ActionListener.onFailure(toNotify, new AlreadyClosedException("connect handler is already closed"));
}
final boolean isClosed() {
return closed.get();
}
/* This class handles the _state response from the remote cluster when sniffing nodes to connect to */
private class SniffClusterStateResponseHandler implements TransportResponseHandler<ClusterStateResponse> {
private final Transport.Connection connection;
private final ActionListener<Void> listener;
private final Iterator<Supplier<DiscoveryNode>> seedNodes;
SniffClusterStateResponseHandler(Transport.Connection connection, ActionListener<Void> listener,
Iterator<Supplier<DiscoveryNode>> seedNodes) {
this.connection = connection;
this.listener = listener;
this.seedNodes = seedNodes;
}
@Override
public ClusterStateResponse read(StreamInput in) throws IOException {
return new ClusterStateResponse(in);
}
@Override
public void handleResponse(ClusterStateResponse response) {
handleNodes(response.getState().nodes().getNodes().valuesIt());
}
private void handleNodes(Iterator<DiscoveryNode> nodesIter) {
while (nodesIter.hasNext()) {
final DiscoveryNode node = maybeAddProxyAddress(proxyAddress, nodesIter.next());
if (nodePredicate.test(node) && remoteConnectionManager.size() < maxNumRemoteConnections) {
remoteConnectionManager.connectToNode(node, null,
transportService.connectionValidator(node), new ActionListener<Void>() {
@Override
public void onResponse(Void aVoid) {
handleNodes(nodesIter);
}
@Override
public void onFailure(Exception e) {
if (e instanceof ConnectTransportException ||
e instanceof IllegalStateException) {
// ISE if we fail the handshake with an version incompatible node
// fair enough we can't connect just move on
logger.debug(() -> new ParameterizedMessage("failed to connect to node {}", node), e);
handleNodes(nodesIter);
} else {
logger.warn(() ->
new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), e);
IOUtils.closeWhileHandlingException(connection);
collectRemoteNodes(seedNodes, listener);
}
}
});
return;
}
}
// We have to close this connection before we notify listeners - this is mainly needed for test correctness
// since if we do it afterwards we might fail assertions that check if all high level connections are closed.
// from a code correctness perspective we could also close it afterwards.
IOUtils.closeWhileHandlingException(connection);
listener.onResponse(null);
}
@Override
public void handleException(TransportException exp) {
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), exp);
try {
IOUtils.closeWhileHandlingException(connection);
} finally {
// once the connection is closed lets try the next node
collectRemoteNodes(seedNodes, listener);
}
}
@Override
public String executor() {
return ThreadPool.Names.MANAGEMENT;
}
}
}
boolean assertNoRunningConnections() { // for testing only
synchronized (connectHandler.mutex) {
assert connectHandler.listeners.isEmpty();
}
return true;
// for testing only
boolean assertNoRunningConnections() {
return connectionStrategy.assertNoRunningConnections();
}
boolean isNodeConnected(final DiscoveryNode node) {
@ -576,13 +206,24 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
* Get the information about remote nodes to be rendered on {@code _remote/info} requests.
*/
public RemoteConnectionInfo getConnectionInfo() {
return new RemoteConnectionInfo(
if (connectionStrategy instanceof SniffConnectionStrategy) {
SniffConnectionStrategy sniffStrategy = (SniffConnectionStrategy) this.connectionStrategy;
return new RemoteConnectionInfo(
clusterAlias,
seedNodes.stream().map(Tuple::v1).collect(Collectors.toList()),
maxNumRemoteConnections,
sniffStrategy.getSeedNodes(),
sniffStrategy.getMaxConnections(),
getNumNodesConnected(),
initialConnectionTimeout,
skipUnavailable);
} else {
return new RemoteConnectionInfo(
clusterAlias,
Collections.emptyList(),
0,
getNumNodesConnected(),
initialConnectionTimeout,
skipUnavailable);
}
}
int getNumNodesConnected() {
@ -596,4 +237,22 @@ final class RemoteClusterConnection implements TransportConnectionListener, Clos
ConnectionManager getConnectionManager() {
return remoteConnectionManager.getConnectionManager();
}
public boolean shouldRebuildConnection(Settings newSettings) {
return connectionStrategy.shouldRebuildConnection(newSettings);
}
static ConnectionProfile buildConnectionProfileFromSettings(Settings settings, String clusterName) {
return new ConnectionProfile.Builder()
.setConnectTimeout(TransportSettings.CONNECT_TIMEOUT.get(settings))
.setHandshakeTimeout(TransportSettings.CONNECT_TIMEOUT.get(settings))
.addConnections(6, TransportRequestOptions.Type.REG, TransportRequestOptions.Type.PING) // TODO make this configurable?
// we don't want this to be used for anything else but search
.addConnections(0, TransportRequestOptions.Type.BULK,
TransportRequestOptions.Type.STATE,
TransportRequestOptions.Type.RECOVERY)
.setCompressionEnabled(RemoteClusterService.REMOTE_CLUSTER_COMPRESS.getConcreteSettingForNamespace(clusterName).get(settings))
.setPingInterval(RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE.getConcreteSettingForNamespace(clusterName).get(settings))
.build();
}
}

View File

@ -24,18 +24,18 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.support.GroupedActionListener;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.SettingUpgrader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.CountDown;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.threadpool.ThreadPool;
@ -43,19 +43,16 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.function.BiFunction;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.elasticsearch.common.settings.Setting.boolSetting;
@ -182,99 +179,12 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
key -> boolSetting(key, TransportSettings.TRANSPORT_COMPRESS, Setting.Property.Dynamic, Setting.Property.NodeScope),
REMOTE_CLUSTERS_SEEDS);
private static final Predicate<DiscoveryNode> DEFAULT_NODE_PREDICATE = (node) -> Version.CURRENT.isCompatible(node.getVersion())
&& (node.isMasterNode() == false || node.isDataNode() || node.isIngestNode());
private final TransportService transportService;
private final int numRemoteConnections;
private volatile Map<String, RemoteClusterConnection> remoteClusters = Collections.emptyMap();
private volatile Map<String, ConnectionProfile> remoteClusterConnectionProfiles = Collections.emptyMap();
private final Map<String, RemoteClusterConnection> remoteClusters = ConcurrentCollections.newConcurrentMap();
RemoteClusterService(Settings settings, TransportService transportService) {
super(settings);
this.transportService = transportService;
numRemoteConnections = REMOTE_CONNECTIONS_PER_CLUSTER.get(settings);
}
/**
* This method updates the list of remote clusters. It's intended to be used as an update consumer on the settings infrastructure
* @param seeds a cluster alias to discovery node mapping representing the remote clusters seeds nodes
* @param connectionListener a listener invoked once every configured cluster has been connected to
*/
private synchronized void updateRemoteClusters(Map<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> seeds,
ActionListener<Void> connectionListener) {
if (seeds.containsKey(LOCAL_CLUSTER_GROUP_KEY)) {
throw new IllegalArgumentException("remote clusters must not have the empty string as its key");
}
Map<String, RemoteClusterConnection> remoteClusters = new HashMap<>();
if (seeds.isEmpty()) {
connectionListener.onResponse(null);
} else {
CountDown countDown = new CountDown(seeds.size());
remoteClusters.putAll(this.remoteClusters);
for (Map.Entry<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> entry : seeds.entrySet()) {
List<Tuple<String, Supplier<DiscoveryNode>>> seedList = entry.getValue().v2();
String proxyAddress = entry.getValue().v1();
String clusterAlias = entry.getKey();
RemoteClusterConnection remote = this.remoteClusters.get(clusterAlias);
ConnectionProfile connectionProfile = this.remoteClusterConnectionProfiles.get(clusterAlias);
if (seedList.isEmpty()) { // with no seed nodes we just remove the connection
try {
IOUtils.close(remote);
} catch (IOException e) {
logger.warn("failed to close remote cluster connections for cluster: " + clusterAlias, e);
}
remoteClusters.remove(clusterAlias);
continue;
}
if (remote == null) { // this is a new cluster we have to add a new representation
remote = new RemoteClusterConnection(settings, clusterAlias, seedList, transportService, numRemoteConnections,
getNodePredicate(settings), proxyAddress, connectionProfile);
remoteClusters.put(clusterAlias, remote);
} else if (connectionProfileChanged(remote.getConnectionManager().getConnectionProfile(), connectionProfile)
|| seedsChanged(remote.getSeedNodes(), seedList)) {
// New ConnectionProfile. Must tear down existing connection
try {
IOUtils.close(remote);
} catch (IOException e) {
logger.warn("failed to close remote cluster connections for cluster: " + clusterAlias, e);
}
remoteClusters.remove(clusterAlias);
remote = new RemoteClusterConnection(settings, clusterAlias, seedList, transportService, numRemoteConnections,
getNodePredicate(settings), proxyAddress, connectionProfile);
remoteClusters.put(clusterAlias, remote);
}
// now update the seed nodes no matter if it's new or already existing
RemoteClusterConnection finalRemote = remote;
remote.updateSeedNodes(proxyAddress, seedList, ActionListener.wrap(
response -> {
if (countDown.countDown()) {
connectionListener.onResponse(response);
}
},
exception -> {
if (countDown.fastForward()) {
connectionListener.onFailure(exception);
}
if (finalRemote.isClosed() == false) {
logger.warn("failed to update seed list for cluster: " + clusterAlias, exception);
}
}));
}
}
this.remoteClusters = Collections.unmodifiableMap(remoteClusters);
}
static Predicate<DiscoveryNode> getNodePredicate(Settings settings) {
if (REMOTE_NODE_ATTRIBUTE.exists(settings)) {
// nodes can be tagged with node.attr.remote_gateway: true to allow a node to be a gateway node for cross cluster search
String attribute = REMOTE_NODE_ATTRIBUTE.get(settings);
return DEFAULT_NODE_PREDICATE.and((node) -> Booleans.parseBoolean(node.getAttributes().getOrDefault(attribute, "false")));
}
return DEFAULT_NODE_PREDICATE;
}
/**
@ -326,6 +236,7 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
/**
* Returns a connection to the given node on the given remote cluster
*
* @throws IllegalArgumentException if the remote cluster is unknown
*/
public Transport.Connection getConnection(DiscoveryNode node, String cluster) {
@ -378,37 +289,69 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
}
@Override
protected void updateRemoteCluster(String clusterAlias, List<String> addresses, String proxyAddress, boolean compressionEnabled,
TimeValue pingSchedule) {
protected void updateRemoteCluster(String clusterAlias, Settings settings) {
if (remoteClusters.containsKey(clusterAlias) == false) {
CountDownLatch latch = new CountDownLatch(1);
updateRemoteCluster(clusterAlias, settings, ActionListener.wrap(latch::countDown));
try {
// Wait 10 seconds for a new cluster. We must use a latch instead of a future because we
// are on the cluster state thread and our custom future implementation will throw an
// assertion.
if (latch.await(10, TimeUnit.SECONDS) == false) {
logger.warn("failed to connect to new remote cluster {} within {}", clusterAlias, TimeValue.timeValueSeconds(10));
}
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
} else {
updateRemoteCluster(clusterAlias, settings, noopListener);
}
}
/**
* This method updates the list of remote clusters. It's intended to be used as an update consumer on the settings infrastructure
*
* @param clusterAlias a cluster alias to discovery node mapping representing the remote clusters seeds nodes
* @param newSettings the updated settings for the remote connection
* @param listener a listener invoked once every configured cluster has been connected to
*/
synchronized void updateRemoteCluster(String clusterAlias, Settings newSettings, ActionListener<Void> listener) {
if (LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias)) {
throw new IllegalArgumentException("remote clusters must not have the empty string as its key");
}
ConnectionProfile oldProfile = remoteClusterConnectionProfiles.get(clusterAlias);
ConnectionProfile newProfile;
if (oldProfile != null) {
ConnectionProfile.Builder builder = new ConnectionProfile.Builder(oldProfile);
builder.setCompressionEnabled(compressionEnabled);
builder.setPingInterval(pingSchedule);
newProfile = builder.build();
} else {
ConnectionProfile.Builder builder = new ConnectionProfile.Builder(buildConnectionProfileFromSettings(clusterAlias));
builder.setCompressionEnabled(compressionEnabled);
builder.setPingInterval(pingSchedule);
newProfile = builder.build();
}
updateRemoteCluster(clusterAlias, addresses, proxyAddress, newProfile, noopListener);
}
void updateRemoteCluster(final String clusterAlias, final List<String> addresses, final String proxyAddress,
final ConnectionProfile connectionProfile, final ActionListener<Void> connectionListener) {
HashMap<String, ConnectionProfile> connectionProfiles = new HashMap<>(remoteClusterConnectionProfiles);
connectionProfiles.put(clusterAlias, connectionProfile);
this.remoteClusterConnectionProfiles = Collections.unmodifiableMap(connectionProfiles);
final List<Tuple<String, Supplier<DiscoveryNode>>> nodes =
addresses.stream().<Tuple<String, Supplier<DiscoveryNode>>>map(address -> Tuple.tuple(address, () ->
buildSeedNode(clusterAlias, address, Strings.hasLength(proxyAddress)))
).collect(Collectors.toList());
updateRemoteClusters(Collections.singletonMap(clusterAlias, new Tuple<>(proxyAddress, nodes)), connectionListener);
RemoteClusterConnection remote = this.remoteClusters.get(clusterAlias);
if (RemoteConnectionStrategy.isConnectionEnabled(clusterAlias, newSettings) == false) {
try {
IOUtils.close(remote);
} catch (IOException e) {
logger.warn("failed to close remote cluster connections for cluster: " + clusterAlias, e);
}
remoteClusters.remove(clusterAlias);
listener.onResponse(null);
return;
}
// this is a new cluster we have to add a new representation
if (remote == null) {
Settings finalSettings = Settings.builder().put(this.settings, false).put(newSettings, false).build();
remote = new RemoteClusterConnection(finalSettings, clusterAlias, transportService);
remoteClusters.put(clusterAlias, remote);
} else if (remote.shouldRebuildConnection(newSettings)) {
// New ConnectionProfile. Must tear down existing connection
try {
IOUtils.close(remote);
} catch (IOException e) {
logger.warn("failed to close remote cluster connections for cluster: " + clusterAlias, e);
}
remoteClusters.remove(clusterAlias);
Settings finalSettings = Settings.builder().put(this.settings, false).put(newSettings, false).build();
remote = new RemoteClusterConnection(finalSettings, clusterAlias, transportService);
remoteClusters.put(clusterAlias, remote);
}
remote.ensureConnected(listener);
}
/**
@ -417,11 +360,22 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
*/
void initializeRemoteClusters() {
final TimeValue timeValue = REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING.get(settings);
final PlainActionFuture<Void> future = new PlainActionFuture<>();
Map<String, Tuple<String, List<Tuple<String, Supplier<DiscoveryNode>>>>> seeds =
RemoteClusterAware.buildRemoteClustersDynamicConfig(settings);
initializeConnectionProfiles(seeds.keySet());
updateRemoteClusters(seeds, future);
final PlainActionFuture<Collection<Void>> future = new PlainActionFuture<>();
Set<String> enabledClusters = RemoteClusterAware.getEnabledRemoteClusters(settings);
if (enabledClusters.isEmpty()) {
return;
}
GroupedActionListener<Void> listener = new GroupedActionListener<>(future, enabledClusters.size());
for (String clusterAlias : enabledClusters) {
updateRemoteCluster(clusterAlias, settings, listener);
}
if (enabledClusters.isEmpty()) {
future.onResponse(null);
}
try {
future.get(timeValue.millis(), TimeUnit.MILLISECONDS);
} catch (InterruptedException e) {
@ -433,32 +387,6 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
}
}
private synchronized void initializeConnectionProfiles(Set<String> remoteClusters) {
Map<String, ConnectionProfile> connectionProfiles = new HashMap<>(remoteClusters.size());
for (String clusterName : remoteClusters) {
connectionProfiles.put(clusterName, buildConnectionProfileFromSettings(clusterName));
}
this.remoteClusterConnectionProfiles = Collections.unmodifiableMap(connectionProfiles);
}
private ConnectionProfile buildConnectionProfileFromSettings(String clusterName) {
return buildConnectionProfileFromSettings(settings, clusterName);
}
static ConnectionProfile buildConnectionProfileFromSettings(Settings settings, String clusterName) {
return new ConnectionProfile.Builder()
.setConnectTimeout(TransportSettings.CONNECT_TIMEOUT.get(settings))
.setHandshakeTimeout(TransportSettings.CONNECT_TIMEOUT.get(settings))
.addConnections(6, TransportRequestOptions.Type.REG, TransportRequestOptions.Type.PING) // TODO make this configurable?
// we don't want this to be used for anything else but search
.addConnections(0, TransportRequestOptions.Type.BULK,
TransportRequestOptions.Type.STATE,
TransportRequestOptions.Type.RECOVERY)
.setCompressionEnabled(REMOTE_CLUSTER_COMPRESS.getConcreteSettingForNamespace(clusterName).get(settings))
.setPingInterval(REMOTE_CLUSTER_PING_SCHEDULE.getConcreteSettingForNamespace(clusterName).get(settings))
.build();
}
@Override
public void close() throws IOException {
IOUtils.close(remoteClusters.values());
@ -468,21 +396,6 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
return remoteClusters.values().stream().map(RemoteClusterConnection::getConnectionInfo);
}
private boolean connectionProfileChanged(ConnectionProfile oldProfile, ConnectionProfile newProfile) {
return Objects.equals(oldProfile.getCompressionEnabled(), newProfile.getCompressionEnabled()) == false
|| Objects.equals(oldProfile.getPingInterval(), newProfile.getPingInterval()) == false;
}
private boolean seedsChanged(final List<Tuple<String, Supplier<DiscoveryNode>>> oldSeedNodes,
final List<Tuple<String, Supplier<DiscoveryNode>>> newSeedNodes) {
if (oldSeedNodes.size() != newSeedNodes.size()) {
return true;
}
Set<String> oldSeeds = oldSeedNodes.stream().map(Tuple::v1).collect(Collectors.toSet());
Set<String> newSeeds = newSeedNodes.stream().map(Tuple::v1).collect(Collectors.toSet());
return oldSeeds.equals(newSeeds) == false;
}
/**
* Collects all nodes of the given clusters and returns / passes a (clusterAlias, nodeId) to {@link DiscoveryNode}
* function on success.
@ -509,7 +422,7 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
}
if (countDown.countDown()) {
listener.onResponse((clusterAlias, nodeId)
-> clusterMap.getOrDefault(clusterAlias, nullFunction).apply(nodeId));
-> clusterMap.getOrDefault(clusterAlias, nullFunction).apply(nodeId));
}
}
@ -525,9 +438,9 @@ public final class RemoteClusterService extends RemoteClusterAware implements Cl
/**
* Returns a client to the remote cluster if the given cluster alias exists.
* @param threadPool the {@link ThreadPool} for the client
* @param clusterAlias the cluster alias the remote cluster is registered under
*
* @param threadPool the {@link ThreadPool} for the client
* @param clusterAlias the cluster alias the remote cluster is registered under
* @throws IllegalArgumentException if the given clusterAlias doesn't exist
*/
public Client getRemoteClusterClient(ThreadPool threadPool, String clusterAlias) {

View File

@ -0,0 +1,225 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Objects;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
public abstract class RemoteConnectionStrategy implements TransportConnectionListener, Closeable {
enum ConnectionStrategy {
SNIFF,
SIMPLE
}
public static final Setting.AffixSetting<ConnectionStrategy> REMOTE_CONNECTION_MODE = Setting.affixKeySetting(
"cluster.remote.", "mode", key -> new Setting<>(
key,
ConnectionStrategy.SNIFF.name(),
value -> ConnectionStrategy.valueOf(value.toUpperCase(Locale.ROOT)),
Setting.Property.Dynamic));
private static final Logger logger = LogManager.getLogger(RemoteConnectionStrategy.class);
private static final int MAX_LISTENERS = 100;
private final AtomicBoolean closed = new AtomicBoolean(false);
private final Object mutex = new Object();
private List<ActionListener<Void>> listeners = new ArrayList<>();
protected final TransportService transportService;
protected final RemoteConnectionManager connectionManager;
protected final String clusterAlias;
RemoteConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager) {
this.clusterAlias = clusterAlias;
this.transportService = transportService;
this.connectionManager = connectionManager;
connectionManager.getConnectionManager().addListener(this);
}
/**
* Triggers a connect round unless there is one running already. If there is a connect round running, the listener will either
* be queued or rejected and failed.
*/
void connect(ActionListener<Void> connectListener) {
boolean runConnect = false;
final ActionListener<Void> listener =
ContextPreservingActionListener.wrapPreservingContext(connectListener, transportService.getThreadPool().getThreadContext());
boolean closed;
synchronized (mutex) {
closed = this.closed.get();
if (closed) {
assert listeners.isEmpty();
} else {
if (listeners.size() >= MAX_LISTENERS) {
assert listeners.size() == MAX_LISTENERS;
listener.onFailure(new RejectedExecutionException("connect listener queue is full"));
return;
} else {
listeners.add(listener);
}
runConnect = listeners.size() == 1;
}
}
if (closed) {
connectListener.onFailure(new AlreadyClosedException("connect handler is already closed"));
return;
}
if (runConnect) {
ExecutorService executor = transportService.getThreadPool().executor(ThreadPool.Names.MANAGEMENT);
executor.submit(new AbstractRunnable() {
@Override
public void onFailure(Exception e) {
ActionListener.onFailure(getAndClearListeners(), e);
}
@Override
protected void doRun() {
connectImpl(new ActionListener<Void>() {
@Override
public void onResponse(Void aVoid) {
ActionListener.onResponse(getAndClearListeners(), aVoid);
}
@Override
public void onFailure(Exception e) {
ActionListener.onFailure(getAndClearListeners(), e);
}
});
}
});
}
}
public static boolean isConnectionEnabled(String clusterAlias, Settings settings) {
ConnectionStrategy mode = REMOTE_CONNECTION_MODE.getConcreteSettingForNamespace(clusterAlias).get(settings);
if (mode.equals(ConnectionStrategy.SNIFF)) {
List<String> seeds = RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).get(settings);
return seeds.isEmpty() == false;
} else {
return false;
}
}
boolean shouldRebuildConnection(Settings newSettings) {
ConnectionStrategy newMode = REMOTE_CONNECTION_MODE.getConcreteSettingForNamespace(clusterAlias).get(newSettings);
if (newMode.equals(strategyType()) == false) {
return true;
} else {
Boolean compressionEnabled = RemoteClusterService.REMOTE_CLUSTER_COMPRESS
.getConcreteSettingForNamespace(clusterAlias)
.get(newSettings);
TimeValue pingSchedule = RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE
.getConcreteSettingForNamespace(clusterAlias)
.get(newSettings);
ConnectionProfile oldProfile = connectionManager.getConnectionManager().getConnectionProfile();
ConnectionProfile.Builder builder = new ConnectionProfile.Builder(oldProfile);
builder.setCompressionEnabled(compressionEnabled);
builder.setPingInterval(pingSchedule);
ConnectionProfile newProfile = builder.build();
return connectionProfileChanged(oldProfile, newProfile) || strategyMustBeRebuilt(newSettings);
}
}
protected abstract boolean strategyMustBeRebuilt(Settings newSettings);
protected abstract ConnectionStrategy strategyType();
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
if (shouldOpenMoreConnections()) {
// try to reconnect and fill up the slot of the disconnected node
connect(ActionListener.wrap(
ignore -> logger.trace("successfully connected after disconnect of {}", node),
e -> logger.trace(() -> new ParameterizedMessage("failed to connect after disconnect of {}", node), e)));
}
}
@Override
public void close() {
final List<ActionListener<Void>> toNotify;
synchronized (mutex) {
if (closed.compareAndSet(false, true)) {
connectionManager.getConnectionManager().removeListener(this);
toNotify = listeners;
listeners = Collections.emptyList();
} else {
toNotify = Collections.emptyList();
}
}
ActionListener.onFailure(toNotify, new AlreadyClosedException("connect handler is already closed"));
}
public boolean isClosed() {
return closed.get();
}
// for testing only
boolean assertNoRunningConnections() {
synchronized (mutex) {
assert listeners.isEmpty();
}
return true;
}
protected abstract boolean shouldOpenMoreConnections();
protected abstract void connectImpl(ActionListener<Void> listener);
private List<ActionListener<Void>> getAndClearListeners() {
final List<ActionListener<Void>> result;
synchronized (mutex) {
if (listeners.isEmpty()) {
result = Collections.emptyList();
} else {
result = listeners;
listeners = new ArrayList<>();
}
}
return result;
}
private boolean connectionProfileChanged(ConnectionProfile oldProfile, ConnectionProfile newProfile) {
return Objects.equals(oldProfile.getCompressionEnabled(), newProfile.getCompressionEnabled()) == false
|| Objects.equals(oldProfile.getPingInterval(), newProfile.getPingInterval()) == false;
}
}

View File

@ -0,0 +1,172 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.util.concurrent.CountDown;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import java.util.stream.Collectors;
public class SimpleConnectionStrategy extends RemoteConnectionStrategy {
private static final int MAX_CONNECT_ATTEMPTS_PER_RUN = 3;
private static final Logger logger = LogManager.getLogger(SimpleConnectionStrategy.class);
private final int maxNumRemoteConnections;
private final AtomicLong counter = new AtomicLong(0);
private final List<Supplier<TransportAddress>> addresses;
private final AtomicReference<ClusterName> remoteClusterName = new AtomicReference<>();
private final ConnectionProfile profile;
private final ConnectionManager.ConnectionValidator clusterNameValidator;
SimpleConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager,
int maxNumRemoteConnections, List<Supplier<TransportAddress>> addresses) {
super(clusterAlias, transportService, connectionManager);
this.maxNumRemoteConnections = maxNumRemoteConnections;
assert addresses.isEmpty() == false : "Cannot use simple connection strategy with no configured addresses";
this.addresses = addresses;
// TODO: Move into the ConnectionManager
this.profile = new ConnectionProfile.Builder()
.addConnections(1, TransportRequestOptions.Type.REG, TransportRequestOptions.Type.PING)
.addConnections(0, TransportRequestOptions.Type.BULK, TransportRequestOptions.Type.STATE, TransportRequestOptions.Type.RECOVERY)
.build();
this.clusterNameValidator = (newConnection, actualProfile, listener) ->
transportService.handshake(newConnection, actualProfile.getHandshakeTimeout().millis(), cn -> true,
ActionListener.map(listener, resp -> {
ClusterName remote = resp.getClusterName();
if (remoteClusterName.compareAndSet(null, remote)) {
return null;
} else {
if (remoteClusterName.get().equals(remote) == false) {
DiscoveryNode node = newConnection.getNode();
throw new ConnectTransportException(node, "handshake failed. unexpected remote cluster name " + remote);
}
return null;
}
}));
}
@Override
protected boolean shouldOpenMoreConnections() {
return connectionManager.size() < maxNumRemoteConnections;
}
@Override
protected boolean strategyMustBeRebuilt(Settings newSettings) {
return false;
}
@Override
protected ConnectionStrategy strategyType() {
return ConnectionStrategy.SIMPLE;
}
@Override
protected void connectImpl(ActionListener<Void> listener) {
performSimpleConnectionProcess(addresses.iterator(), listener);
}
private void performSimpleConnectionProcess(Iterator<Supplier<TransportAddress>> addressIter, ActionListener<Void> listener) {
openConnections(listener, 1);
}
private void openConnections(ActionListener<Void> finished, int attemptNumber) {
if (attemptNumber <= MAX_CONNECT_ATTEMPTS_PER_RUN) {
List<TransportAddress> resolved = addresses.stream().map(Supplier::get).collect(Collectors.toList());
int remaining = maxNumRemoteConnections - connectionManager.size();
ActionListener<Void> compositeListener = new ActionListener<Void>() {
private final AtomicInteger successfulConnections = new AtomicInteger(0);
private final CountDown countDown = new CountDown(remaining);
@Override
public void onResponse(Void v) {
successfulConnections.incrementAndGet();
if (countDown.countDown()) {
if (shouldOpenMoreConnections()) {
openConnections(finished, attemptNumber + 1);
} else {
finished.onResponse(v);
}
}
}
@Override
public void onFailure(Exception e) {
if (countDown.countDown()) {
openConnections(finished, attemptNumber + 1);
}
}
};
for (int i = 0; i < remaining; ++i) {
TransportAddress address = nextAddress(resolved);
String id = clusterAlias + "#" + address;
DiscoveryNode node = new DiscoveryNode(id, address, Version.CURRENT.minimumCompatibilityVersion());
connectionManager.connectToNode(node, profile, clusterNameValidator, new ActionListener<Void>() {
@Override
public void onResponse(Void v) {
compositeListener.onResponse(v);
}
@Override
public void onFailure(Exception e) {
logger.debug(new ParameterizedMessage("failed to open remote connection [remote cluster: {}, address: {}]",
clusterAlias, address), e);
compositeListener.onFailure(e);
}
});
}
} else {
int openConnections = connectionManager.size();
if (openConnections == 0) {
finished.onFailure(new IllegalStateException("Unable to open any simple connections to remote cluster [" + clusterAlias
+ "]"));
} else {
logger.debug("unable to open maximum number of connections [remote cluster: {}, opened: {}, maximum: {}]", clusterAlias,
openConnections, maxNumRemoteConnections);
finished.onResponse(null);
}
}
}
private TransportAddress nextAddress(List<TransportAddress> resolvedAddresses) {
long curr;
while ((curr = counter.getAndIncrement()) == Long.MIN_VALUE) ;
return resolvedAddresses.get(Math.toIntExact(Math.floorMod(curr, (long) resolvedAddresses.size())));
}
}

View File

@ -0,0 +1,378 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.util.SetOnce;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.StepListener;
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.function.Consumer;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
public class SniffConnectionStrategy extends RemoteConnectionStrategy {
private static final Logger logger = LogManager.getLogger(SniffConnectionStrategy.class);
private static final Predicate<DiscoveryNode> DEFAULT_NODE_PREDICATE = (node) -> Version.CURRENT.isCompatible(node.getVersion())
&& (node.isMasterNode() == false || node.isDataNode() || node.isIngestNode());
private final List<String> configuredSeedNodes;
private final List<Supplier<DiscoveryNode>> seedNodes;
private final int maxNumRemoteConnections;
private final Predicate<DiscoveryNode> nodePredicate;
private final SetOnce<ClusterName> remoteClusterName = new SetOnce<>();
private volatile String proxyAddress;
SniffConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager,
Settings settings) {
this(
clusterAlias,
transportService,
connectionManager,
RemoteClusterAware.REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterAlias).get(settings),
RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER.get(settings),
getNodePredicate(settings),
RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).get(settings));
}
SniffConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager,
String proxyAddress, int maxNumRemoteConnections, Predicate<DiscoveryNode> nodePredicate,
List<String> configuredSeedNodes) {
this(clusterAlias, transportService, connectionManager, proxyAddress, maxNumRemoteConnections, nodePredicate, configuredSeedNodes,
configuredSeedNodes.stream().map(seedAddress ->
(Supplier<DiscoveryNode>) () -> resolveSeedNode(clusterAlias, seedAddress, proxyAddress)).collect(Collectors.toList()));
}
SniffConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager,
String proxyAddress, int maxNumRemoteConnections, Predicate<DiscoveryNode> nodePredicate,
List<String> configuredSeedNodes, List<Supplier<DiscoveryNode>> seedNodes) {
super(clusterAlias, transportService, connectionManager);
this.proxyAddress = proxyAddress;
this.maxNumRemoteConnections = maxNumRemoteConnections;
this.nodePredicate = nodePredicate;
this.configuredSeedNodes = configuredSeedNodes;
this.seedNodes = seedNodes;
}
@Override
protected boolean shouldOpenMoreConnections() {
return connectionManager.size() < maxNumRemoteConnections;
}
@Override
protected boolean strategyMustBeRebuilt(Settings newSettings) {
String proxy = RemoteClusterAware.REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterAlias).get(newSettings);
List<String> addresses = RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).get(newSettings);
return seedsChanged(configuredSeedNodes, addresses) || proxyChanged(proxyAddress, proxy);
}
@Override
protected ConnectionStrategy strategyType() {
return ConnectionStrategy.SNIFF;
}
@Override
protected void connectImpl(ActionListener<Void> listener) {
collectRemoteNodes(seedNodes.iterator(), listener);
}
private void collectRemoteNodes(Iterator<Supplier<DiscoveryNode>> seedNodes, ActionListener<Void> listener) {
if (Thread.currentThread().isInterrupted()) {
listener.onFailure(new InterruptedException("remote connect thread got interrupted"));
return;
}
if (seedNodes.hasNext()) {
final Consumer<Exception> onFailure = e -> {
if (e instanceof ConnectTransportException ||
e instanceof IOException ||
e instanceof IllegalStateException) {
// ISE if we fail the handshake with an version incompatible node
if (seedNodes.hasNext()) {
logger.debug(() -> new ParameterizedMessage(
"fetching nodes from external cluster [{}] failed moving to next node", clusterAlias), e);
collectRemoteNodes(seedNodes, listener);
return;
}
}
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster [{}] failed", clusterAlias), e);
listener.onFailure(e);
};
final DiscoveryNode seedNode = seedNodes.next().get();
logger.debug("[{}] opening connection to seed node: [{}] proxy address: [{}]", clusterAlias, seedNode,
proxyAddress);
final ConnectionProfile profile = ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG);
final StepListener<Transport.Connection> openConnectionStep = new StepListener<>();
try {
connectionManager.openConnection(seedNode, profile, openConnectionStep);
} catch (Exception e) {
onFailure.accept(e);
}
final StepListener<TransportService.HandshakeResponse> handshakeStep = new StepListener<>();
openConnectionStep.whenComplete(connection -> {
ConnectionProfile connectionProfile = connectionManager.getConnectionManager().getConnectionProfile();
transportService.handshake(connection, connectionProfile.getHandshakeTimeout().millis(),
getRemoteClusterNamePredicate(), handshakeStep);
}, onFailure);
final StepListener<Void> fullConnectionStep = new StepListener<>();
handshakeStep.whenComplete(handshakeResponse -> {
final DiscoveryNode handshakeNode = maybeAddProxyAddress(proxyAddress, handshakeResponse.getDiscoveryNode());
if (nodePredicate.test(handshakeNode) && shouldOpenMoreConnections()) {
connectionManager.connectToNode(handshakeNode, null,
transportService.connectionValidator(handshakeNode), fullConnectionStep);
} else {
fullConnectionStep.onResponse(null);
}
}, e -> {
final Transport.Connection connection = openConnectionStep.result();
logger.warn(new ParameterizedMessage("failed to connect to seed node [{}]", connection.getNode()), e);
IOUtils.closeWhileHandlingException(connection);
onFailure.accept(e);
});
fullConnectionStep.whenComplete(aVoid -> {
if (remoteClusterName.get() == null) {
TransportService.HandshakeResponse handshakeResponse = handshakeStep.result();
assert handshakeResponse.getClusterName().value() != null;
remoteClusterName.set(handshakeResponse.getClusterName());
}
final Transport.Connection connection = openConnectionStep.result();
ClusterStateRequest request = new ClusterStateRequest();
request.clear();
request.nodes(true);
// here we pass on the connection since we can only close it once the sendRequest returns otherwise
// due to the async nature (it will return before it's actually sent) this can cause the request to fail
// due to an already closed connection.
ThreadPool threadPool = transportService.getThreadPool();
ThreadContext threadContext = threadPool.getThreadContext();
TransportService.ContextRestoreResponseHandler<ClusterStateResponse> responseHandler = new TransportService
.ContextRestoreResponseHandler<>(threadContext.newRestorableContext(false),
new SniffClusterStateResponseHandler(connection, listener, seedNodes));
try (ThreadContext.StoredContext ignore = threadContext.stashContext()) {
// we stash any context here since this is an internal execution and should not leak any
// existing context information.
threadContext.markAsSystemContext();
transportService.sendRequest(connection, ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY,
responseHandler);
}
}, e -> {
IOUtils.closeWhileHandlingException(openConnectionStep.result());
onFailure.accept(e);
});
} else {
listener.onFailure(new IllegalStateException("no seed node left"));
}
}
List<String> getSeedNodes() {
return configuredSeedNodes;
}
int getMaxConnections() {
return maxNumRemoteConnections;
}
/* This class handles the _state response from the remote cluster when sniffing nodes to connect to */
private class SniffClusterStateResponseHandler implements TransportResponseHandler<ClusterStateResponse> {
private final Transport.Connection connection;
private final ActionListener<Void> listener;
private final Iterator<Supplier<DiscoveryNode>> seedNodes;
SniffClusterStateResponseHandler(Transport.Connection connection, ActionListener<Void> listener,
Iterator<Supplier<DiscoveryNode>> seedNodes) {
this.connection = connection;
this.listener = listener;
this.seedNodes = seedNodes;
}
@Override
public ClusterStateResponse read(StreamInput in) throws IOException {
return new ClusterStateResponse(in);
}
@Override
public void handleResponse(ClusterStateResponse response) {
handleNodes(response.getState().nodes().getNodes().valuesIt());
}
private void handleNodes(Iterator<DiscoveryNode> nodesIter) {
while (nodesIter.hasNext()) {
final DiscoveryNode node = maybeAddProxyAddress(proxyAddress, nodesIter.next());
if (nodePredicate.test(node) && shouldOpenMoreConnections()) {
connectionManager.connectToNode(node, null,
transportService.connectionValidator(node), new ActionListener<Void>() {
@Override
public void onResponse(Void aVoid) {
handleNodes(nodesIter);
}
@Override
public void onFailure(Exception e) {
if (e instanceof ConnectTransportException ||
e instanceof IllegalStateException) {
// ISE if we fail the handshake with an version incompatible node
// fair enough we can't connect just move on
logger.debug(() -> new ParameterizedMessage("failed to connect to node {}", node), e);
handleNodes(nodesIter);
} else {
logger.warn(() ->
new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), e);
IOUtils.closeWhileHandlingException(connection);
collectRemoteNodes(seedNodes, listener);
}
}
});
return;
}
}
// We have to close this connection before we notify listeners - this is mainly needed for test correctness
// since if we do it afterwards we might fail assertions that check if all high level connections are closed.
// from a code correctness perspective we could also close it afterwards.
IOUtils.closeWhileHandlingException(connection);
listener.onResponse(null);
}
@Override
public void handleException(TransportException exp) {
logger.warn(() -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), exp);
try {
IOUtils.closeWhileHandlingException(connection);
} finally {
// once the connection is closed lets try the next node
collectRemoteNodes(seedNodes, listener);
}
}
@Override
public String executor() {
return ThreadPool.Names.MANAGEMENT;
}
}
private Predicate<ClusterName> getRemoteClusterNamePredicate() {
return new Predicate<ClusterName>() {
@Override
public boolean test(ClusterName c) {
return remoteClusterName.get() == null || c.equals(remoteClusterName.get());
}
@Override
public String toString() {
return remoteClusterName.get() == null ? "any cluster name"
: "expected remote cluster name [" + remoteClusterName.get().value() + "]";
}
};
}
private static DiscoveryNode resolveSeedNode(String clusterAlias, String address, String proxyAddress) {
if (proxyAddress == null || proxyAddress.isEmpty()) {
TransportAddress transportAddress = new TransportAddress(RemoteClusterAware.parseSeedAddress(address));
return new DiscoveryNode(clusterAlias + "#" + transportAddress.toString(), transportAddress,
Version.CURRENT.minimumCompatibilityVersion());
} else {
TransportAddress transportAddress = new TransportAddress(RemoteClusterAware.parseSeedAddress(proxyAddress));
String hostName = address.substring(0, indexOfPortSeparator(address));
return new DiscoveryNode("", clusterAlias + "#" + address, UUIDs.randomBase64UUID(), hostName, address,
transportAddress, Collections.singletonMap("server_name", hostName), DiscoveryNodeRole.BUILT_IN_ROLES,
Version.CURRENT.minimumCompatibilityVersion());
}
}
// Default visibility for tests
static Predicate<DiscoveryNode> getNodePredicate(Settings settings) {
if (RemoteClusterService.REMOTE_NODE_ATTRIBUTE.exists(settings)) {
// nodes can be tagged with node.attr.remote_gateway: true to allow a node to be a gateway node for cross cluster search
String attribute = RemoteClusterService.REMOTE_NODE_ATTRIBUTE.get(settings);
return DEFAULT_NODE_PREDICATE.and((node) -> Booleans.parseBoolean(node.getAttributes().getOrDefault(attribute, "false")));
}
return DEFAULT_NODE_PREDICATE;
}
private static int indexOfPortSeparator(String remoteHost) {
int portSeparator = remoteHost.lastIndexOf(':'); // in case we have a IPv6 address ie. [::1]:9300
if (portSeparator == -1 || portSeparator == remoteHost.length()) {
throw new IllegalArgumentException("remote hosts need to be configured as [host:port], found [" + remoteHost + "] instead");
}
return portSeparator;
}
private static DiscoveryNode maybeAddProxyAddress(String proxyAddress, DiscoveryNode node) {
if (proxyAddress == null || proxyAddress.isEmpty()) {
return node;
} else {
// resolve proxy address lazy here
InetSocketAddress proxyInetAddress = RemoteClusterAware.parseSeedAddress(proxyAddress);
return new DiscoveryNode(node.getName(), node.getId(), node.getEphemeralId(), node.getHostName(), node
.getHostAddress(), new TransportAddress(proxyInetAddress), node.getAttributes(), node.getRoles(), node.getVersion());
}
}
private boolean seedsChanged(final List<String> oldSeedNodes, final List<String> newSeedNodes) {
if (oldSeedNodes.size() != newSeedNodes.size()) {
return true;
}
Set<String> oldSeeds = new HashSet<>(oldSeedNodes);
Set<String> newSeeds = new HashSet<>(newSeedNodes);
return oldSeeds.equals(newSeeds) == false;
}
private boolean proxyChanged(String oldProxy, String newProxy) {
if (oldProxy == null || oldProxy.isEmpty()) {
return (newProxy == null || newProxy.isEmpty()) == false;
}
return Objects.equals(oldProxy, newProxy) == false;
}
}

View File

@ -22,7 +22,6 @@ import org.apache.lucene.search.TotalHits;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction;
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup;
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest;
@ -41,7 +40,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
@ -60,7 +58,6 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.test.transport.StubbableConnectionManager;
import org.elasticsearch.test.transport.StubbableTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
@ -73,40 +70,28 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static java.util.Collections.emptyMap;
import static java.util.Collections.emptySet;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.iterableWithSize;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.sameInstance;
import static org.hamcrest.Matchers.startsWith;
import static org.mockito.Mockito.mock;
public class RemoteClusterConnectionTests extends ESTestCase {
private final ThreadPool threadPool = new TestThreadPool(getClass().getName());
private final ConnectionProfile profile = RemoteClusterService.buildConnectionProfileFromSettings(Settings.EMPTY, "cluster");
@Override
public void tearDown() throws Exception {
@ -150,7 +135,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
SearchHits searchHits;
if ("null_target".equals(request.preference())) {
searchHits = new SearchHits(new SearchHit[] {new SearchHit(0)}, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1F);
searchHits = new SearchHits(new SearchHit[]{new SearchHit(0)}, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1F);
} else {
searchHits = new SearchHits(new SearchHit[0], new TotalHits(0, TotalHits.Relation.EQUAL_TO), Float.NaN);
}
@ -180,345 +165,6 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
}
public void testRemoteProfileIsUsedForLocalCluster() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
Arrays.asList(Tuple.tuple(seedNode.toString(), () -> seedNode)), service, Integer.MAX_VALUE, n -> true, null,
profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes(seedNode));
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
PlainTransportFuture<ClusterSearchShardsResponse> futureHandler = new PlainTransportFuture<>(
new FutureTransportResponseHandler<ClusterSearchShardsResponse>() {
@Override
public ClusterSearchShardsResponse read(StreamInput in) throws IOException {
return new ClusterSearchShardsResponse(in);
}
});
TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.BULK)
.build();
IllegalStateException ise = (IllegalStateException) expectThrows(SendRequestTransportException.class, () -> {
service.sendRequest(connectionManager.getConnection(discoverableNode),
ClusterSearchShardsAction.NAME, new ClusterSearchShardsRequest(), options, futureHandler);
futureHandler.txGet();
}).getCause();
assertEquals(ise.getMessage(), "can't select channel size is 0 for types: [RECOVERY, BULK, STATE]");
}
}
}
}
public void testRemoteProfileIsUsedForRemoteCluster() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT, threadPool,
Settings.builder().put("cluster.name", "foobar").build());
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT,
threadPool, Settings.builder().put("cluster.name", "foobar").build())) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
Arrays.asList(Tuple.tuple(seedNode.toString(), () -> seedNode)), service, Integer.MAX_VALUE, n -> true, null,
profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes(seedNode));
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
PlainTransportFuture<ClusterSearchShardsResponse> futureHandler = new PlainTransportFuture<>(
new FutureTransportResponseHandler<ClusterSearchShardsResponse>() {
@Override
public ClusterSearchShardsResponse read(StreamInput in) throws IOException {
return new ClusterSearchShardsResponse(in);
}
});
TransportRequestOptions options = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.BULK)
.build();
IllegalStateException ise = (IllegalStateException) expectThrows(SendRequestTransportException.class, () -> {
service.sendRequest(connectionManager.getConnection(discoverableNode),
ClusterSearchShardsAction.NAME, new ClusterSearchShardsRequest(), options, futureHandler);
futureHandler.txGet();
}).getCause();
assertEquals(ise.getMessage(), "can't select channel size is 0 for types: [RECOVERY, BULK, STATE]");
PlainTransportFuture<ClusterSearchShardsResponse> handler = new PlainTransportFuture<>(
new FutureTransportResponseHandler<ClusterSearchShardsResponse>() {
@Override
public ClusterSearchShardsResponse read(StreamInput in) throws IOException {
return new ClusterSearchShardsResponse(in);
}
});
TransportRequestOptions ops = TransportRequestOptions.builder().withType(TransportRequestOptions.Type.REG)
.build();
service.sendRequest(connection.getConnection(), ClusterSearchShardsAction.NAME, new ClusterSearchShardsRequest(),
ops, handler);
handler.txGet();
}
}
}
}
public void testDiscoverSingleNode() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
Arrays.asList(Tuple.tuple(seedNode.toString(), () -> seedNode)), service, Integer.MAX_VALUE, n -> true, null,
profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes(seedNode));
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
}
}
}
}
public void testDiscoverSingleNodeWithIncompatibleSeed() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService incompatibleTransport = startTransport("incompat_seed_node", knownNodes, Version.fromString("2.0.0"));
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
DiscoveryNode incompatibleSeedNode = incompatibleTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
knownNodes.add(incompatibleTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes = Arrays.asList(
Tuple.tuple(incompatibleSeedNode.toString(), () -> incompatibleSeedNode),
Tuple.tuple(seedNode.toString(), () -> seedNode));
Collections.shuffle(seedNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes, service, Integer.MAX_VALUE, n -> true, null, profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes);
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertFalse(connectionManager.nodeConnected(incompatibleSeedNode));
assertTrue(connection.assertNoRunningConnections());
}
}
}
}
public void testNodeDisconnected() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT);
MockTransportService spareTransport = startTransport("spare_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
DiscoveryNode spareNode = spareTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> true, null, profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes(seedNode));
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertFalse(connectionManager.nodeConnected(spareNode));
knownNodes.add(spareNode);
CountDownLatch latchDisconnect = new CountDownLatch(1);
CountDownLatch latchConnected = new CountDownLatch(1);
connectionManager.addListener(new TransportConnectionListener() {
@Override
public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connection) {
if (node.equals(discoverableNode)) {
latchDisconnect.countDown();
}
}
@Override
public void onNodeConnected(DiscoveryNode node, Transport.Connection connection) {
if (node.equals(spareNode)) {
latchConnected.countDown();
}
}
});
discoverableTransport.close();
// now make sure we try to connect again to other nodes once we got disconnected
assertTrue(latchDisconnect.await(10, TimeUnit.SECONDS));
assertTrue(latchConnected.await(10, TimeUnit.SECONDS));
assertTrue(connectionManager.nodeConnected(spareNode));
}
}
}
}
public void testFilterDiscoveredNodes() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
DiscoveryNode rejectedNode = randomBoolean() ? seedNode : discoverableNode;
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> n.equals(rejectedNode) == false, null, profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes(seedNode));
if (rejectedNode.equals(seedNode)) {
assertFalse(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
} else {
assertTrue(connectionManager.nodeConnected(seedNode));
assertFalse(connectionManager.nodeConnected(discoverableNode));
}
assertTrue(connection.assertNoRunningConnections());
}
}
}
}
private void updateSeedNodes(
final RemoteClusterConnection connection, final List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes) throws Exception {
updateSeedNodes(connection, seedNodes, null);
}
private void updateSeedNodes(
final RemoteClusterConnection connection,
final List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes,
final String proxyAddress)
throws Exception {
CountDownLatch latch = new CountDownLatch(1);
AtomicReference<Exception> exceptionAtomicReference = new AtomicReference<>();
ActionListener<Void> listener = ActionListener.wrap(
x -> latch.countDown(),
x -> {
exceptionAtomicReference.set(x);
latch.countDown();
}
);
connection.updateSeedNodes(proxyAddress, seedNodes, listener);
latch.await();
if (exceptionAtomicReference.get() != null) {
throw exceptionAtomicReference.get();
}
}
public void testConnectWithIncompatibleTransports() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.fromString("2.0.0"))) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
Arrays.asList(Tuple.tuple(seedNode.toString(), () -> seedNode)), service, Integer.MAX_VALUE, n -> true, null,
profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
expectThrows(
Exception.class,
() -> updateSeedNodes(connection, Arrays.asList(Tuple.tuple(seedNode.toString(), () -> seedNode))));
assertFalse(connectionManager.nodeConnected(seedNode));
assertTrue(connection.assertNoRunningConnections());
}
}
}
}
public void testRemoteConnectionVersionMatchesTransportConnectionVersion() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
final Version previousVersion = randomValueOtherThan(Version.CURRENT, () -> VersionUtils.randomVersionBetween(random(),
Version.CURRENT.minimumCompatibilityVersion(), Version.CURRENT));
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, previousVersion)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
assertThat(seedNode, notNullValue());
knownNodes.add(seedNode);
DiscoveryNode oldVersionNode = discoverableTransport.getLocalDiscoNode();
assertThat(oldVersionNode, notNullValue());
knownNodes.add(oldVersionNode);
assertThat(seedNode.getVersion(), not(equalTo(oldVersionNode.getVersion())));
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
final Transport.Connection seedConnection = new CloseableConnection() {
@Override
public DiscoveryNode getNode() {
return seedNode;
}
@Override
public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options)
throws IOException, TransportException {
// no-op
}
};
ConnectionManager delegate = new ConnectionManager(Settings.EMPTY, service.transport);
StubbableConnectionManager connectionManager = new StubbableConnectionManager(delegate, Settings.EMPTY, service.transport);
connectionManager.addGetConnectionBehavior(seedNode.getAddress(), (cm, discoveryNode) -> {
if (discoveryNode == seedNode) {
return seedConnection;
}
return cm.getConnection(discoveryNode);
});
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> true, null, connectionManager)) {
PlainActionFuture.get(fut -> connection.ensureConnected(ActionListener.map(fut, x -> null)));
assertThat(knownNodes, iterableWithSize(2));
assertThat(connection.getConnection(seedNode).getVersion(), equalTo(Version.CURRENT));
assertThat(connection.getConnection(oldVersionNode).getVersion(), equalTo(previousVersion));
}
}
}
}
@SuppressForbidden(reason = "calls getLocalHost here but it's fine in this case")
public void testSlowNodeCanBeCancelled() throws IOException, InterruptedException {
try (ServerSocket socket = new MockServerSocket()) {
@ -549,8 +195,9 @@ public class RemoteClusterConnectionTests extends ESTestCase {
service.acceptIncomingRequests();
CountDownLatch listenerCalled = new CountDownLatch(1);
AtomicReference<Exception> exceptionReference = new AtomicReference<>();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> true, null, profile)) {
String clusterAlias = "test-cluster";
Settings settings = buildSniffSettings(clusterAlias, seedNodes(seedNode));
try (RemoteClusterConnection connection = new RemoteClusterConnection(settings, clusterAlias, service)) {
ActionListener<Void> listener = ActionListener.wrap(x -> {
listenerCalled.countDown();
fail("expected exception");
@ -558,7 +205,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
exceptionReference.set(x);
listenerCalled.countDown();
});
connection.updateSeedNodes(null, seedNodes(seedNode), listener);
connection.ensureConnected(listener);
acceptedLatch.await();
connection.close(); // now close it, this should trigger an interrupt on the socket and we can move on
assertTrue(connection.assertNoRunningConnections());
@ -574,86 +221,8 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
}
private static List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes(final DiscoveryNode... seedNodes) {
if (seedNodes.length == 0) {
return Collections.emptyList();
} else if (seedNodes.length == 1) {
return Collections.singletonList(Tuple.tuple(seedNodes[0].toString(), () -> seedNodes[0]));
} else {
return Arrays.stream(seedNodes)
.map(s -> Tuple.tuple(s.toString(), (Supplier<DiscoveryNode>)() -> s))
.collect(Collectors.toList());
}
}
public void testTriggerUpdatesConcurrently() throws IOException, InterruptedException {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService seedTransport1 = startTransport("seed_node_1", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
DiscoveryNode seedNode1 = seedTransport1.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
knownNodes.add(seedTransport1.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes = seedNodes(seedNode1, seedNode);
Collections.shuffle(seedNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes, service, Integer.MAX_VALUE, n -> true, null, profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
int numThreads = randomIntBetween(4, 10);
Thread[] threads = new Thread[numThreads];
CyclicBarrier barrier = new CyclicBarrier(numThreads);
for (int i = 0; i < threads.length; i++) {
final int numConnectionAttempts = randomIntBetween(10, 200);
threads[i] = new Thread() {
@Override
public void run() {
try {
barrier.await();
CountDownLatch latch = new CountDownLatch(numConnectionAttempts);
for (int i = 0; i < numConnectionAttempts; i++) {
AtomicBoolean executed = new AtomicBoolean(false);
ActionListener<Void> listener = ActionListener.wrap(
x -> {
assertTrue(executed.compareAndSet(false, true));
latch.countDown();
},
x -> {
assertTrue(executed.compareAndSet(false, true));
latch.countDown();
if (!(x instanceof RejectedExecutionException)) {
throw new AssertionError(x);
}
});
connection.updateSeedNodes(null, seedNodes, listener);
}
latch.await();
} catch (Exception ex) {
throw new AssertionError(ex);
}
}
};
threads[i].start();
}
for (int i = 0; i < threads.length; i++) {
threads[i].join();
}
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connectionManager.nodeConnected(seedNode1));
assertTrue(connection.assertNoRunningConnections());
}
}
}
private static List<String> seedNodes(final DiscoveryNode... seedNodes) {
return Arrays.stream(seedNodes).map(s -> s.getAddress().toString()).collect(Collectors.toList());
}
public void testCloseWhileConcurrentlyConnecting() throws IOException, InterruptedException, BrokenBarrierException {
@ -667,14 +236,15 @@ public class RemoteClusterConnectionTests extends ESTestCase {
knownNodes.add(discoverableTransport.getLocalDiscoNode());
knownNodes.add(seedTransport1.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes = seedNodes(seedNode1, seedNode);
List<String> seedNodes = seedNodes(seedNode1, seedNode);
Collections.shuffle(seedNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes, service, Integer.MAX_VALUE, n -> true, null, profile)) {
String clusterAlias = "test-cluster";
Settings settings = buildSniffSettings(clusterAlias, seedNodes);
try (RemoteClusterConnection connection = new RemoteClusterConnection(settings, clusterAlias, service)) {
int numThreads = randomIntBetween(4, 10);
Thread[] threads = new Thread[numThreads];
CyclicBarrier barrier = new CyclicBarrier(numThreads + 1);
@ -720,7 +290,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
});
try {
connection.updateSeedNodes(null, seedNodes, listener);
connection.ensureConnected(listener);
} catch (Exception e) {
// it's ok if we're shutting down
assertThat(e.getMessage(), containsString("threadcontext is already closed"));
@ -753,32 +323,24 @@ public class RemoteClusterConnectionTests extends ESTestCase {
knownNodes.add(transport3.getLocalDiscoNode());
knownNodes.add(transport2.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes = seedNodes(node3, node1, node2);
List<String> seedNodes = seedNodes(node3, node1, node2);
Collections.shuffle(seedNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
int maxNumConnections = randomIntBetween(1, 5);
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes, service, maxNumConnections, n -> true, null, profile)) {
String clusterAlias = "test-cluster";
Settings settings = Settings.builder().put(buildSniffSettings(clusterAlias, seedNodes))
.put(RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER.getKey(), maxNumConnections).build();
try (RemoteClusterConnection connection = new RemoteClusterConnection(settings, clusterAlias, service)) {
// test no nodes connected
RemoteConnectionInfo remoteConnectionInfo = assertSerialization(connection.getConnectionInfo());
assertNotNull(remoteConnectionInfo);
assertEquals(0, remoteConnectionInfo.numNodesConnected);
assertEquals(3, remoteConnectionInfo.seedNodes.size());
assertEquals(maxNumConnections, remoteConnectionInfo.connectionsPerCluster);
assertEquals("test-cluster", remoteConnectionInfo.clusterAlias);
// Connect some nodes
updateSeedNodes(connection, seedNodes);
remoteConnectionInfo = assertSerialization(connection.getConnectionInfo());
assertNotNull(remoteConnectionInfo);
assertEquals(connection.getNumNodesConnected(), remoteConnectionInfo.numNodesConnected);
assertEquals(Math.min(3, maxNumConnections), connection.getNumNodesConnected());
assertEquals(3, remoteConnectionInfo.seedNodes.size());
assertEquals(maxNumConnections, remoteConnectionInfo.connectionsPerCluster);
assertEquals("test-cluster", remoteConnectionInfo.clusterAlias);
assertEquals(clusterAlias, remoteConnectionInfo.clusterAlias);
}
}
}
@ -786,11 +348,11 @@ public class RemoteClusterConnectionTests extends ESTestCase {
public void testRemoteConnectionInfo() throws IOException {
RemoteConnectionInfo stats =
new RemoteConnectionInfo("test_cluster", Arrays.asList("seed:1"), 4, 3, TimeValue.timeValueMinutes(30), false);
new RemoteConnectionInfo("test_cluster", Arrays.asList("seed:1"), 4, 3, TimeValue.timeValueMinutes(30), false);
assertSerialization(stats);
RemoteConnectionInfo stats1 =
new RemoteConnectionInfo("test_cluster", Arrays.asList("seed:1"), 4, 4, TimeValue.timeValueMinutes(30), true);
new RemoteConnectionInfo("test_cluster", Arrays.asList("seed:1"), 4, 4, TimeValue.timeValueMinutes(30), true);
assertSerialization(stats1);
assertNotEquals(stats, stats1);
@ -857,7 +419,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
public void testRenderConnectionInfoXContent() throws IOException {
RemoteConnectionInfo stats =
new RemoteConnectionInfo("test_cluster", Arrays.asList("seed:1"), 4, 3, TimeValue.timeValueMinutes(30), true);
new RemoteConnectionInfo("test_cluster", Arrays.asList("seed:1"), 4, 3, TimeValue.timeValueMinutes(30), true);
stats = assertSerialization(stats);
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
@ -868,7 +430,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
"\"skip_unavailable\":true}}", Strings.toString(builder));
stats = new RemoteConnectionInfo(
"some_other_cluster", Arrays.asList("seed:1", "seed:2"), 2, 0, TimeValue.timeValueSeconds(30), false);
"some_other_cluster", Arrays.asList("seed:1", "seed:2"), 2, 0, TimeValue.timeValueSeconds(30), false);
stats = assertSerialization(stats);
builder = XContentFactory.jsonBuilder();
builder.startObject();
@ -879,61 +441,6 @@ public class RemoteClusterConnectionTests extends ESTestCase {
"\"skip_unavailable\":false}}", Strings.toString(builder));
}
public void testEnsureConnected() throws IOException, InterruptedException {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> true, null, profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
assertFalse(connectionManager.nodeConnected(seedNode));
assertFalse(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
CountDownLatch latch = new CountDownLatch(1);
connection.ensureConnected(new LatchedActionListener<>(new ActionListener<Void>() {
@Override
public void onResponse(Void aVoid) {
}
@Override
public void onFailure(Exception e) {
throw new AssertionError(e);
}
}, latch));
latch.await();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
// exec again we are already connected
connection.ensureConnected(new LatchedActionListener<>(new ActionListener<Void>() {
@Override
public void onResponse(Void aVoid) {
}
@Override
public void onFailure(Exception e) {
throw new AssertionError(e);
}
}, latch));
latch.await();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
}
}
}
}
public void testCollectNodes() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT)) {
@ -942,11 +449,9 @@ public class RemoteClusterConnectionTests extends ESTestCase {
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> true, null, profile)) {
if (randomBoolean()) {
updateSeedNodes(connection, seedNodes(seedNode));
}
String clusterAlias = "test-cluster";
Settings settings = buildSniffSettings(clusterAlias, seedNodes(seedNode));
try (RemoteClusterConnection connection = new RemoteClusterConnection(settings, clusterAlias, service)) {
CountDownLatch responseLatch = new CountDownLatch(1);
AtomicReference<Function<String, DiscoveryNode>> reference = new AtomicReference<>();
AtomicReference<Exception> failReference = new AtomicReference<>();
@ -977,21 +482,24 @@ public class RemoteClusterConnectionTests extends ESTestCase {
List<MockTransportService> discoverableTransports = new CopyOnWriteArrayList<>();
try {
final int numDiscoverableNodes = randomIntBetween(5, 20);
List<Tuple<String, Supplier<DiscoveryNode>>> discoverableNodes = new ArrayList<>(numDiscoverableNodes);
for (int i = 0; i < numDiscoverableNodes; i++ ) {
List<DiscoveryNode> discoverableNodes = new ArrayList<>(numDiscoverableNodes);
for (int i = 0; i < numDiscoverableNodes; i++) {
MockTransportService transportService = startTransport("discoverable_node" + i, knownNodes, Version.CURRENT);
discoverableNodes.add(Tuple.tuple("discoverable_node" + i, transportService::getLocalDiscoNode));
discoverableNodes.add(transportService.getLocalNode());
discoverableTransports.add(transportService);
}
List<Tuple<String, Supplier<DiscoveryNode>>> seedNodes = new CopyOnWriteArrayList<>(randomSubsetOf(discoverableNodes));
List<String> seedNodes = new CopyOnWriteArrayList<>(randomSubsetOf(discoverableNodes.stream()
.map(d -> d.getAddress().toString()).collect(Collectors.toList())));
Collections.shuffle(seedNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes, service, Integer.MAX_VALUE, n -> true, null, profile)) {
String clusterAlias = "test-cluster";
Settings settings = buildSniffSettings(clusterAlias, seedNodes);
try (RemoteClusterConnection connection = new RemoteClusterConnection(settings, clusterAlias, service)) {
final int numGetThreads = randomIntBetween(4, 10);
final Thread[] getThreads = new Thread[numGetThreads];
final int numModifyingThreads = randomIntBetween(4, 10);
@ -1017,25 +525,17 @@ public class RemoteClusterConnectionTests extends ESTestCase {
getThreads[i].start();
}
final AtomicInteger counter = new AtomicInteger();
for (int i = 0; i < modifyingThreads.length; i++) {
final int numDisconnects = randomIntBetween(5, 10);
modifyingThreads[i] = new Thread(() -> {
try {
barrier.await();
for (int j = 0; j < numDisconnects; j++) {
if (randomBoolean()) {
String node = "discoverable_node_added" + counter.incrementAndGet();
MockTransportService transportService =
startTransport(node, knownNodes,
Version.CURRENT);
discoverableTransports.add(transportService);
seedNodes.add(Tuple.tuple(node, () -> transportService.getLocalDiscoNode()));
PlainActionFuture.get(fut -> connection.updateSeedNodes(null, seedNodes,
ActionListener.map(fut, x -> null)));
} else {
DiscoveryNode node = randomFrom(discoverableNodes).v2().get();
connection.onNodeDisconnected(node, mock(Transport.Connection.class));
DiscoveryNode node = randomFrom(discoverableNodes);
try {
connection.getConnectionManager().getConnection(node);
} catch (NodeNotConnectedException e) {
// Ignore
}
}
} catch (Exception ex) {
@ -1058,61 +558,6 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
}
public void testClusterNameIsChecked() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
List<DiscoveryNode> otherClusterKnownNodes = new CopyOnWriteArrayList<>();
Settings settings = Settings.builder().put("cluster.name", "testClusterNameIsChecked").build();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT, threadPool, settings);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT, threadPool,
settings);
MockTransportService otherClusterTransport = startTransport("other_cluster_discoverable_node", otherClusterKnownNodes,
Version.CURRENT, threadPool, Settings.builder().put("cluster.name", "otherCluster").build());
MockTransportService otherClusterDiscoverable = startTransport("other_cluster_discoverable_node", otherClusterKnownNodes,
Version.CURRENT, threadPool, Settings.builder().put("cluster.name", "otherCluster").build())) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
otherClusterKnownNodes.add(otherClusterDiscoverable.getLocalDiscoNode());
otherClusterKnownNodes.add(otherClusterTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(seedNode), service, Integer.MAX_VALUE, n -> true, null, profile)) {
ConnectionManager connectionManager = connection.getConnectionManager();
updateSeedNodes(connection, seedNodes(seedNode));
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
List<Tuple<String, Supplier<DiscoveryNode>>> discoveryNodes =
Arrays.asList(
Tuple.tuple("other", otherClusterTransport::getLocalDiscoNode),
Tuple.tuple(seedNode.toString(), () -> seedNode));
Collections.shuffle(discoveryNodes, random());
updateSeedNodes(connection, discoveryNodes);
assertTrue(connectionManager.nodeConnected(seedNode));
for (DiscoveryNode otherClusterNode : otherClusterKnownNodes) {
assertFalse(connectionManager.nodeConnected(otherClusterNode));
}
assertFalse(connectionManager.nodeConnected(otherClusterTransport.getLocalDiscoNode()));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(connection.assertNoRunningConnections());
IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () ->
updateSeedNodes(connection, Arrays.asList(Tuple.tuple("other", otherClusterTransport::getLocalDiscoNode))));
assertThat(illegalStateException.getMessage(), allOf(
startsWith("handshake with [{other_cluster_discoverable_node}"),
containsString(otherClusterTransport.getLocalDiscoNode().toString()),
endsWith(" failed: remote cluster name [otherCluster] " +
"does not match expected remote cluster name [testClusterNameIsChecked]")));
}
}
}
}
public void testGetConnection() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
@ -1153,8 +598,9 @@ public class RemoteClusterConnectionTests extends ESTestCase {
service.start();
service.acceptIncomingRequests();
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
seedNodes(connectedNode), service, Integer.MAX_VALUE, n -> true, null, connectionManager)) {
String clusterAlias = "test-cluster";
Settings settings = buildSniffSettings(clusterAlias, seedNodes(connectedNode));
try (RemoteClusterConnection connection = new RemoteClusterConnection(settings, clusterAlias, service, connectionManager)) {
PlainActionFuture.get(fut -> connection.ensureConnected(ActionListener.map(fut, x -> null)));
for (int i = 0; i < 10; i++) {
//always a direct connection as the remote node is already connected
@ -1178,128 +624,10 @@ public class RemoteClusterConnectionTests extends ESTestCase {
}
}
public void testLazyResolveTransportAddress() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalDiscoNode();
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) {
service.start();
service.acceptIncomingRequests();
CountDownLatch multipleResolveLatch = new CountDownLatch(2);
Tuple<String, Supplier<DiscoveryNode>> seedSupplier = Tuple.tuple(seedNode.toString(), () -> {
multipleResolveLatch.countDown();
return seedNode;
});
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
Arrays.asList(seedSupplier), service, Integer.MAX_VALUE, n -> true, null, profile)) {
updateSeedNodes(connection, Arrays.asList(seedSupplier));
// Closing connections leads to RemoteClusterConnection.ConnectHandler.collectRemoteNodes
// being called again so we try to resolve the same seed node's host twice
discoverableTransport.close();
seedTransport.close();
assertTrue(multipleResolveLatch.await(30L, TimeUnit.SECONDS));
}
}
}
}
public void testProxyMode() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("node_0", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("node_1", knownNodes, Version.CURRENT)) {
knownNodes.add(seedTransport.getLocalDiscoNode());
knownNodes.add(discoverableTransport.getLocalDiscoNode());
Collections.shuffle(knownNodes, random());
final String proxyAddress = "1.1.1.1:99";
Map<String, DiscoveryNode> nodes = new HashMap<>();
nodes.put("node_0", seedTransport.getLocalDiscoNode());
nodes.put("node_1", discoverableTransport.getLocalDiscoNode());
Transport mockTcpTransport = getProxyTransport(threadPool, Collections.singletonMap(proxyAddress, nodes));
try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, mockTcpTransport, Version.CURRENT,
threadPool, null, Collections.emptySet())) {
service.start();
service.acceptIncomingRequests();
Tuple<String, Supplier<DiscoveryNode>> seedSupplier = Tuple.tuple("node_0", () ->
RemoteClusterAware.buildSeedNode("some-remote-cluster", "node_0:" + randomIntBetween(1, 10000), true));
assertEquals("node_0", seedSupplier.v2().get().getAttributes().get("server_name"));
try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster",
Arrays.asList(seedSupplier), service, Integer.MAX_VALUE, n -> true, proxyAddress, profile)) {
updateSeedNodes(connection, Arrays.asList(seedSupplier), proxyAddress);
assertEquals(2, connection.getNumNodesConnected());
assertNotNull(connection.getConnection(discoverableTransport.getLocalDiscoNode()));
assertNotNull(connection.getConnection(seedTransport.getLocalDiscoNode()));
assertEquals(proxyAddress, connection.getConnection(seedTransport.getLocalDiscoNode())
.getNode().getAddress().toString());
assertEquals(proxyAddress, connection.getConnection(discoverableTransport.getLocalDiscoNode())
.getNode().getAddress().toString());
connection.getConnectionManager().disconnectFromNode(knownNodes.get(0));
// ensure we reconnect
assertBusy(() -> {
assertEquals(2, connection.getNumNodesConnected());
});
discoverableTransport.close();
seedTransport.close();
}
}
}
}
public static Transport getProxyTransport(ThreadPool threadPool, Map<String, Map<String, DiscoveryNode>> nodeMap) {
if (nodeMap.isEmpty()) {
throw new IllegalArgumentException("nodeMap must be non-empty");
}
StubbableTransport stubbableTransport = new StubbableTransport(MockTransportService.newMockTransport(Settings.EMPTY,
Version.CURRENT, threadPool));
stubbableTransport.setDefaultConnectBehavior((t, node, profile, listener) -> {
Map<String, DiscoveryNode> proxyMapping = nodeMap.get(node.getAddress().toString());
if (proxyMapping == null) {
throw new IllegalStateException("no proxy mapping for node: " + node);
}
DiscoveryNode proxyNode = proxyMapping.get(node.getName());
if (proxyNode == null) {
// this is a seednode - lets pick one randomly
assertEquals("seed node must not have a port in the hostname: " + node.getHostName(),
-1, node.getHostName().lastIndexOf(':'));
assertTrue("missing hostname: " + node, proxyMapping.containsKey(node.getHostName()));
// route by seed hostname
proxyNode = proxyMapping.get(node.getHostName());
}
t.openConnection(proxyNode, profile, ActionListener.delegateFailure(listener,
(delegatedListener, connection) -> delegatedListener.onResponse(
new Transport.Connection() {
@Override
public DiscoveryNode getNode() {
return node;
}
@Override
public void sendRequest(long requestId, String action, TransportRequest request,
TransportRequestOptions options) throws IOException {
connection.sendRequest(requestId, action, request, options);
}
@Override
public void addCloseListener(ActionListener<Void> listener) {
connection.addCloseListener(listener);
}
@Override
public boolean isClosed() {
return connection.isClosed();
}
@Override
public void close() {
connection.close();
}
})));
});
return stubbableTransport;
private static Settings buildSniffSettings(String clusterAlias, List<String> seedNodes) {
Settings.Builder builder = Settings.builder();
builder.put(RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).getKey(),
Strings.collectionToCommaDelimitedString(seedNodes));
return builder.build();
}
}

View File

@ -22,8 +22,8 @@ import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.settings.AbstractScopedSettings;
@ -33,7 +33,6 @@ import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
@ -44,19 +43,15 @@ import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.BiFunction;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.Matchers.containsInAnyOrder;
@ -322,17 +317,33 @@ public class RemoteClusterServiceTests extends ESTestCase {
assertFalse(service.isCrossClusterSearchEnabled());
service.initializeRemoteClusters();
assertFalse(service.isCrossClusterSearchEnabled());
service.updateRemoteCluster("cluster_1", Collections.singletonList(cluster1Seed.getAddress().toString()), null);
Settings cluster1Settings = createSettings("cluster_1",
Collections.singletonList(cluster1Seed.getAddress().toString()));
PlainActionFuture<Void> clusterAdded = PlainActionFuture.newFuture();
// Add the cluster on a different thread to test that we wait for a new cluster to
// connect before returning.
new Thread(() -> {
try {
service.validateAndUpdateRemoteCluster("cluster_1", cluster1Settings);
clusterAdded.onResponse(null);
} catch (Exception e) {
clusterAdded.onFailure(e);
}
}).start();
clusterAdded.actionGet();
assertTrue(service.isCrossClusterSearchEnabled());
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
service.updateRemoteCluster("cluster_2", Collections.singletonList(cluster2Seed.getAddress().toString()), null);
Settings cluster2Settings = createSettings("cluster_2",
Collections.singletonList(cluster2Seed.getAddress().toString()));
service.validateAndUpdateRemoteCluster("cluster_2", cluster2Settings);
assertTrue(service.isCrossClusterSearchEnabled());
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
assertTrue(service.isRemoteClusterRegistered("cluster_2"));
service.updateRemoteCluster("cluster_2", Collections.emptyList(), null);
Settings cluster2SettingsDisabled = createSettings("cluster_2", Collections.emptyList());
service.validateAndUpdateRemoteCluster("cluster_2", cluster2SettingsDisabled);
assertFalse(service.isRemoteClusterRegistered("cluster_2"));
IllegalArgumentException iae = expectThrows(IllegalArgumentException.class,
() -> service.updateRemoteCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, Collections.emptyList(), null));
() -> service.validateAndUpdateRemoteCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, Settings.EMPTY));
assertEquals("remote clusters must not have the empty string as its key", iae.getMessage());
}
}
@ -362,7 +373,8 @@ public class RemoteClusterServiceTests extends ESTestCase {
assertFalse(service.isCrossClusterSearchEnabled());
service.initializeRemoteClusters();
assertTrue(service.isCrossClusterSearchEnabled());
service.updateRemoteCluster("cluster_1", Collections.singletonList(seedNode.getAddress().toString()), null);
service.validateAndUpdateRemoteCluster("cluster_1",
createSettings("cluster_1", Collections.singletonList(seedNode.getAddress().toString())));
assertTrue(service.isCrossClusterSearchEnabled());
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
RemoteClusterConnection remoteClusterConnection = service.getRemoteClusterConnection("cluster_1");
@ -434,7 +446,7 @@ public class RemoteClusterServiceTests extends ESTestCase {
boolean compressionEnabled = true;
settingsChange.put("cluster.remote.cluster_1.transport.compress", compressionEnabled);
settingsChange.putList("cluster.remote.cluster_1.seeds", cluster1Seed.getAddress().toString());
service.updateRemoteCluster("cluster_1", settingsChange.build());
service.validateAndUpdateRemoteCluster("cluster_1", settingsChange.build());
assertBusy(remoteClusterConnection::isClosed);
remoteClusterConnection = service.getRemoteClusterConnection("cluster_1");
@ -490,15 +502,15 @@ public class RemoteClusterServiceTests extends ESTestCase {
final CountDownLatch firstLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_1",
Arrays.asList(c1N1Node.getAddress().toString(), c1N2Node.getAddress().toString()), null,
genericProfile("cluster_1"), connectionListener(firstLatch));
createSettings("cluster_1", Arrays.asList(c1N1Node.getAddress().toString(), c1N2Node.getAddress().toString())),
connectionListener(firstLatch));
firstLatch.await();
final CountDownLatch secondLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_2",
Arrays.asList(c2N1Node.getAddress().toString(), c2N2Node.getAddress().toString()), null,
genericProfile("cluster_2"), connectionListener(secondLatch));
createSettings("cluster_2", Arrays.asList(c2N1Node.getAddress().toString(), c2N2Node.getAddress().toString())),
connectionListener(secondLatch));
secondLatch.await();
assertTrue(service.isCrossClusterSearchEnabled());
@ -555,15 +567,15 @@ public class RemoteClusterServiceTests extends ESTestCase {
final CountDownLatch firstLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_1",
Arrays.asList(c1N1Node.getAddress().toString(), c1N2Node.getAddress().toString()), null,
genericProfile("cluster_1"), connectionListener(firstLatch));
createSettings("cluster_1", Arrays.asList(c1N1Node.getAddress().toString(), c1N2Node.getAddress().toString())),
connectionListener(firstLatch));
firstLatch.await();
final CountDownLatch secondLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_2",
Arrays.asList(c2N1Node.getAddress().toString(), c2N2Node.getAddress().toString()), null,
genericProfile("cluster_2"), connectionListener(secondLatch));
createSettings("cluster_2", Arrays.asList(c2N1Node.getAddress().toString(), c2N2Node.getAddress().toString())),
connectionListener(secondLatch));
secondLatch.await();
assertTrue(service.isCrossClusterSearchEnabled());
@ -628,15 +640,15 @@ public class RemoteClusterServiceTests extends ESTestCase {
final CountDownLatch firstLatch = new CountDownLatch(1);
service.updateRemoteCluster("cluster_1",
Arrays.asList(c1N1Node.getAddress().toString(), c1N2Node.getAddress().toString()), null,
genericProfile("cluster_1"), connectionListener(firstLatch));
createSettings("cluster_1", Arrays.asList(c1N1Node.getAddress().toString(), c1N2Node.getAddress().toString())),
connectionListener(firstLatch));
firstLatch.await();
final CountDownLatch secondLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_2",
Arrays.asList(c2N1Node.getAddress().toString(), c2N2Node.getAddress().toString()), null,
genericProfile("cluster_2"), connectionListener(secondLatch));
createSettings("cluster_2", Arrays.asList(c2N1Node.getAddress().toString(), c2N2Node.getAddress().toString())),
connectionListener(secondLatch));
secondLatch.await();
CountDownLatch latch = new CountDownLatch(1);
service.collectNodes(new HashSet<>(Arrays.asList("cluster_1", "cluster_2")),
@ -771,118 +783,7 @@ public class RemoteClusterServiceTests extends ESTestCase {
}
}
public void testGetNodePredicateNodeRoles() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Predicate<DiscoveryNode> nodePredicate = RemoteClusterService.getNodePredicate(Settings.EMPTY);
{
DiscoveryNode all = new DiscoveryNode("id", address, Collections.emptyMap(),
DiscoveryNodeRole.BUILT_IN_ROLES, Version.CURRENT);
assertTrue(nodePredicate.test(all));
}
{
final Set<DiscoveryNodeRole> roles =
Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)));
DiscoveryNode dataMaster = new DiscoveryNode("id", address, Collections.emptyMap(),
roles, Version.CURRENT);
assertTrue(nodePredicate.test(dataMaster));
}
{
DiscoveryNode dedicatedMaster = new DiscoveryNode("id", address, Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.MASTER_ROLE), Version.CURRENT);
assertFalse(nodePredicate.test(dedicatedMaster));
}
{
DiscoveryNode dedicatedIngest = new DiscoveryNode("id", address, Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.INGEST_ROLE), Version.CURRENT);
assertTrue(nodePredicate.test(dedicatedIngest));
}
{
final Set<DiscoveryNodeRole> roles =
Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNodeRole.INGEST_ROLE, DiscoveryNodeRole.MASTER_ROLE)));
DiscoveryNode masterIngest = new DiscoveryNode("id", address, Collections.emptyMap(),
roles, Version.CURRENT);
assertTrue(nodePredicate.test(masterIngest));
}
{
DiscoveryNode dedicatedData = new DiscoveryNode("id", address, Collections.emptyMap(),
Collections.singleton(DiscoveryNodeRole.DATA_ROLE), Version.CURRENT);
assertTrue(nodePredicate.test(dedicatedData));
}
{
final Set<DiscoveryNodeRole> roles =
Collections.unmodifiableSet(new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.INGEST_ROLE)));
DiscoveryNode ingestData = new DiscoveryNode("id", address, Collections.emptyMap(),
roles, Version.CURRENT);
assertTrue(nodePredicate.test(ingestData));
}
{
DiscoveryNode coordOnly = new DiscoveryNode("id", address, Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
assertTrue(nodePredicate.test(coordOnly));
}
}
public void testGetNodePredicateNodeVersion() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Set<DiscoveryNodeRole> roles = DiscoveryNodeRole.BUILT_IN_ROLES;
Predicate<DiscoveryNode> nodePredicate = RemoteClusterService.getNodePredicate(Settings.EMPTY);
Version version = VersionUtils.randomVersion(random());
DiscoveryNode node = new DiscoveryNode("id", address, Collections.emptyMap(), roles, version);
assertThat(nodePredicate.test(node), equalTo(Version.CURRENT.isCompatible(version)));
}
public void testGetNodePredicateNodeAttrs() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Set<DiscoveryNodeRole> roles = DiscoveryNodeRole.BUILT_IN_ROLES;
Settings settings = Settings.builder().put("cluster.remote.node.attr", "gateway").build();
Predicate<DiscoveryNode> nodePredicate = RemoteClusterService.getNodePredicate(settings);
{
DiscoveryNode nonGatewayNode = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "false"),
roles, Version.CURRENT);
assertFalse(nodePredicate.test(nonGatewayNode));
assertTrue(RemoteClusterService.getNodePredicate(Settings.EMPTY).test(nonGatewayNode));
}
{
DiscoveryNode gatewayNode = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "true"),
roles, Version.CURRENT);
assertTrue(nodePredicate.test(gatewayNode));
assertTrue(RemoteClusterService.getNodePredicate(Settings.EMPTY).test(gatewayNode));
}
{
DiscoveryNode noAttrNode = new DiscoveryNode("id", address, Collections.emptyMap(), roles, Version.CURRENT);
assertFalse(nodePredicate.test(noAttrNode));
assertTrue(RemoteClusterService.getNodePredicate(Settings.EMPTY).test(noAttrNode));
}
}
public void testGetNodePredicatesCombination() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Settings settings = Settings.builder().put("cluster.remote.node.attr", "gateway").build();
Predicate<DiscoveryNode> nodePredicate = RemoteClusterService.getNodePredicate(settings);
Set<DiscoveryNodeRole> allRoles = DiscoveryNodeRole.BUILT_IN_ROLES;
Set<DiscoveryNodeRole> dedicatedMasterRoles = Collections.singleton(DiscoveryNodeRole.MASTER_ROLE);
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "true"),
dedicatedMasterRoles, Version.CURRENT);
assertFalse(nodePredicate.test(node));
}
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "false"),
dedicatedMasterRoles, Version.CURRENT);
assertFalse(nodePredicate.test(node));
}
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "false"),
dedicatedMasterRoles, Version.CURRENT);
assertFalse(nodePredicate.test(node));
}
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "true"),
allRoles, Version.CURRENT);
assertTrue(nodePredicate.test(node));
}
}
public void testReconnectWhenSeedsNodesAreUpdated() throws Exception {
public void testReconnectWhenStrategySettingsUpdated() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService cluster_node_0 = startTransport("cluster_node_0", knownNodes, Version.CURRENT);
MockTransportService cluster_node_1 = startTransport("cluster_node_1", knownNodes, Version.CURRENT)) {
@ -914,8 +815,8 @@ public class RemoteClusterServiceTests extends ESTestCase {
final CountDownLatch firstLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_test",
Collections.singletonList(node0.getAddress().toString()), null,
genericProfile("cluster_test"), connectionListener(firstLatch));
createSettings("cluster_test", Collections.singletonList(node0.getAddress().toString())),
connectionListener(firstLatch));
firstLatch.await();
assertTrue(service.isCrossClusterSearchEnabled());
@ -935,8 +836,8 @@ public class RemoteClusterServiceTests extends ESTestCase {
final CountDownLatch secondLatch = new CountDownLatch(1);
service.updateRemoteCluster(
"cluster_test",
newSeeds, null,
genericProfile("cluster_test"), connectionListener(secondLatch));
createSettings("cluster_test", newSeeds),
connectionListener(secondLatch));
secondLatch.await();
assertTrue(service.isCrossClusterSearchEnabled());
@ -957,78 +858,6 @@ public class RemoteClusterServiceTests extends ESTestCase {
}
}
public void testRemoteClusterWithProxy() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService cluster_1_node0 = startTransport("cluster_1_node0", knownNodes, Version.CURRENT);
MockTransportService cluster_1_node_1 = startTransport("cluster_1_node1", knownNodes, Version.CURRENT);
MockTransportService cluster_2_node0 = startTransport("cluster_2_node0", Collections.emptyList(), Version.CURRENT)) {
knownNodes.add(cluster_1_node0.getLocalDiscoNode());
knownNodes.add(cluster_1_node_1.getLocalDiscoNode());
String cluster1Proxy = "1.1.1.1:99";
String cluster2Proxy = "2.2.2.2:99";
Map<String, DiscoveryNode> nodesCluster1 = new HashMap<>();
nodesCluster1.put("cluster_1_node0", cluster_1_node0.getLocalDiscoNode());
nodesCluster1.put("cluster_1_node1", cluster_1_node_1.getLocalDiscoNode());
Map<String, Map<String, DiscoveryNode>> mapping = new HashMap<>();
mapping.put(cluster1Proxy, nodesCluster1);
mapping.put(cluster2Proxy, Collections.singletonMap("cluster_2_node0", cluster_2_node0.getLocalDiscoNode()));
Collections.shuffle(knownNodes, random());
Transport proxyTransport = RemoteClusterConnectionTests.getProxyTransport(threadPool, mapping);
try (MockTransportService transportService = MockTransportService.createNewService(Settings.EMPTY, proxyTransport,
Version.CURRENT, threadPool, null, Collections.emptySet());) {
transportService.start();
transportService.acceptIncomingRequests();
Settings.Builder builder = Settings.builder();
builder.putList("cluster.remote.cluster_1.seeds", "cluster_1_node0:8080");
builder.put("cluster.remote.cluster_1.proxy", cluster1Proxy);
try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) {
assertFalse(service.isCrossClusterSearchEnabled());
service.initializeRemoteClusters();
assertTrue(service.isCrossClusterSearchEnabled());
updateRemoteCluster(service, "cluster_1", Collections.singletonList("cluster_1_node1:8081"), cluster1Proxy);
assertTrue(service.isCrossClusterSearchEnabled());
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
assertFalse(service.isRemoteClusterRegistered("cluster_2"));
updateRemoteCluster(service, "cluster_2", Collections.singletonList("cluster_2_node0:9300"), cluster2Proxy);
assertTrue(service.isCrossClusterSearchEnabled());
assertTrue(service.isRemoteClusterRegistered("cluster_1"));
assertTrue(service.isRemoteClusterRegistered("cluster_2"));
List<RemoteConnectionInfo> infos = service.getRemoteConnectionInfos().collect(Collectors.toList());
for (RemoteConnectionInfo info : infos) {
switch (info.clusterAlias) {
case "cluster_1":
assertEquals(2, info.numNodesConnected);
break;
case "cluster_2":
assertEquals(1, info.numNodesConnected);
break;
default:
fail("unknown cluster: " + info.clusterAlias);
}
}
service.updateRemoteCluster("cluster_2", Collections.emptyList(), randomBoolean() ? cluster2Proxy : null);
assertFalse(service.isRemoteClusterRegistered("cluster_2"));
}
}
}
}
private static void updateRemoteCluster(RemoteClusterService service, String clusterAlias, List<String> addresses, String proxyAddress)
throws Exception {
CountDownLatch latch = new CountDownLatch(1);
AtomicReference<Exception> exceptionAtomicReference = new AtomicReference<>();
ActionListener<Void> listener = ActionListener.wrap(x -> latch.countDown(), x -> {
exceptionAtomicReference.set(x);
latch.countDown();
});
service.updateRemoteCluster(clusterAlias, addresses, proxyAddress, genericProfile(clusterAlias), listener);
latch.await();
if (exceptionAtomicReference.get() != null) {
throw exceptionAtomicReference.get();
}
}
public static void updateSkipUnavailable(RemoteClusterService service, String clusterAlias, boolean skipUnavailable) {
RemoteClusterConnection connection = service.getRemoteClusterConnection(clusterAlias);
connection.updateSkipUnavailable(skipUnavailable);
@ -1065,7 +894,10 @@ public class RemoteClusterServiceTests extends ESTestCase {
}
}
private static ConnectionProfile genericProfile(String clusterName) {
return RemoteClusterService.buildConnectionProfileFromSettings(Settings.EMPTY, clusterName);
private static Settings createSettings(String clusterAlias, List<String> seeds) {
Settings.Builder builder = Settings.builder();
builder.put(RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).getKey(),
Strings.collectionToCommaDelimitedString(seeds));
return builder.build();
}
}

View File

@ -0,0 +1,105 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import static org.mockito.Mockito.mock;
public class RemoteConnectionStrategyTests extends ESTestCase {
public void testStrategyChangeMeansThatStrategyMustBeRebuilt() {
ConnectionManager connectionManager = new ConnectionManager(Settings.EMPTY, mock(Transport.class));
RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager("cluster-alias", connectionManager);
FakeConnectionStrategy first = new FakeConnectionStrategy("cluster-alias", mock(TransportService.class), remoteConnectionManager,
RemoteConnectionStrategy.ConnectionStrategy.SIMPLE);
Settings newSettings = Settings.builder()
.put(RemoteConnectionStrategy.REMOTE_CONNECTION_MODE.getConcreteSettingForNamespace("cluster-alias").getKey(), "sniff")
.build();
assertTrue(first.shouldRebuildConnection(newSettings));
}
public void testSameStrategyChangeMeansThatStrategyDoesNotNeedToBeRebuilt() {
ConnectionManager connectionManager = new ConnectionManager(Settings.EMPTY, mock(Transport.class));
RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager("cluster-alias", connectionManager);
FakeConnectionStrategy first = new FakeConnectionStrategy("cluster-alias", mock(TransportService.class), remoteConnectionManager,
RemoteConnectionStrategy.ConnectionStrategy.SIMPLE);
Settings newSettings = Settings.builder()
.put(RemoteConnectionStrategy.REMOTE_CONNECTION_MODE.getConcreteSettingForNamespace("cluster-alias").getKey(), "simple")
.build();
assertFalse(first.shouldRebuildConnection(newSettings));
}
public void testChangeInConnectionProfileMeansTheStrategyMustBeRebuilt() {
ConnectionManager connectionManager = new ConnectionManager(TestProfiles.LIGHT_PROFILE, mock(Transport.class));
assertEquals(TimeValue.MINUS_ONE, connectionManager.getConnectionProfile().getPingInterval());
assertEquals(false, connectionManager.getConnectionProfile().getCompressionEnabled());
RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager("cluster-alias", connectionManager);
FakeConnectionStrategy first = new FakeConnectionStrategy("cluster-alias", mock(TransportService.class), remoteConnectionManager,
RemoteConnectionStrategy.ConnectionStrategy.SIMPLE);
ConnectionProfile profile = connectionManager.getConnectionProfile();
Settings.Builder newBuilder = Settings.builder();
newBuilder.put(RemoteConnectionStrategy.REMOTE_CONNECTION_MODE.getConcreteSettingForNamespace("cluster-alias").getKey(), "simple");
if (randomBoolean()) {
newBuilder.put(RemoteClusterService.REMOTE_CLUSTER_PING_SCHEDULE.getConcreteSettingForNamespace("cluster-alias").getKey(),
TimeValue.timeValueSeconds(5));
} else {
newBuilder.put(RemoteClusterService.REMOTE_CLUSTER_COMPRESS.getConcreteSettingForNamespace("cluster-alias").getKey(), true);
}
assertTrue(first.shouldRebuildConnection(newBuilder.build()));
}
private static class FakeConnectionStrategy extends RemoteConnectionStrategy {
private final ConnectionStrategy strategy;
FakeConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager,
RemoteConnectionStrategy.ConnectionStrategy strategy) {
super(clusterAlias, transportService, connectionManager);
this.strategy = strategy;
}
@Override
protected boolean strategyMustBeRebuilt(Settings newSettings) {
return false;
}
@Override
protected ConnectionStrategy strategyType() {
return this.strategy;
}
@Override
protected boolean shouldOpenMoreConnections() {
return false;
}
@Override
protected void connectImpl(ActionListener<Void> listener) {
}
}
}

View File

@ -0,0 +1,268 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.test.transport.StubbableTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Arrays;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import java.util.stream.Collectors;
public class SimpleConnectionStrategyTests extends ESTestCase {
private final String clusterAlias = "cluster-alias";
private final ConnectionProfile profile = RemoteClusterConnection.buildConnectionProfileFromSettings(Settings.EMPTY, "cluster");
private final ThreadPool threadPool = new TestThreadPool(getClass().getName());
@Override
public void tearDown() throws Exception {
super.tearDown();
ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS);
}
private MockTransportService startTransport(String id, Version version) {
return startTransport(id, version, Settings.EMPTY);
}
public MockTransportService startTransport(final String id, final Version version, final Settings settings) {
boolean success = false;
final Settings s = Settings.builder()
.put(ClusterName.CLUSTER_NAME_SETTING.getKey(), clusterAlias)
.put("node.name", id)
.put(settings)
.build();
MockTransportService newService = MockTransportService.createNewService(settings, version, threadPool);
try {
newService.start();
newService.acceptIncomingRequests();
success = true;
return newService;
} finally {
if (success == false) {
newService.close();
}
}
}
public void testSimpleStrategyWillOpenExpectedNumberOfConnectionsToAddresses() {
try (MockTransportService transport1 = startTransport("node1", Version.CURRENT);
MockTransportService transport2 = startTransport("node2", Version.CURRENT)) {
TransportAddress address1 = transport1.boundAddress().publishAddress();
TransportAddress address2 = transport2.boundAddress().publishAddress();
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
int numOfConnections = randomIntBetween(4, 8);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SimpleConnectionStrategy strategy = new SimpleConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
numOfConnections, addresses(address1, address2))) {
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
assertTrue(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
assertEquals(numOfConnections, connectionManager.size());
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testSimpleStrategyWillOpenNewConnectionsOnDisconnect() throws Exception {
try (MockTransportService transport1 = startTransport("node1", Version.CURRENT);
MockTransportService transport2 = startTransport("node2", Version.CURRENT)) {
TransportAddress address1 = transport1.boundAddress().publishAddress();
TransportAddress address2 = transport2.boundAddress().publishAddress();
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
int numOfConnections = randomIntBetween(4, 8);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SimpleConnectionStrategy strategy = new SimpleConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
numOfConnections, addresses(address1, address2))) {
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
long initialConnectionsToTransport2 = connectionManager.getAllConnectedNodes().stream()
.filter(n -> n.getAddress().equals(address2))
.count();
assertNotEquals(0, initialConnectionsToTransport2);
assertEquals(numOfConnections, connectionManager.size());
assertTrue(strategy.assertNoRunningConnections());
transport1.close();
assertBusy(() -> {
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
// More connections now pointing to transport2
long finalConnectionsToTransport2 = connectionManager.getAllConnectedNodes().stream()
.filter(n -> n.getAddress().equals(address2))
.count();
assertTrue(finalConnectionsToTransport2 > initialConnectionsToTransport2);
assertTrue(strategy.assertNoRunningConnections());
});
}
}
}
}
public void testConnectWithSingleIncompatibleNode() {
Version incompatibleVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion();
try (MockTransportService transport1 = startTransport("compatible-node", Version.CURRENT);
MockTransportService transport2 = startTransport("incompatible-node", incompatibleVersion)) {
TransportAddress address1 = transport1.boundAddress().publishAddress();
TransportAddress address2 = transport2.boundAddress().publishAddress();
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
StubbableTransport stubbableTransport = new StubbableTransport(localService.transport);
ConnectionManager connectionManager = new ConnectionManager(profile, stubbableTransport);
AtomicInteger address1Attempts = new AtomicInteger(0);
AtomicInteger address2Attempts = new AtomicInteger(0);
stubbableTransport.setDefaultConnectBehavior((transport, discoveryNode, profile, listener) -> {
if (discoveryNode.getAddress().equals(address1)) {
address1Attempts.incrementAndGet();
transport.openConnection(discoveryNode, profile, listener);
} else if (discoveryNode.getAddress().equals(address2)) {
address2Attempts.incrementAndGet();
transport.openConnection(discoveryNode, profile, listener);
} else {
throw new AssertionError("Unexpected address");
}
});
int numOfConnections = 5;
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SimpleConnectionStrategy strategy = new SimpleConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
numOfConnections, addresses(address1, address2))) {
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertEquals(4 ,connectionManager.size());
assertEquals(4 ,connectionManager.getAllConnectedNodes().stream().map(n -> n.getAddress().equals(address1)).count());
// Three attempts on first round, one attempts on second round, zero attempts on third round
assertEquals(4, address1Attempts.get());
// Two attempts on first round, one attempt on second round, one attempt on third round
assertEquals(4, address2Attempts.get());
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testConnectFailsWithIncompatibleNodes() {
Version incompatibleVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion();
try (MockTransportService transport1 = startTransport("incompatible-node", incompatibleVersion)) {
TransportAddress address1 = transport1.boundAddress().publishAddress();
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
int numOfConnections = randomIntBetween(4, 8);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SimpleConnectionStrategy strategy = new SimpleConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
numOfConnections, addresses(address1))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
expectThrows(Exception.class, connectFuture::actionGet);
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
assertEquals(0, connectionManager.size());
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testClusterNameValidationPreventConnectingToDifferentClusters() throws Exception {
Settings otherSettings = Settings.builder().put("cluster.name", "otherCluster").build();
try (MockTransportService transport1 = startTransport("cluster1", Version.CURRENT);
MockTransportService transport2 = startTransport("cluster2", Version.CURRENT, otherSettings)) {
TransportAddress address1 = transport1.boundAddress().publishAddress();
TransportAddress address2 = transport2.boundAddress().publishAddress();
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
int numOfConnections = randomIntBetween(4, 8);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SimpleConnectionStrategy strategy = new SimpleConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
numOfConnections, addresses(address1, address2))) {
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1)));
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
if (connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address1))) {
assertFalse(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
} else {
assertTrue(connectionManager.getAllConnectedNodes().stream().anyMatch(n -> n.getAddress().equals(address2)));
}
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
private static List<Supplier<TransportAddress>> addresses(final TransportAddress... addresses) {
return Arrays.stream(addresses).map(s -> (Supplier<TransportAddress>) () -> s).collect(Collectors.toList());
}
}

View File

@ -0,0 +1,642 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.transport;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodeRole;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.VersionUtils;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.test.transport.StubbableTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Predicate;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static org.hamcrest.CoreMatchers.startsWith;
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.endsWith;
import static org.hamcrest.Matchers.equalTo;
public class SniffConnectionStrategyTests extends ESTestCase {
private final String clusterAlias = "cluster-alias";
private final ConnectionProfile profile = RemoteClusterConnection.buildConnectionProfileFromSettings(Settings.EMPTY, "cluster");
private final ThreadPool threadPool = new TestThreadPool(getClass().getName());
@Override
public void tearDown() throws Exception {
super.tearDown();
ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS);
}
private MockTransportService startTransport(String id, List<DiscoveryNode> knownNodes, Version version) {
return startTransport(id, knownNodes, version, Settings.EMPTY);
}
public MockTransportService startTransport(final String id, final List<DiscoveryNode> knownNodes, final Version version,
final Settings settings) {
boolean success = false;
final Settings s = Settings.builder()
.put(ClusterName.CLUSTER_NAME_SETTING.getKey(), clusterAlias)
.put("node.name", id)
.put(settings)
.build();
ClusterName clusterName = ClusterName.CLUSTER_NAME_SETTING.get(s);
MockTransportService newService = MockTransportService.createNewService(s, version, threadPool);
try {
newService.registerRequestHandler(ClusterStateAction.NAME, ThreadPool.Names.SAME, ClusterStateRequest::new,
(request, channel, task) -> {
DiscoveryNodes.Builder builder = DiscoveryNodes.builder();
for (DiscoveryNode node : knownNodes) {
builder.add(node);
}
ClusterState build = ClusterState.builder(clusterName).nodes(builder.build()).build();
channel.sendResponse(new ClusterStateResponse(clusterName, build, false));
});
newService.start();
newService.acceptIncomingRequests();
success = true;
return newService;
} finally {
if (success == false) {
newService.close();
}
}
}
public void testSniffStrategyWillConnectToAndDiscoverNodes() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(seedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testSniffStrategyWillResolveDiscoveryNodesEachConnect() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
CountDownLatch multipleResolveLatch = new CountDownLatch(2);
Supplier<DiscoveryNode> seedNodeSupplier = () -> {
multipleResolveLatch.countDown();
return seedNode;
};
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(seedNode), Collections.singletonList(seedNodeSupplier))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
// Closing connections leads to RemoteClusterConnection.ConnectHandler.collectRemoteNodes
connectionManager.getConnection(seedNode).close();
assertTrue(multipleResolveLatch.await(30L, TimeUnit.SECONDS));
assertTrue(connectionManager.nodeConnected(discoverableNode));
}
}
}
}
public void testSniffStrategyWillConnectToMaxAllowedNodesAndOpenNewConnectionsOnDisconnect() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport1 = startTransport("discoverable_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport2 = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode discoverableNode1 = discoverableTransport1.getLocalNode();
DiscoveryNode discoverableNode2 = discoverableTransport2.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode1);
knownNodes.add(discoverableNode2);
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 2, n -> true, seedNodes(seedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertEquals(2, connectionManager.size());
assertTrue(connectionManager.nodeConnected(seedNode));
// Assert that one of the discovered nodes is connected. After that, disconnect the connected
// discovered node and ensure the other discovered node is eventually connected
if (connectionManager.nodeConnected(discoverableNode1)) {
assertTrue(connectionManager.nodeConnected(discoverableNode1));
discoverableTransport1.close();
assertBusy(() -> assertTrue(connectionManager.nodeConnected(discoverableNode2)));
} else {
assertTrue(connectionManager.nodeConnected(discoverableNode2));
discoverableTransport2.close();
assertBusy(() -> assertTrue(connectionManager.nodeConnected(discoverableNode1)));
}
}
}
}
}
public void testDiscoverWithSingleIncompatibleSeedNode() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
Version incompatibleVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService incompatibleSeedTransport = startTransport("discoverable_node", knownNodes, incompatibleVersion);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode incompatibleSeedNode = incompatibleSeedTransport.getLocalNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(incompatibleSeedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(seedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertEquals(2, connectionManager.size());
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertFalse(connectionManager.nodeConnected(incompatibleSeedNode));
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testConnectFailsWithIncompatibleNodes() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
Version incompatibleVersion = Version.CURRENT.minimumCompatibilityVersion().minimumCompatibilityVersion();
try (MockTransportService incompatibleSeedTransport = startTransport("seed_node", knownNodes, incompatibleVersion)) {
DiscoveryNode incompatibleSeedNode = incompatibleSeedTransport.getLocalNode();
knownNodes.add(incompatibleSeedNode);
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(incompatibleSeedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
expectThrows(Exception.class, connectFuture::actionGet);
assertFalse(connectionManager.nodeConnected(incompatibleSeedNode));
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testFilterNodesWithNodePredicate() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode);
DiscoveryNode rejectedNode = randomBoolean() ? seedNode : discoverableNode;
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> n.equals(rejectedNode) == false, seedNodes(seedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
if (rejectedNode.equals(seedNode)) {
assertFalse(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
} else {
assertTrue(connectionManager.nodeConnected(seedNode));
assertFalse(connectionManager.nodeConnected(discoverableNode));
}
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testClusterNameValidationPreventConnectingToDifferentClusters() throws Exception {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
List<DiscoveryNode> otherKnownNodes = new CopyOnWriteArrayList<>();
Settings otherSettings = Settings.builder().put("cluster.name", "otherCluster").build();
try (MockTransportService seed = startTransport("other_seed", knownNodes, Version.CURRENT);
MockTransportService discoverable = startTransport("other_discoverable", knownNodes, Version.CURRENT);
MockTransportService otherSeed = startTransport("other_seed", knownNodes, Version.CURRENT, otherSettings);
MockTransportService otherDiscoverable = startTransport("other_discoverable", knownNodes, Version.CURRENT, otherSettings)) {
DiscoveryNode seedNode = seed.getLocalNode();
DiscoveryNode discoverableNode = discoverable.getLocalNode();
DiscoveryNode otherSeedNode = otherSeed.getLocalNode();
DiscoveryNode otherDiscoverableNode = otherDiscoverable.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
Collections.shuffle(otherKnownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(seedNode, otherSeedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
seed.close();
assertBusy(strategy::assertNoRunningConnections);
PlainActionFuture<Void> newConnect = PlainActionFuture.newFuture();
strategy.connect(newConnect);
IllegalStateException ise = expectThrows(IllegalStateException.class, newConnect::actionGet);
assertThat(ise.getMessage(), allOf(
startsWith("handshake with [{cluster-alias#"),
endsWith(" failed: remote cluster name [otherCluster] " +
"does not match expected remote cluster name [" + clusterAlias + "]")));
assertFalse(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertFalse(connectionManager.nodeConnected(otherSeedNode));
assertFalse(connectionManager.nodeConnected(otherDiscoverableNode));
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testMultipleCallsToConnectEnsuresConnection() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(seedNode))) {
assertFalse(connectionManager.nodeConnected(seedNode));
assertFalse(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
// exec again we are already connected
PlainActionFuture<Void> ensureConnectFuture = PlainActionFuture.newFuture();
strategy.connect(ensureConnectFuture);
ensureConnectFuture.actionGet();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testConfiguredProxyAddressModeWillReplaceNodeAddress() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService accessible = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService unresponsive1 = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool);
MockTransportService unresponsive2 = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
// We start in order to get a valid address + port, but do not start accepting connections as we
// will not actually connect to these transports
unresponsive1.start();
unresponsive2.start();
DiscoveryNode accessibleNode = accessible.getLocalNode();
DiscoveryNode discoverableNode = unresponsive2.getLocalNode();
// Use the address for the node that will not respond
DiscoveryNode unaddressableSeedNode = new DiscoveryNode(accessibleNode.getName(), accessibleNode.getId(),
accessibleNode.getEphemeralId(), accessibleNode.getHostName(), accessibleNode.getHostAddress(),
unresponsive1.getLocalNode().getAddress(), accessibleNode.getAttributes(), accessibleNode.getRoles(),
accessibleNode.getVersion());
knownNodes.add(unaddressableSeedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
StubbableTransport transport = new StubbableTransport(localService.transport);
AtomicReference<TransportAddress> discoverableNodeAddress = new AtomicReference<>();
transport.setDefaultConnectBehavior((delegate, node, profile, listener) -> {
if (node.equals(discoverableNode)) {
// Do not actually try to connect because the node will not respond. Just capture the
// address for later assertion
discoverableNodeAddress.set(node.getAddress());
listener.onFailure(new ConnectTransportException(node, "general failure"));
} else {
delegate.openConnection(node, profile, listener);
}
});
List<String> seedNodes = Collections.singletonList(accessibleNode.toString());
TransportAddress proxyAddress = accessibleNode.getAddress();
ConnectionManager connectionManager = new ConnectionManager(profile, transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
proxyAddress.toString(), 3, n -> true, seedNodes)) {
assertFalse(connectionManager.nodeConnected(unaddressableSeedNode));
assertFalse(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.nodeConnected(unaddressableSeedNode));
// Connection to discoverable will fail due to the stubbable transport
assertFalse(connectionManager.nodeConnected(discoverableNode));
assertEquals(proxyAddress, discoverableNodeAddress.get());
assertTrue(strategy.assertNoRunningConnections());
}
}
}
}
public void testSniffStrategyWillNeedToBeRebuiltIfSeedsOrProxyChange() {
List<DiscoveryNode> knownNodes = new CopyOnWriteArrayList<>();
try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT);
MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes, Version.CURRENT)) {
DiscoveryNode seedNode = seedTransport.getLocalNode();
DiscoveryNode discoverableNode = discoverableTransport.getLocalNode();
knownNodes.add(seedNode);
knownNodes.add(discoverableNode);
Collections.shuffle(knownNodes, random());
try (MockTransportService localService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool)) {
localService.start();
localService.acceptIncomingRequests();
ConnectionManager connectionManager = new ConnectionManager(profile, localService.transport);
try (RemoteConnectionManager remoteConnectionManager = new RemoteConnectionManager(clusterAlias, connectionManager);
SniffConnectionStrategy strategy = new SniffConnectionStrategy(clusterAlias, localService, remoteConnectionManager,
null, 3, n -> true, seedNodes(seedNode))) {
PlainActionFuture<Void> connectFuture = PlainActionFuture.newFuture();
strategy.connect(connectFuture);
connectFuture.actionGet();
assertTrue(connectionManager.nodeConnected(seedNode));
assertTrue(connectionManager.nodeConnected(discoverableNode));
assertTrue(strategy.assertNoRunningConnections());
Setting<?> seedSetting = RemoteClusterService.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace("cluster-alias");
Setting<?> proxySetting = RemoteClusterService.REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace("cluster-alias");
Settings noChange = Settings.builder()
.put(seedSetting.getKey(), Strings.arrayToCommaDelimitedString(seedNodes(seedNode).toArray()))
.build();
assertFalse(strategy.shouldRebuildConnection(noChange));
Settings seedsChanged = Settings.builder()
.put(seedSetting.getKey(), Strings.arrayToCommaDelimitedString(seedNodes(discoverableNode).toArray()))
.build();
assertTrue(strategy.shouldRebuildConnection(seedsChanged));
Settings proxyChanged = Settings.builder()
.put(seedSetting.getKey(), Strings.arrayToCommaDelimitedString(seedNodes(seedNode).toArray()))
.put(proxySetting.getKey(), "proxy_address:9300")
.build();
assertTrue(strategy.shouldRebuildConnection(proxyChanged));
}
}
}
}
public void testGetNodePredicateNodeRoles() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Predicate<DiscoveryNode> nodePredicate = SniffConnectionStrategy.getNodePredicate(Settings.EMPTY);
{
DiscoveryNode all = new DiscoveryNode("id", address, Collections.emptyMap(),
DiscoveryNodeRole.BUILT_IN_ROLES, Version.CURRENT);
assertTrue(nodePredicate.test(all));
}
{
DiscoveryNode dataMaster = new DiscoveryNode("id", address, Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.MASTER_ROLE)), Version.CURRENT);
assertTrue(nodePredicate.test(dataMaster));
}
{
DiscoveryNode dedicatedMaster = new DiscoveryNode("id", address, Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.MASTER_ROLE)), Version.CURRENT);
assertFalse(nodePredicate.test(dedicatedMaster));
}
{
DiscoveryNode dedicatedIngest = new DiscoveryNode("id", address, Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.INGEST_ROLE)), Version.CURRENT);
assertTrue(nodePredicate.test(dedicatedIngest));
}
{
DiscoveryNode masterIngest = new DiscoveryNode("id", address, Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.INGEST_ROLE, DiscoveryNodeRole.MASTER_ROLE)), Version.CURRENT);
assertTrue(nodePredicate.test(masterIngest));
}
{
DiscoveryNode dedicatedData = new DiscoveryNode("id", address, Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE)), Version.CURRENT);
assertTrue(nodePredicate.test(dedicatedData));
}
{
DiscoveryNode ingestData = new DiscoveryNode("id", address, Collections.emptyMap(),
new HashSet<>(Arrays.asList(DiscoveryNodeRole.DATA_ROLE, DiscoveryNodeRole.INGEST_ROLE)), Version.CURRENT);
assertTrue(nodePredicate.test(ingestData));
}
{
DiscoveryNode coordOnly = new DiscoveryNode("id", address, Collections.emptyMap(),
Collections.emptySet(), Version.CURRENT);
assertTrue(nodePredicate.test(coordOnly));
}
}
public void testGetNodePredicateNodeVersion() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Set<DiscoveryNodeRole> roles = DiscoveryNodeRole.BUILT_IN_ROLES;
Predicate<DiscoveryNode> nodePredicate = SniffConnectionStrategy.getNodePredicate(Settings.EMPTY);
Version version = VersionUtils.randomVersion(random());
DiscoveryNode node = new DiscoveryNode("id", address, Collections.emptyMap(), roles, version);
assertThat(nodePredicate.test(node), equalTo(Version.CURRENT.isCompatible(version)));
}
public void testGetNodePredicateNodeAttrs() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Set<DiscoveryNodeRole> roles = DiscoveryNodeRole.BUILT_IN_ROLES;
Settings settings = Settings.builder().put("cluster.remote.node.attr", "gateway").build();
Predicate<DiscoveryNode> nodePredicate = SniffConnectionStrategy.getNodePredicate(settings);
{
DiscoveryNode nonGatewayNode = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "false"),
roles, Version.CURRENT);
assertFalse(nodePredicate.test(nonGatewayNode));
assertTrue(SniffConnectionStrategy.getNodePredicate(Settings.EMPTY).test(nonGatewayNode));
}
{
DiscoveryNode gatewayNode = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "true"),
roles, Version.CURRENT);
assertTrue(nodePredicate.test(gatewayNode));
assertTrue(SniffConnectionStrategy.getNodePredicate(Settings.EMPTY).test(gatewayNode));
}
{
DiscoveryNode noAttrNode = new DiscoveryNode("id", address, Collections.emptyMap(), roles, Version.CURRENT);
assertFalse(nodePredicate.test(noAttrNode));
assertTrue(SniffConnectionStrategy.getNodePredicate(Settings.EMPTY).test(noAttrNode));
}
}
public void testGetNodePredicatesCombination() {
TransportAddress address = new TransportAddress(TransportAddress.META_ADDRESS, 0);
Settings settings = Settings.builder().put("cluster.remote.node.attr", "gateway").build();
Predicate<DiscoveryNode> nodePredicate = SniffConnectionStrategy.getNodePredicate(settings);
Set<DiscoveryNodeRole> dedicatedMasterRoles = new HashSet<>(Arrays.asList(DiscoveryNodeRole.MASTER_ROLE));
Set<DiscoveryNodeRole> allRoles = DiscoveryNodeRole.BUILT_IN_ROLES;
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "true"),
dedicatedMasterRoles, Version.CURRENT);
assertFalse(nodePredicate.test(node));
}
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "false"),
dedicatedMasterRoles, Version.CURRENT);
assertFalse(nodePredicate.test(node));
}
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "false"),
dedicatedMasterRoles, Version.CURRENT);
assertFalse(nodePredicate.test(node));
}
{
DiscoveryNode node = new DiscoveryNode("id", address, Collections.singletonMap("gateway", "true"),
allRoles, Version.CURRENT);
assertTrue(nodePredicate.test(node));
}
}
private static List<String> seedNodes(final DiscoveryNode... seedNodes) {
return Arrays.stream(seedNodes).map(s -> s.getAddress().toString()).collect(Collectors.toList());
}
}

View File

@ -99,6 +99,10 @@ public final class MockTransportService extends TransportService {
}
}
public static MockTransportService createNewService(Settings settings, Version version, ThreadPool threadPool) {
return createNewService(settings, version, threadPool, null);
}
public static MockTransportService createNewService(Settings settings, Version version, ThreadPool threadPool,
@Nullable ClusterSettings clusterSettings) {
MockNioTransport mockTransport = newMockTransport(settings, version, threadPool);

View File

@ -12,8 +12,8 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.transport.RemoteClusterAware;
import org.elasticsearch.transport.RemoteConnectionStrategy;
import org.elasticsearch.xpack.ccr.action.repositories.DeleteInternalCcrRepositoryAction;
import org.elasticsearch.xpack.ccr.action.repositories.DeleteInternalCcrRepositoryRequest;
import org.elasticsearch.xpack.ccr.action.repositories.PutInternalCcrRepositoryAction;
@ -21,7 +21,6 @@ import org.elasticsearch.xpack.ccr.action.repositories.PutInternalCcrRepositoryR
import org.elasticsearch.xpack.ccr.repository.CcrRepository;
import java.io.IOException;
import java.util.List;
import java.util.Set;
class CcrRepositoryManager extends AbstractLifecycleComponent {
@ -69,20 +68,19 @@ class CcrRepositoryManager extends AbstractLifecycleComponent {
}
void init() {
Set<String> clusterAliases = buildRemoteClustersDynamicConfig(settings).keySet();
Set<String> clusterAliases = getEnabledRemoteClusters(settings);
for (String clusterAlias : clusterAliases) {
putRepository(CcrRepository.NAME_PREFIX + clusterAlias);
}
}
@Override
protected void updateRemoteCluster(String clusterAlias, List<String> addresses, String proxy, boolean compressionEnabled,
TimeValue pingSchedule) {
protected void updateRemoteCluster(String clusterAlias, Settings settings) {
String repositoryName = CcrRepository.NAME_PREFIX + clusterAlias;
if (addresses.isEmpty()) {
deleteRepository(repositoryName);
} else {
if (RemoteConnectionStrategy.isConnectionEnabled(clusterAlias, settings)) {
putRepository(repositoryName);
} else {
deleteRepository(repositoryName);
}
}
}

View File

@ -67,11 +67,9 @@ public abstract class CcrSingleNodeTestCase extends ESSingleNodeTestCase {
updateSettingsRequest.transientSettings(Settings.builder().put("cluster.remote.local.seeds", address));
assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
assertBusy(() -> {
List<RemoteConnectionInfo> infos = client().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos();
assertThat(infos.size(), equalTo(1));
assertThat(infos.get(0).getNumNodesConnected(), equalTo(1));
});
List<RemoteConnectionInfo> infos = client().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos();
assertThat(infos.size(), equalTo(1));
assertThat(infos.get(0).getNumNodesConnected(), equalTo(1));
}
@Before

View File

@ -93,13 +93,10 @@ public class RestartIndexFollowingIT extends CcrIntegTestCase {
String address = getLeaderCluster().getMasterNodeInstance(TransportService.class).boundAddress().publishAddress().toString();
updateSettingsRequest.persistentSettings(Settings.builder().put("cluster.remote.leader_cluster.seeds", address));
assertAcked(followerClient().admin().cluster().updateSettings(updateSettingsRequest).actionGet());
assertBusy(() -> {
List<RemoteConnectionInfo> infos =
followerClient().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos();
assertThat(infos.size(), equalTo(1));
assertThat(infos.get(0).getNumNodesConnected(), greaterThanOrEqualTo(1));
});
List<RemoteConnectionInfo> infos =
followerClient().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos();
assertThat(infos.size(), equalTo(1));
assertThat(infos.get(0).getNumNodesConnected(), greaterThanOrEqualTo(1));
}
private void cleanRemoteCluster() throws Exception {

View File

@ -24,10 +24,10 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.protocol.xpack.graph.GraphExploreRequest;
import org.elasticsearch.transport.RemoteClusterAware;
import org.elasticsearch.transport.RemoteConnectionStrategy;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.xpack.core.security.authz.ResolvedIndices;
@ -448,17 +448,16 @@ class IndicesAndAliasesResolver {
private RemoteClusterResolver(Settings settings, ClusterSettings clusterSettings) {
super(settings);
clusters = new CopyOnWriteArraySet<>(buildRemoteClustersDynamicConfig(settings).keySet());
clusters = new CopyOnWriteArraySet<>(getEnabledRemoteClusters(settings));
listenForUpdates(clusterSettings);
}
@Override
protected void updateRemoteCluster(String clusterAlias, List<String> addresses, String proxyAddress, boolean compressionEnabled,
TimeValue pingSchedule) {
if (addresses.isEmpty()) {
clusters.remove(clusterAlias);
} else {
protected void updateRemoteCluster(String clusterAlias, Settings settings) {
if (RemoteConnectionStrategy.isConnectionEnabled(clusterAlias, settings)) {
clusters.add(clusterAlias);
} else {
clusters.remove(clusterAlias);
}
}