diff --git a/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java b/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java index c11afa088aa..110053bcee7 100644 --- a/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java +++ b/server/src/main/java/org/elasticsearch/transport/ConnectionManager.java @@ -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 getAllConnectedNodes() { + return Collections.unmodifiableSet(connectedNodes.keySet()); + } + @Override public void close() { internalClose(true); diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java index 78bbf9775cc..fc21add37c2 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterAware.java @@ -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> 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> SEARCH_REMOTE_CLUSTER_SEEDS_UPGRADER = new SettingUpgrader>() { @@ -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> 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 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 SEARCH_REMOTE_CLUSTERS_PROXY_UPGRADER = new SettingUpgrader() { @@ -149,29 +148,30 @@ public abstract class RemoteClusterAware { * undocumented as it does not work well with all proxies. */ public static final Setting.AffixSetting 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 getEnabledRemoteClusters(final Settings settings) { + final Stream>> 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>>>> buildRemoteClustersDynamicConfig( - final Settings settings) { + final Settings settings) { final Map>>>> remoteSeeds = - buildRemoteClustersDynamicConfig(settings, REMOTE_CLUSTERS_SEEDS); + buildRemoteClustersDynamicConfig(settings, REMOTE_CLUSTERS_SEEDS); final Map>>>> searchRemoteSeeds = - buildRemoteClustersDynamicConfig(settings, SEARCH_REMOTE_CLUSTERS_SEEDS); + buildRemoteClustersDynamicConfig(settings, SEARCH_REMOTE_CLUSTERS_SEEDS); // sort the intersection for predictable output order final NavigableSet 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>>>> buildRemoteClustersDynamicConfig( - final Settings settings, final Setting.AffixSetting> seedsSetting) { + final Settings settings, final Setting.AffixSetting> seedsSetting) { final Stream>> allConcreteSettings = seedsSetting.getAllConcreteSettings(settings); return allConcreteSettings.collect( - Collectors.toMap(seedsSetting::getNamespace, concreteSetting -> { - String clusterName = seedsSetting.getNamespace(concreteSetting); - List addresses = concreteSetting.get(settings); - final boolean proxyMode = - REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterName).existsOrFallbackExists(settings); - List>> 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 addresses = concreteSetting.get(settings); + final boolean proxyMode = + REMOTE_CLUSTERS_PROXY.getConcreteSettingForNamespace(clusterName).existsOrFallbackExists(settings); + List>> 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> groupClusterIndices(Set 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 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 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 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> 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) { diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java index 0281565a7d2..d7f5896838b 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java @@ -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 nodePredicate; private final ThreadPool threadPool; - private volatile String proxyAddress; - private volatile List>> seedNodes; private volatile boolean skipUnavailable; - private final ConnectHandler connectHandler; private final TimeValue initialConnectionTimeout; - private final SetOnce 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>> seedNodes, - TransportService transportService, int maxNumRemoteConnections, Predicate 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>> seedNodes, - TransportService transportService, int maxNumRemoteConnections, Predicate 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>> seedNodes, - final ActionListener 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 voidActionListener) { + void ensureConnected(ActionListener 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 getRemoteClusterNamePredicate() { - return - new Predicate() { - @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>> 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> 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 connectListener) { - boolean runConnect = false; - final ActionListener 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() { - @Override - public void onResponse(Void aVoid) { - ActionListener.onResponse(getAndClearListeners(), aVoid); - } - - @Override - public void onFailure(Exception e) { - ActionListener.onFailure(getAndClearListeners(), e); - } - }); - } - }); - } - } - - private List> getAndClearListeners() { - final List> result; - synchronized (mutex) { - if (listeners.isEmpty()) { - result = Collections.emptyList(); - } else { - result = listeners; - listeners = new ArrayList<>(); - } - } - return result; - } - - private void collectRemoteNodes(Iterator> seedNodes, ActionListener listener) { - if (Thread.currentThread().isInterrupted()) { - listener.onFailure(new InterruptedException("remote connect thread got interrupted")); - } - - if (seedNodes.hasNext()) { - final Consumer 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 openConnectionStep = new StepListener<>(); - try { - remoteConnectionManager.openConnection(seedNode, profile, openConnectionStep); - } catch (Exception e) { - onFailure.accept(e); - } - - final StepListener handShakeStep = new StepListener<>(); - openConnectionStep.whenComplete(connection -> { - ConnectionProfile connectionProfile = remoteConnectionManager.getConnectionManager().getConnectionProfile(); - transportService.handshake(connection, connectionProfile.getHandshakeTimeout().millis(), - getRemoteClusterNamePredicate(), handShakeStep); - }, onFailure); - - final StepListener 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 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> 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 { - - private final Transport.Connection connection; - private final ActionListener listener; - private final Iterator> seedNodes; - - SniffClusterStateResponseHandler(Transport.Connection connection, ActionListener listener, - Iterator> 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 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() { - @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(); + } } diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java index ccf8876318f..2c33b32c323 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java @@ -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 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 remoteClusters = Collections.emptyMap(); - private volatile Map remoteClusterConnectionProfiles = Collections.emptyMap(); + private final Map 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>>>> seeds, - ActionListener connectionListener) { - if (seeds.containsKey(LOCAL_CLUSTER_GROUP_KEY)) { - throw new IllegalArgumentException("remote clusters must not have the empty string as its key"); - } - Map remoteClusters = new HashMap<>(); - if (seeds.isEmpty()) { - connectionListener.onResponse(null); - } else { - CountDown countDown = new CountDown(seeds.size()); - remoteClusters.putAll(this.remoteClusters); - for (Map.Entry>>>> entry : seeds.entrySet()) { - List>> 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 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 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 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 addresses, final String proxyAddress, - final ConnectionProfile connectionProfile, final ActionListener connectionListener) { - HashMap connectionProfiles = new HashMap<>(remoteClusterConnectionProfiles); - connectionProfiles.put(clusterAlias, connectionProfile); - this.remoteClusterConnectionProfiles = Collections.unmodifiableMap(connectionProfiles); - final List>> nodes = - addresses.stream().>>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 future = new PlainActionFuture<>(); - Map>>>> seeds = - RemoteClusterAware.buildRemoteClustersDynamicConfig(settings); - initializeConnectionProfiles(seeds.keySet()); - updateRemoteClusters(seeds, future); + final PlainActionFuture> future = new PlainActionFuture<>(); + Set enabledClusters = RemoteClusterAware.getEnabledRemoteClusters(settings); + + if (enabledClusters.isEmpty()) { + return; + } + + GroupedActionListener 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 remoteClusters) { - Map 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>> oldSeedNodes, - final List>> newSeedNodes) { - if (oldSeedNodes.size() != newSeedNodes.size()) { - return true; - } - Set oldSeeds = oldSeedNodes.stream().map(Tuple::v1).collect(Collectors.toSet()); - Set 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) { diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteConnectionStrategy.java b/server/src/main/java/org/elasticsearch/transport/RemoteConnectionStrategy.java new file mode 100644 index 00000000000..e20ff8c1760 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/RemoteConnectionStrategy.java @@ -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 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> 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 connectListener) { + boolean runConnect = false; + final ActionListener 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() { + @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 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> 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 listener); + + private List> getAndClearListeners() { + final List> 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; + } +} diff --git a/server/src/main/java/org/elasticsearch/transport/SimpleConnectionStrategy.java b/server/src/main/java/org/elasticsearch/transport/SimpleConnectionStrategy.java new file mode 100644 index 00000000000..21b340c396b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/SimpleConnectionStrategy.java @@ -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> addresses; + private final AtomicReference remoteClusterName = new AtomicReference<>(); + private final ConnectionProfile profile; + private final ConnectionManager.ConnectionValidator clusterNameValidator; + + SimpleConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager, + int maxNumRemoteConnections, List> 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 listener) { + performSimpleConnectionProcess(addresses.iterator(), listener); + } + + private void performSimpleConnectionProcess(Iterator> addressIter, ActionListener listener) { + openConnections(listener, 1); + } + + private void openConnections(ActionListener finished, int attemptNumber) { + if (attemptNumber <= MAX_CONNECT_ATTEMPTS_PER_RUN) { + List resolved = addresses.stream().map(Supplier::get).collect(Collectors.toList()); + + int remaining = maxNumRemoteConnections - connectionManager.size(); + ActionListener compositeListener = new ActionListener() { + + 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() { + @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 resolvedAddresses) { + long curr; + while ((curr = counter.getAndIncrement()) == Long.MIN_VALUE) ; + return resolvedAddresses.get(Math.toIntExact(Math.floorMod(curr, (long) resolvedAddresses.size()))); + } +} diff --git a/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java b/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java new file mode 100644 index 00000000000..c865ddba0aa --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java @@ -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 DEFAULT_NODE_PREDICATE = (node) -> Version.CURRENT.isCompatible(node.getVersion()) + && (node.isMasterNode() == false || node.isDataNode() || node.isIngestNode()); + + + private final List configuredSeedNodes; + private final List> seedNodes; + private final int maxNumRemoteConnections; + private final Predicate nodePredicate; + private final SetOnce 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 nodePredicate, + List configuredSeedNodes) { + this(clusterAlias, transportService, connectionManager, proxyAddress, maxNumRemoteConnections, nodePredicate, configuredSeedNodes, + configuredSeedNodes.stream().map(seedAddress -> + (Supplier) () -> resolveSeedNode(clusterAlias, seedAddress, proxyAddress)).collect(Collectors.toList())); + } + + SniffConnectionStrategy(String clusterAlias, TransportService transportService, RemoteConnectionManager connectionManager, + String proxyAddress, int maxNumRemoteConnections, Predicate nodePredicate, + List configuredSeedNodes, List> 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 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 listener) { + collectRemoteNodes(seedNodes.iterator(), listener); + } + + private void collectRemoteNodes(Iterator> seedNodes, ActionListener listener) { + if (Thread.currentThread().isInterrupted()) { + listener.onFailure(new InterruptedException("remote connect thread got interrupted")); + return; + } + + if (seedNodes.hasNext()) { + final Consumer 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 openConnectionStep = new StepListener<>(); + try { + connectionManager.openConnection(seedNode, profile, openConnectionStep); + } catch (Exception e) { + onFailure.accept(e); + } + + final StepListener handshakeStep = new StepListener<>(); + openConnectionStep.whenComplete(connection -> { + ConnectionProfile connectionProfile = connectionManager.getConnectionManager().getConnectionProfile(); + transportService.handshake(connection, connectionProfile.getHandshakeTimeout().millis(), + getRemoteClusterNamePredicate(), handshakeStep); + }, onFailure); + + final StepListener 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 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 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 { + + private final Transport.Connection connection; + private final ActionListener listener; + private final Iterator> seedNodes; + + SniffClusterStateResponseHandler(Transport.Connection connection, ActionListener listener, + Iterator> 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 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() { + @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 getRemoteClusterNamePredicate() { + return new Predicate() { + @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 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 oldSeedNodes, final List newSeedNodes) { + if (oldSeedNodes.size() != newSeedNodes.size()) { + return true; + } + Set oldSeeds = new HashSet<>(oldSeedNodes); + Set 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; + } +} diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java index 5cb00cbdc9d..b5cb7aa269a 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java @@ -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 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 futureHandler = new PlainTransportFuture<>( - new FutureTransportResponseHandler() { - @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 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 futureHandler = new PlainTransportFuture<>( - new FutureTransportResponseHandler() { - @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 handler = new PlainTransportFuture<>( - new FutureTransportResponseHandler() { - @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 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 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>> 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 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 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>> seedNodes) throws Exception { - updateSeedNodes(connection, seedNodes, null); - } - - private void updateSeedNodes( - final RemoteClusterConnection connection, - final List>> seedNodes, - final String proxyAddress) - throws Exception { - CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener 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 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 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 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 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>> 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)() -> s)) - .collect(Collectors.toList()); - } - } - - public void testTriggerUpdatesConcurrently() throws IOException, InterruptedException { - List 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>> 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 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 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>> seedNodes = seedNodes(seedNode1, seedNode); + List 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>> seedNodes = seedNodes(node3, node1, node2); + List 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 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() { - @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() { - @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 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> reference = new AtomicReference<>(); AtomicReference failReference = new AtomicReference<>(); @@ -977,21 +482,24 @@ public class RemoteClusterConnectionTests extends ESTestCase { List discoverableTransports = new CopyOnWriteArrayList<>(); try { final int numDiscoverableNodes = randomIntBetween(5, 20); - List>> discoverableNodes = new ArrayList<>(numDiscoverableNodes); - for (int i = 0; i < numDiscoverableNodes; i++ ) { + List 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>> seedNodes = new CopyOnWriteArrayList<>(randomSubsetOf(discoverableNodes)); + List 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 knownNodes = new CopyOnWriteArrayList<>(); - List 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>> 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 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 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> 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 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 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> 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> 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 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 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 seedNodes) { + Settings.Builder builder = Settings.builder(); + builder.put(RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).getKey(), + Strings.collectionToCommaDelimitedString(seedNodes)); + return builder.build(); } } diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java index 976623a9ab5..a494c1c72fb 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java @@ -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 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 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 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 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 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 roles = DiscoveryNodeRole.BUILT_IN_ROLES; - Predicate 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 roles = DiscoveryNodeRole.BUILT_IN_ROLES; - Settings settings = Settings.builder().put("cluster.remote.node.attr", "gateway").build(); - Predicate 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 nodePredicate = RemoteClusterService.getNodePredicate(settings); - Set allRoles = DiscoveryNodeRole.BUILT_IN_ROLES; - Set 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 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 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 nodesCluster1 = new HashMap<>(); - nodesCluster1.put("cluster_1_node0", cluster_1_node0.getLocalDiscoNode()); - nodesCluster1.put("cluster_1_node1", cluster_1_node_1.getLocalDiscoNode()); - Map> 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 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 addresses, String proxyAddress) - throws Exception { - CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener 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 seeds) { + Settings.Builder builder = Settings.builder(); + builder.put(RemoteClusterAware.REMOTE_CLUSTERS_SEEDS.getConcreteSettingForNamespace(clusterAlias).getKey(), + Strings.collectionToCommaDelimitedString(seeds)); + return builder.build(); } } diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteConnectionStrategyTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteConnectionStrategyTests.java new file mode 100644 index 00000000000..e669cfb1956 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/transport/RemoteConnectionStrategyTests.java @@ -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 listener) { + + } + } +} diff --git a/server/src/test/java/org/elasticsearch/transport/SimpleConnectionStrategyTests.java b/server/src/test/java/org/elasticsearch/transport/SimpleConnectionStrategyTests.java new file mode 100644 index 00000000000..207f7ad0e3d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/transport/SimpleConnectionStrategyTests.java @@ -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 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 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 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 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 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> addresses(final TransportAddress... addresses) { + return Arrays.stream(addresses).map(s -> (Supplier) () -> s).collect(Collectors.toList()); + } +} diff --git a/server/src/test/java/org/elasticsearch/transport/SniffConnectionStrategyTests.java b/server/src/test/java/org/elasticsearch/transport/SniffConnectionStrategyTests.java new file mode 100644 index 00000000000..6be3b71c016 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/transport/SniffConnectionStrategyTests.java @@ -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 knownNodes, Version version) { + return startTransport(id, knownNodes, version, Settings.EMPTY); + } + + public MockTransportService startTransport(final String id, final List 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 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 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 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 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 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 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 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 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 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 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 connectFuture = PlainActionFuture.newFuture(); + strategy.connect(connectFuture); + + expectThrows(Exception.class, connectFuture::actionGet); + assertFalse(connectionManager.nodeConnected(incompatibleSeedNode)); + assertTrue(strategy.assertNoRunningConnections()); + } + } + } + } + + public void testFilterNodesWithNodePredicate() { + List 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 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 knownNodes = new CopyOnWriteArrayList<>(); + List 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 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 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 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 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 ensureConnectFuture = PlainActionFuture.newFuture(); + strategy.connect(ensureConnectFuture); + ensureConnectFuture.actionGet(); + + assertTrue(connectionManager.nodeConnected(seedNode)); + assertTrue(connectionManager.nodeConnected(discoverableNode)); + assertTrue(strategy.assertNoRunningConnections()); + } + } + } + } + + public void testConfiguredProxyAddressModeWillReplaceNodeAddress() { + List 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 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 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 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 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 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 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 roles = DiscoveryNodeRole.BUILT_IN_ROLES; + Predicate 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 roles = DiscoveryNodeRole.BUILT_IN_ROLES; + Settings settings = Settings.builder().put("cluster.remote.node.attr", "gateway").build(); + Predicate 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 nodePredicate = SniffConnectionStrategy.getNodePredicate(settings); + Set dedicatedMasterRoles = new HashSet<>(Arrays.asList(DiscoveryNodeRole.MASTER_ROLE)); + Set 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 seedNodes(final DiscoveryNode... seedNodes) { + return Arrays.stream(seedNodes).map(s -> s.getAddress().toString()).collect(Collectors.toList()); + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java index cb509dfbf9d..02688a635f0 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java @@ -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); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRepositoryManager.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRepositoryManager.java index c241c7a9aa0..917308eea97 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRepositoryManager.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/CcrRepositoryManager.java @@ -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 clusterAliases = buildRemoteClustersDynamicConfig(settings).keySet(); + Set clusterAliases = getEnabledRemoteClusters(settings); for (String clusterAlias : clusterAliases) { putRepository(CcrRepository.NAME_PREFIX + clusterAlias); } } @Override - protected void updateRemoteCluster(String clusterAlias, List 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); } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java index 0316482571e..3a07ef9aa88 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java @@ -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 infos = client().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos(); - assertThat(infos.size(), equalTo(1)); - assertThat(infos.get(0).getNumNodesConnected(), equalTo(1)); - }); + List infos = client().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos(); + assertThat(infos.size(), equalTo(1)); + assertThat(infos.get(0).getNumNodesConnected(), equalTo(1)); } @Before diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java index 1073fee6779..5792c9643f5 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java @@ -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 infos = - followerClient().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()).get().getInfos(); - assertThat(infos.size(), equalTo(1)); - assertThat(infos.get(0).getNumNodesConnected(), greaterThanOrEqualTo(1)); - }); + List 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 { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index bf5ced4e540..906f78c0ea6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -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 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); } }