From b440ea946fd70133e2e014d358b0a133208e8fb6 Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 21 Nov 2016 17:03:49 +0100 Subject: [PATCH 01/59] add support multi cluster search Search api looks at indices that the request relates to, if they contain a certain separator ("|" at the moment, to be better defined in the future) the index name is split into two part where the first portion is the name of a remote cluster and the second part is the name of the index expression. Remote clusters are defined as dynamic cluster settings. There are some TODOs and open question but the main functionality works. --- .../action/search/SearchTransportService.java | 159 +++++++++++++++++- .../action/search/TransportSearchAction.java | 140 +++++++++++++-- .../common/settings/ClusterSettings.java | 4 +- .../java/org/elasticsearch/node/Node.java | 3 +- .../action/search/SearchAsyncActionTests.java | 4 +- 5 files changed, 294 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 5b052132566..892567d096b 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -19,16 +19,25 @@ package org.elasticsearch.action.search; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Randomness; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult; @@ -44,13 +53,23 @@ import org.elasticsearch.search.query.QuerySearchResultProvider; import org.elasticsearch.search.query.ScrollQuerySearchResult; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TaskAwareTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; /** * An encapsulation of {@link org.elasticsearch.search.SearchService} operations exposed through @@ -71,11 +90,147 @@ public class SearchTransportService extends AbstractComponent { public static final String FETCH_ID_SCROLL_ACTION_NAME = "indices:data/read/search[phase/fetch/id/scroll]"; public static final String FETCH_ID_ACTION_NAME = "indices:data/read/search[phase/fetch/id]"; - private final TransportService transportService; + //TODO what should the setting name be? + public static final Setting REMOTE_CLUSTERS_SEEDS = Setting.groupSetting("action.search.remote.", + SearchTransportService::validateRemoteClustersSeeds, + Setting.Property.NodeScope, + Setting.Property.Dynamic); - public SearchTransportService(Settings settings, TransportService transportService) { + private final TransportService transportService; + private volatile Map> remoteClustersSeeds; + + public SearchTransportService(Settings settings, ClusterSettings clusterSettings, TransportService transportService) { super(settings); this.transportService = transportService; + setRemoteClustersSeeds(REMOTE_CLUSTERS_SEEDS.get(settings)); + clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTERS_SEEDS, this::setRemoteClustersSeeds, + SearchTransportService::validateRemoteClustersSeeds); + } + + private static void validateRemoteClustersSeeds(Settings settings) { + //TODO add a static whitelist like in reindex from remote + for (String clusterName : settings.names()) { + String[] remoteHosts = settings.getAsArray(clusterName); + if (remoteHosts.length == 0) { + throw new IllegalArgumentException("no hosts set for remote cluster [" + clusterName + "], at least one host is required"); + } + for (String remoteHost : remoteHosts) { + String[] strings = remoteHost.split(":"); + if (strings.length != 2) { + throw new IllegalArgumentException("remote hosts need to be configured as [host:port], found [" + remoteHost + "] " + + "instead for remote cluster [" + clusterName + "]"); + } + try { + Integer.valueOf(strings[1]); + } catch(NumberFormatException e) { + throw new IllegalArgumentException("port must be a number, found [" + strings[1] + "] instead for remote cluster [" + + clusterName + "]"); + } + } + } + } + + private void setRemoteClustersSeeds(Settings settings) { + Map> remoteClustersNodes = new HashMap<>(); + for (String clusterName : settings.names()) { + String[] remoteHosts = settings.getAsArray(clusterName); + for (String remoteHost : remoteHosts) { + String[] strings = remoteHost.split(":"); + String host = strings[0]; + int port = Integer.valueOf(strings[1]); + DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, + new TransportAddress(new InetSocketAddress(host, port)), Version.CURRENT.minimumCompatibilityVersion()); + //don't connect yet as that would require the remote node to be up and would fail the local node startup otherwise + List nodes = remoteClustersNodes.get(clusterName); + if (nodes == null) { + nodes = new ArrayList<>(); + remoteClustersNodes.put(clusterName, nodes); + } + nodes.add(node); + } + } + remoteClustersSeeds = remoteClustersNodes; + } + + private DiscoveryNode connectToRemoteCluster(String clusterName) { + List nodes = remoteClustersSeeds.get(clusterName); + if (nodes == null) { + throw new IllegalArgumentException("no remote cluster configured with name [" + clusterName + "]"); + } + DiscoveryNode node = nodes.get(Randomness.get().nextInt(nodes.size())); + //TODO we just take a random host for now, implement fallback in case of connect failure + try { + //TODO we should call liveness api and get back an updated discovery node. that would have the updated version + // and would make the search shards call more future-proof. Also validating the cluster name may be a thing. + connectToRemoteNode(node); + } catch(ConnectTransportException e) { + throw new ConnectTransportException(node, "unable to connect to remote cluster [" + clusterName + "]", e); + } + return node; + } + + void connectToRemoteNode(DiscoveryNode remoteNode) { + //TODO should the list of seeds get updated based on nodes that we get back from the remote cluster through search_shards? + transportService.connectToNode(remoteNode); + //TODO is it ok to connect and leave the node connected? It will be pinged from now on? + } + + void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, + ActionListener> listener) { + final CountDown responsesCountDown = new CountDown(remoteIndicesByCluster.size()); + final Map searchShardsResponses = new ConcurrentHashMap<>(); + final AtomicReference transportException = new AtomicReference<>(); + for (Map.Entry> entry : remoteIndicesByCluster.entrySet()) { + final String clusterName = entry.getKey(); + final DiscoveryNode node = connectToRemoteCluster(clusterName); + final List indices = entry.getValue(); + //local true so we don't go to the master for each single remote search + ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices.toArray(new String[indices.size()])) + .indicesOptions(searchRequest.indicesOptions()).local(true).preference(searchRequest.preference()) + .routing(searchRequest.routing()); + + transportService.sendRequest(node, ClusterSearchShardsAction.NAME, searchShardsRequest, + new TransportResponseHandler() { + + @Override + public ClusterSearchShardsResponse newInstance() { + return new ClusterSearchShardsResponse(); + } + + @Override + public void handleResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { + searchShardsResponses.put(clusterName, clusterSearchShardsResponse); + if (responsesCountDown.countDown()) { + TransportException exception = transportException.get(); + if (exception == null) { + listener.onResponse(searchShardsResponses); + } else { + listener.onFailure(transportException.get()); + } + } + } + + @Override + public void handleException(TransportException e) { + TransportException exception = new TransportException("unable to communicate with remote cluster [" + + clusterName + "]", e); + if (transportException.compareAndSet(null, exception) == false) { + exception = transportException.accumulateAndGet(exception, (previous, current) -> { + current.addSuppressed(previous); + return current; + }); + } + if (responsesCountDown.countDown()) { + listener.onFailure(exception); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SEARCH; + } + }); + } } public void sendFreeContext(DiscoveryNode node, final long contextId, SearchRequest request) { diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index b94f4b35096..c56c7bac94e 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -20,27 +20,39 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.PlainShardIterator; +import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +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.Executor; @@ -55,6 +67,8 @@ public class TransportSearchAction extends HandledTransportAction SHARD_COUNT_LIMIT_SETTING = Setting.longSetting( "action.search.shard_count.limit", 1000L, 1L, Property.Dynamic, Property.NodeScope); + private static final char REMOTE_CLUSTER_INDEX_SEPARATOR = '|'; + private final ClusterService clusterService; private final SearchTransportService searchTransportService; private final SearchPhaseController searchPhaseController; @@ -73,7 +87,8 @@ public class TransportSearchAction extends HandledTransportAction buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState, Index[] concreteIndices) { + private Map buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState, + Index[] concreteIndices, String[] remoteUUIDs) { final Map aliasFilterMap = new HashMap<>(); for (Index index : concreteIndices) { clusterState.blocks().indexBlockedRaiseException(ClusterBlockLevel.READ, index.getName()); @@ -81,6 +96,11 @@ public class TransportSearchAction extends HandledTransportAction listener) { // pure paranoia if time goes backwards we are at least positive final long startTimeInMillis = Math.max(0, System.currentTimeMillis()); + + //TODO make selection smarter: aliases can still contain any character and remote indices should have the precedence all the time. + //e.g. we could skip the remote logic if no remote clusters are registered. Also don't go remotely if the prefix is not + //a registered cluster rather than throwing an error? + final List localIndicesList = new ArrayList<>(); + final Map> remoteIndicesByCluster = new HashMap<>(); + for (String index : searchRequest.indices()) { + int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); + if (i >= 0) { + String remoteCluster = index.substring(0, i); + String remoteIndex = index.substring(i + 1); + List indices = remoteIndicesByCluster.get(remoteCluster); + if (indices == null) { + indices = new ArrayList<>(); + remoteIndicesByCluster.put(remoteCluster, indices); + } + indices.add(remoteIndex); + } else { + localIndicesList.add(index); + } + } + + String[] localIndices = localIndicesList.toArray(new String[localIndicesList.size()]); + + if (remoteIndicesByCluster.isEmpty()) { + executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, + Strings.EMPTY_ARRAY, Collections.emptyList(), Collections.emptySet(), listener); + } else { + searchTransportService.sendSearchShards(searchRequest, remoteIndicesByCluster, + new ActionListener>() { + @Override + public void onResponse(Map searchShardsResponses) { + List remoteShardIterators = new ArrayList<>(); + Set remoteNodes = new HashSet<>(); + Set remoteUUIDs = new HashSet<>(); + for (Map.Entry entry : searchShardsResponses.entrySet()) { + String clusterName = entry.getKey(); + ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); + Collections.addAll(remoteNodes, searchShardsResponse.getNodes()); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + //add the cluster name to the remote index names for indices disambiguation + //this ends up in the hits returned with the search response + Index index = new Index(clusterName + REMOTE_CLUSTER_INDEX_SEPARATOR + + clusterSearchShardsGroup.getShardId().getIndex().getName(), + clusterSearchShardsGroup.getShardId().getIndex().getUUID()); + ShardId shardId = new ShardId(index, clusterSearchShardsGroup.getShardId().getId()); + ShardIterator shardIterator = new PlainShardIterator(shardId, + Arrays.asList(clusterSearchShardsGroup.getShards())); + remoteShardIterators.add(shardIterator); + remoteUUIDs.add(clusterSearchShardsGroup.getShardId().getIndex().getUUID()); + } + } + executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, + remoteUUIDs.toArray(new String[remoteUUIDs.size()]), remoteShardIterators, remoteNodes, listener); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }); + } + } + + private void executeSearch(SearchTask task, long startTimeInMillis, SearchRequest searchRequest, String[] localIndices, + String[] remoteUUIDs, List remoteShardIterators, Set remoteNodes, + ActionListener listener) { + ClusterState clusterState = clusterService.state(); clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ); - // TODO: I think startTime() should become part of ActionRequest and that should be used both for index name // date math expressions and $now in scripts. This way all apis will deal with now in the same way instead // of just for the _search api Index[] indices = indexNameExpressionResolver.concreteIndices(clusterState, searchRequest.indicesOptions(), - startTimeInMillis, searchRequest.indices()); - Map aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices); + startTimeInMillis, localIndices); + Map aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteUUIDs); Map> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(), searchRequest.indices()); String[] concreteIndices = new String[indices.length]; for (int i = 0; i < indices.length; i++) { concreteIndices[i] = indices[i].getName(); } - GroupShardsIterator shardIterators = clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, + GroupShardsIterator localShardsIterator = clusterService.operationRouting().searchShards(clusterState, concreteIndices, routingMap, searchRequest.preference()); + GroupShardsIterator shardIterators = mergeShardsIterators(localShardsIterator, remoteShardIterators); + failIfOverShardCountLimit(clusterService, shardIterators.size()); // optimize search type for cases where there is only one shard group to search on @@ -124,8 +213,38 @@ public class TransportSearchAction extends HandledTransportAction nodesLookup = mergeNodesLookup(clusterState.nodes(), remoteNodes); + + searchAsyncAction(task, searchRequest, shardIterators, startTimeInMillis, nodesLookup, clusterState.version(), + Collections.unmodifiableMap(aliasFilter), listener).start(); + } + + private static GroupShardsIterator mergeShardsIterators(GroupShardsIterator localShardsIterator, + List remoteShardIterators) { + if (remoteShardIterators.isEmpty()) { + return localShardsIterator; + } + List shards = new ArrayList<>(); + for (ShardIterator shardIterator : remoteShardIterators) { + shards.add(shardIterator); + } + for (ShardIterator shardIterator : localShardsIterator) { + shards.add(shardIterator); + } + return new GroupShardsIterator(shards); + } + + private Function mergeNodesLookup(DiscoveryNodes nodes, Set remoteNodes) { + if (remoteNodes.isEmpty()) { + return nodes::get; + } + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(nodes.getNodes()); + for (DiscoveryNode remoteNode : remoteNodes) { + //TODO shall we catch connect exceptions here? Otherwise we will return an error but we could rather return partial results? + searchTransportService.connectToRemoteNode(remoteNode); + builder.put(remoteNode.getId(), remoteNode); + } + return builder.build()::get; } @Override @@ -134,10 +253,9 @@ public class TransportSearchAction extends HandledTransportAction aliasFilter, + long startTime, Function nodesLookup, + long clusterStateVersion, Map aliasFilter, ActionListener listener) { - final Function nodesLookup = state.nodes()::get; - final long clusterStateVersion = state.version(); Executor executor = threadPool.executor(ThreadPool.Names.SEARCH); AbstractSearchAsyncAction searchAsyncAction; switch(searchRequest.searchType()) { @@ -167,7 +285,7 @@ public class TransportSearchAction extends HandledTransportAction shardCountLimit) { throw new IllegalArgumentException("Trying to query " + shardCount + " shards, which is over the limit of " diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 037109bff12..fabce2d7562 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -19,6 +19,7 @@ package org.elasticsearch.common.settings; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; +import org.elasticsearch.action.search.SearchTransportService; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.DestructiveOperations; @@ -54,9 +55,9 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.zen.ElectMasterService; -import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.discovery.zen.FaultDetection; import org.elasticsearch.discovery.zen.UnicastZenPing; +import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.GatewayService; @@ -255,6 +256,7 @@ public final class ClusterSettings extends AbstractScopedSettings { SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING, ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, + SearchTransportService.REMOTE_CLUSTERS_SEEDS, TransportService.TRACE_LOG_EXCLUDE_SETTING, TransportService.TRACE_LOG_INCLUDE_SETTING, TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING, diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 29781657613..aa13043bc67 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -426,7 +426,8 @@ public class Node implements Closeable { b.bind(IndicesService.class).toInstance(indicesService); b.bind(SearchService.class).toInstance(newSearchService(clusterService, indicesService, threadPool, scriptModule.getScriptService(), bigArrays, searchModule.getFetchPhase())); - b.bind(SearchTransportService.class).toInstance(new SearchTransportService(settings, transportService)); + b.bind(SearchTransportService.class).toInstance(new SearchTransportService(settings, + settingsModule.getClusterSettings(), transportService)); b.bind(SearchPhaseController.class).toInstance(new SearchPhaseController(settings, bigArrays, scriptModule.getScriptService())); b.bind(Transport.class).toInstance(transport); diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index c4645e20a5e..ac01ac5dcd4 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -76,7 +77,8 @@ public class SearchAsyncActionTests extends ESTestCase { AtomicInteger contextIdGenerator = new AtomicInteger(0); GroupShardsIterator shardsIter = getShardsIter("idx", randomIntBetween(1, 10), randomBoolean(), primaryNode, replicaNode); AtomicInteger numFreedContext = new AtomicInteger(); - SearchTransportService transportService = new SearchTransportService(Settings.EMPTY, null) { + SearchTransportService transportService = new SearchTransportService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, + Collections.singleton(SearchTransportService.REMOTE_CLUSTERS_SEEDS)), null) { @Override public void sendFreeContext(DiscoveryNode node, long contextId, SearchRequest request) { numFreedContext.incrementAndGet(); From ec86771f6ede25feff8db9f8d06e3eee83f6a298 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 23 Nov 2016 18:12:42 +0100 Subject: [PATCH 02/59] Add a dedicated integ test project for multi-cluster-search This commit adds `qa/multi-cluster-search` which currently does a simple search across 2 clusters. This commit also adds support for IPv6 addresses and fixes an issue where all shards of the local cluster are searched when only a remote index was given. --- .../gradle/test/ClusterConfiguration.groovy | 9 +++ .../gradle/test/ClusterFormationTasks.groovy | 2 + .../action/search/SearchTransportService.java | 30 +++++--- .../action/search/TransportSearchAction.java | 68 ++++++++++--------- .../search/SearchTransportServiceTests.java | 67 ++++++++++++++++++ qa/multi-cluster-search/build.gradle | 53 +++++++++++++++ .../MultiClusterSearchYamlTestSuiteIT.java | 48 +++++++++++++ .../test/multi_cluster/10_basic.yaml | 47 +++++++++++++ .../test/remote_cluster/10_basic.yaml | 35 ++++++++++ settings.gradle | 1 + 10 files changed, 316 insertions(+), 44 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java create mode 100644 qa/multi-cluster-search/build.gradle create mode 100644 qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java create mode 100644 qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml create mode 100644 qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy index ca4957f7a6c..2c5159bc2d8 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy @@ -96,6 +96,15 @@ class ClusterConfiguration { return seedNode.transportUri() } + /** + * A closure to call which returns a map of settings. + * + * This can be used to pass settings to a cluster that are not available at evaluation time ie. + * the address of a remote cluster etc. + */ + @Input + Closure dynamicSettings = { -> Collections.emptyMap() } + /** * A closure to call before the cluster is considered ready. The closure is passed the node info, * as well as a groovy AntBuilder, to enable running ant condition checks. The default wait diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 73d7a3c7cd9..b911a7b5ea8 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -294,6 +294,8 @@ class ClusterFormationTasks { if (unicastTransportUri != null) { esConfig['discovery.zen.ping.unicast.hosts'] = "\"${unicastTransportUri}\"" } + Map dynamicSettings = node.config.dynamicSettings(); + esConfig.putAll(dynamicSettings) File configFile = new File(node.confDir, 'elasticsearch.yml') logger.info("Configuring ${configFile}") configFile.setText(esConfig.collect { key, value -> "${key}: ${value}" }.join('\n'), 'UTF-8') diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 892567d096b..65254a75193 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -115,31 +115,35 @@ public class SearchTransportService extends AbstractComponent { throw new IllegalArgumentException("no hosts set for remote cluster [" + clusterName + "], at least one host is required"); } for (String remoteHost : remoteHosts) { - String[] strings = remoteHost.split(":"); - if (strings.length != 2) { + 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 for remote cluster [" + clusterName + "]"); + "instead for remote cluster [" + clusterName + "]"); } + String port = remoteHost.substring(portSeparator+1); try { - Integer.valueOf(strings[1]); + Integer portValue = Integer.valueOf(port); + if (portValue <= 0) { + throw new IllegalArgumentException("port number must be > 0 but was: [" + portValue + "]"); + } } catch(NumberFormatException e) { - throw new IllegalArgumentException("port must be a number, found [" + strings[1] + "] instead for remote cluster [" + + throw new IllegalArgumentException("port must be a number, found [" + port + "] instead for remote cluster [" + clusterName + "]"); } } } } - private void setRemoteClustersSeeds(Settings settings) { + static Map> builtRemoteClustersSeeds(Settings settings) { Map> remoteClustersNodes = new HashMap<>(); for (String clusterName : settings.names()) { String[] remoteHosts = settings.getAsArray(clusterName); for (String remoteHost : remoteHosts) { - String[] strings = remoteHost.split(":"); - String host = strings[0]; - int port = Integer.valueOf(strings[1]); + int portSeparator = remoteHost.lastIndexOf(':'); // in case we have a IPv6 address ie. [::1]:9300 + String host = remoteHost.substring(0, portSeparator); + int port = Integer.valueOf(remoteHost.substring(portSeparator+1)); DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, - new TransportAddress(new InetSocketAddress(host, port)), Version.CURRENT.minimumCompatibilityVersion()); + new TransportAddress(new InetSocketAddress(host, port)), Version.CURRENT.minimumCompatibilityVersion()); //don't connect yet as that would require the remote node to be up and would fail the local node startup otherwise List nodes = remoteClustersNodes.get(clusterName); if (nodes == null) { @@ -149,7 +153,11 @@ public class SearchTransportService extends AbstractComponent { nodes.add(node); } } - remoteClustersSeeds = remoteClustersNodes; + return remoteClustersNodes; + } + + private void setRemoteClustersSeeds(Settings settings) { + remoteClustersSeeds = builtRemoteClustersSeeds(settings); } private DiscoveryNode connectToRemoteCluster(String clusterName) { diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index c56c7bac94e..fb767dc4dbf 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -137,38 +137,35 @@ public class TransportSearchAction extends HandledTransportAction>() { - @Override - public void onResponse(Map searchShardsResponses) { - List remoteShardIterators = new ArrayList<>(); - Set remoteNodes = new HashSet<>(); - Set remoteUUIDs = new HashSet<>(); - for (Map.Entry entry : searchShardsResponses.entrySet()) { - String clusterName = entry.getKey(); - ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); - Collections.addAll(remoteNodes, searchShardsResponse.getNodes()); - for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { - //add the cluster name to the remote index names for indices disambiguation - //this ends up in the hits returned with the search response - Index index = new Index(clusterName + REMOTE_CLUSTER_INDEX_SEPARATOR + - clusterSearchShardsGroup.getShardId().getIndex().getName(), - clusterSearchShardsGroup.getShardId().getIndex().getUUID()); - ShardId shardId = new ShardId(index, clusterSearchShardsGroup.getShardId().getId()); - ShardIterator shardIterator = new PlainShardIterator(shardId, - Arrays.asList(clusterSearchShardsGroup.getShards())); - remoteShardIterators.add(shardIterator); - remoteUUIDs.add(clusterSearchShardsGroup.getShardId().getIndex().getUUID()); - } - } - executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, - remoteUUIDs.toArray(new String[remoteUUIDs.size()]), remoteShardIterators, remoteNodes, listener); - } + ActionListener.wrap((searchShardsResponses) -> { + List remoteShardIterators = new ArrayList<>(); + Set remoteNodes = new HashSet<>(); + Set remoteUUIDs = new HashSet<>(); + processRemoteShards(searchShardsResponses, remoteShardIterators, remoteNodes, remoteUUIDs); + executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, + remoteUUIDs.toArray(new String[remoteUUIDs.size()]), remoteShardIterators, remoteNodes, listener); + }, listener::onFailure)); + } + } - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }); + private void processRemoteShards(Map searchShardsResponses, + List remoteShardIterators, Set remoteNodes, Set remoteUUIDs) { + for (Map.Entry entry : searchShardsResponses.entrySet()) { + String clusterName = entry.getKey(); + ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); + Collections.addAll(remoteNodes, searchShardsResponse.getNodes()); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + //add the cluster name to the remote index names for indices disambiguation + //this ends up in the hits returned with the search response + Index index = new Index(clusterName + REMOTE_CLUSTER_INDEX_SEPARATOR + + clusterSearchShardsGroup.getShardId().getIndex().getName(), + clusterSearchShardsGroup.getShardId().getIndex().getUUID()); + ShardId shardId = new ShardId(index, clusterSearchShardsGroup.getShardId().getId()); + ShardIterator shardIterator = new PlainShardIterator(shardId, + Arrays.asList(clusterSearchShardsGroup.getShards())); + remoteShardIterators.add(shardIterator); + remoteUUIDs.add(clusterSearchShardsGroup.getShardId().getIndex().getUUID()); + } } } @@ -181,8 +178,13 @@ public class TransportSearchAction extends HandledTransportAction 0) { + indices = new Index[0]; // don't search on ALL if nothing is specified + } else { + indices = indexNameExpressionResolver.concreteIndices(clusterState, searchRequest.indicesOptions(), + startTimeInMillis, localIndices); + } Map aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteUUIDs); Map> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(), searchRequest.indices()); diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java new file mode 100644 index 00000000000..81c26357543 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java @@ -0,0 +1,67 @@ +/* + * 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.action.search; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.test.ESTestCase; + +import java.net.InetSocketAddress; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; + +public class SearchTransportServiceTests extends ESTestCase { + + public void testRemoteClusterSeedSetting() { + // simple validation + SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() + .put("action.search.remote.foo", "192.168.0.1:8080") + .put("action.search.remote.bar", "[::1]:9090").build()); + + expectThrows(IllegalArgumentException.class, () -> + SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() + .put("action.search.remote.foo", "192.168.0.1").build())); + } + + public void testBuiltRemoteClustersSeeds() { + Map> map = SearchTransportService.builtRemoteClustersSeeds( + SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() + .put("action.search.remote.foo", "192.168.0.1:8080") + .put("action.search.remote.bar", "[::1]:9090").build())); + assertEquals(2, map.size()); + assertTrue(map.containsKey("foo")); + assertTrue(map.containsKey("bar")); + assertEquals(1, map.get("foo").size()); + assertEquals(1, map.get("bar").size()); + + DiscoveryNode foo = map.get("foo").get(0); + assertEquals(foo.getAddress(), new TransportAddress(new InetSocketAddress("192.168.0.1", 8080))); + assertEquals(foo.getId(), "foo#192.168.0.1:8080"); + assertEquals(foo.getVersion(), Version.CURRENT.minimumCompatibilityVersion()); + + DiscoveryNode bar = map.get("bar").get(0); + assertEquals(bar.getAddress(), new TransportAddress(new InetSocketAddress("[::1]", 9090))); + assertEquals(bar.getId(), "bar#[::1]:9090"); + assertEquals(bar.getVersion(), Version.CURRENT.minimumCompatibilityVersion()); + } +} diff --git a/qa/multi-cluster-search/build.gradle b/qa/multi-cluster-search/build.gradle new file mode 100644 index 00000000000..7721ed660f0 --- /dev/null +++ b/qa/multi-cluster-search/build.gradle @@ -0,0 +1,53 @@ +/* + * 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. + */ + +import org.elasticsearch.gradle.test.RestIntegTestTask + +apply plugin: 'elasticsearch.standalone-test' + +task remoteClusterTest(type: RestIntegTestTask) { + mustRunAfter(precommit) + cluster { + distribution = 'zip' + numNodes = 1 + clusterName = 'remote-cluster' + } + systemProperty 'tests.rest.suite', 'remote_cluster' +} + +task mixedClusterTest(type: RestIntegTestTask) { + dependsOn(remoteClusterTest) + cluster { + distribution = 'zip' + dynamicSettings = { -> + Collections.singletonMap("action.search.remote.my_remote_cluster", + "\"${remoteClusterTest.nodes.get(0).transportUri()}\"") + } + } + systemProperty 'tests.rest.suite', 'multi_cluster' + finalizedBy 'remoteClusterTest#stop' +} + +task integTest { + dependsOn = [mixedClusterTest] +} + +test.enabled = false // no unit tests for multi-cluster-search, only the rest integration test + +check.dependsOn(integTest) diff --git a/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java b/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java new file mode 100644 index 00000000000..ee140da23e7 --- /dev/null +++ b/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java @@ -0,0 +1,48 @@ +/* + * 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.upgrades; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; +import org.apache.lucene.util.TimeUnits; +import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; +import org.elasticsearch.test.rest.yaml.parser.ClientYamlTestParseException; + +import java.io.IOException; + +@TimeoutSuite(millis = 5 * TimeUnits.MINUTE) // to account for slow as hell VMs +public class MultiClusterSearchYamlTestSuiteIT extends ESClientYamlSuiteTestCase { + + @Override + protected boolean preserveIndicesUponCompletion() { + return true; + } + + public MultiClusterSearchYamlTestSuiteIT(ClientYamlTestCandidate testCandidate) { + super(testCandidate); + } + + @ParametersFactory + public static Iterable parameters() throws IOException, ClientYamlTestParseException { + return createParameters(); + } +} + diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml new file mode 100644 index 00000000000..a6acdf7b98f --- /dev/null +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -0,0 +1,47 @@ +--- +"Index data and search on the mixed cluster": + + - do: + indices.create: + index: test_index_1 + body: + settings: + index: + number_of_replicas: 0 + + - do: + bulk: + refresh: true + body: + - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"f1": "v1_old", "f2": 0}' + - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"f1": "v2_old", "f2": 1}' + - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"f1": "v3_old", "f2": 2}' + - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"f1": "v4_old", "f2": 3}' + - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"f1": "v5_old", "f2": 4}' + + - do: + indices.flush: + index: test_index_1 + + - do: + search: + index: test_index_1,my_remote_cluster|test_index + + - match: { hits.total: 10 } + + - do: + search: + index: my_remote_cluster|test_index + + - match: { hits.total: 5} + + - do: + search: + index: test_index_1 + + - match: { hits.total: 5} diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml new file mode 100644 index 00000000000..5f55407d6a4 --- /dev/null +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml @@ -0,0 +1,35 @@ +--- +"Index data and search on the old cluster": + + - do: + indices.create: + index: test_index + body: + settings: + index: + number_of_replicas: 0 + + - do: + bulk: + refresh: true + body: + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "v1_old", "f2": 0}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "v2_old", "f2": 1}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "v3_old", "f2": 2}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "v4_old", "f2": 3}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "v5_old", "f2": 4}' + + - do: + indices.flush: + index: test_index + + - do: + search: + index: test_index + + - match: { hits.total: 5 } diff --git a/settings.gradle b/settings.gradle index f3fb14674eb..65f47fb84c7 100644 --- a/settings.gradle +++ b/settings.gradle @@ -54,6 +54,7 @@ List projects = [ 'qa:evil-tests', 'qa:no-bootstrap-tests', 'qa:rolling-upgrade', + 'qa:multi-cluster-search', 'qa:smoke-test-client', 'qa:smoke-test-http', 'qa:smoke-test-ingest-with-all-dependencies', From 7804aa09886cbb30870ed7ca9b28388859cc8c51 Mon Sep 17 00:00:00 2001 From: javanna Date: Thu, 24 Nov 2016 13:02:30 +0100 Subject: [PATCH 03/59] fix usage of forbidden InetAddress constructor, also fix typo built->build --- .../action/search/SearchTransportService.java | 25 +++++++++++++++---- .../search/SearchTransportServiceTests.java | 12 ++++----- 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 65254a75193..08d7d907661 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -63,7 +63,9 @@ import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.net.InetAddress; import java.net.InetSocketAddress; +import java.net.UnknownHostException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -120,7 +122,13 @@ public class SearchTransportService extends AbstractComponent { throw new IllegalArgumentException("remote hosts need to be configured as [host:port], found [" + remoteHost + "] " + "instead for remote cluster [" + clusterName + "]"); } - String port = remoteHost.substring(portSeparator+1); + String host = remoteHost.substring(0, portSeparator); + try { + InetAddress.getByName(host); + } catch (UnknownHostException e) { + throw new IllegalArgumentException("unknown host [" + host + "]", e); + } + String port = remoteHost.substring(portSeparator + 1); try { Integer portValue = Integer.valueOf(port); if (portValue <= 0) { @@ -134,16 +142,23 @@ public class SearchTransportService extends AbstractComponent { } } - static Map> builtRemoteClustersSeeds(Settings settings) { + static Map> buildRemoteClustersSeeds(Settings settings) { Map> remoteClustersNodes = new HashMap<>(); for (String clusterName : settings.names()) { String[] remoteHosts = settings.getAsArray(clusterName); for (String remoteHost : remoteHosts) { int portSeparator = remoteHost.lastIndexOf(':'); // in case we have a IPv6 address ie. [::1]:9300 String host = remoteHost.substring(0, portSeparator); - int port = Integer.valueOf(remoteHost.substring(portSeparator+1)); + InetAddress hostAddress; + try { + hostAddress = InetAddress.getByName(host); + } catch (UnknownHostException e) { + throw new IllegalArgumentException("unknown host [" + host + "]", e); + } + int port = Integer.valueOf(remoteHost.substring(portSeparator + 1)); DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, - new TransportAddress(new InetSocketAddress(host, port)), Version.CURRENT.minimumCompatibilityVersion()); + new TransportAddress(new InetSocketAddress(hostAddress, port)), + Version.CURRENT.minimumCompatibilityVersion()); //don't connect yet as that would require the remote node to be up and would fail the local node startup otherwise List nodes = remoteClustersNodes.get(clusterName); if (nodes == null) { @@ -157,7 +172,7 @@ public class SearchTransportService extends AbstractComponent { } private void setRemoteClustersSeeds(Settings settings) { - remoteClustersSeeds = builtRemoteClustersSeeds(settings); + remoteClustersSeeds = buildRemoteClustersSeeds(settings); } private DiscoveryNode connectToRemoteCluster(String clusterName) { diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java index 81c26357543..def29de9688 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java @@ -24,9 +24,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESTestCase; +import java.net.InetAddress; import java.net.InetSocketAddress; -import java.util.Collections; -import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -43,8 +42,8 @@ public class SearchTransportServiceTests extends ESTestCase { .put("action.search.remote.foo", "192.168.0.1").build())); } - public void testBuiltRemoteClustersSeeds() { - Map> map = SearchTransportService.builtRemoteClustersSeeds( + public void testBuiltRemoteClustersSeeds() throws Exception { + Map> map = SearchTransportService.buildRemoteClustersSeeds( SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() .put("action.search.remote.foo", "192.168.0.1:8080") .put("action.search.remote.bar", "[::1]:9090").build())); @@ -55,12 +54,13 @@ public class SearchTransportServiceTests extends ESTestCase { assertEquals(1, map.get("bar").size()); DiscoveryNode foo = map.get("foo").get(0); - assertEquals(foo.getAddress(), new TransportAddress(new InetSocketAddress("192.168.0.1", 8080))); + + assertEquals(foo.getAddress(), new TransportAddress(new InetSocketAddress(InetAddress.getByName("192.168.0.1"), 8080))); assertEquals(foo.getId(), "foo#192.168.0.1:8080"); assertEquals(foo.getVersion(), Version.CURRENT.minimumCompatibilityVersion()); DiscoveryNode bar = map.get("bar").get(0); - assertEquals(bar.getAddress(), new TransportAddress(new InetSocketAddress("[::1]", 9090))); + assertEquals(bar.getAddress(), new TransportAddress(new InetSocketAddress(InetAddress.getByName("[::1]"), 9090))); assertEquals(bar.getId(), "bar#[::1]:9090"); assertEquals(bar.getVersion(), Version.CURRENT.minimumCompatibilityVersion()); } From 276adbd9903907ef6aaba713c6752605e6782f44 Mon Sep 17 00:00:00 2001 From: javanna Date: Thu, 24 Nov 2016 15:35:52 +0100 Subject: [PATCH 04/59] Enable cross cluster search only when at least one remote cluster is configured If no remote clusters are registered, we shouldn't even try resolving remote clusters as part of indices names. Just go ahead and treat the index as a local one, which may or may not exist. In fact, the character that we use a separator may be part of an alias name, or part of a date math expression. We will go ahead with the remote search only if the prefix before the first occurrence of the separator is an actual registered cluster. Otherwise we will treat the index as a local one. Previously we would go remotely anytime we'd find the separator in an index name, which caused false positives with aliases and date math expressions. It is also safer to not go remotely unless there is some remote cluster registered. We'd also throw exception whenever an unknown remote cluster was referred to, but this could again conflict with date math expressions or aliases: say we have some remote cluster configured, and we are using the separator in a date math expression, we only want to go remotely when the prefix matches a configured cluster, stay local otherwise, as the index name may still be valid locally. --- .../action/search/SearchTransportService.java | 8 ++++ .../action/search/TransportSearchAction.java | 41 +++++++++++-------- 2 files changed, 31 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 08d7d907661..e26f248d081 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -175,6 +175,14 @@ public class SearchTransportService extends AbstractComponent { remoteClustersSeeds = buildRemoteClustersSeeds(settings); } + boolean isCrossClusterSearchEnabled() { + return remoteClustersSeeds.isEmpty() == false; + } + + boolean isRemoteClusterRegistered(String clusterName) { + return remoteClustersSeeds.containsKey(clusterName); + } + private DiscoveryNode connectToRemoteCluster(String clusterName) { List nodes = remoteClustersSeeds.get(clusterName); if (nodes == null) { diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index fb767dc4dbf..912c0d30ead 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -109,29 +109,34 @@ public class TransportSearchAction extends HandledTransportAction localIndicesList = new ArrayList<>(); + final String[] localIndices; final Map> remoteIndicesByCluster = new HashMap<>(); - for (String index : searchRequest.indices()) { - int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); - if (i >= 0) { - String remoteCluster = index.substring(0, i); - String remoteIndex = index.substring(i + 1); - List indices = remoteIndicesByCluster.get(remoteCluster); - if (indices == null) { - indices = new ArrayList<>(); - remoteIndicesByCluster.put(remoteCluster, indices); + if (searchTransportService.isCrossClusterSearchEnabled()) { + List localIndicesList = new ArrayList<>(); + for (String index : searchRequest.indices()) { + int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); + if (i >= 0) { + String remoteCluster = index.substring(0, i); + if (searchTransportService.isRemoteClusterRegistered(remoteCluster)) { + String remoteIndex = index.substring(i + 1); + List indices = remoteIndicesByCluster.get(remoteCluster); + if (indices == null) { + indices = new ArrayList<>(); + remoteIndicesByCluster.put(remoteCluster, indices); + } + indices.add(remoteIndex); + } else { + localIndicesList.add(index); + } + } else { + localIndicesList.add(index); } - indices.add(remoteIndex); - } else { - localIndicesList.add(index); } + localIndices = localIndicesList.toArray(new String[localIndicesList.size()]); + } else { + localIndices = searchRequest.indices(); } - String[] localIndices = localIndicesList.toArray(new String[localIndicesList.size()]); - if (remoteIndicesByCluster.isEmpty()) { executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, Strings.EMPTY_ARRAY, Collections.emptyList(), Collections.emptySet(), listener); From 89a23849880a2ad7f61728eaf8531c01ac8a8f51 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 24 Nov 2016 15:44:06 +0100 Subject: [PATCH 05/59] Use GString and closures to delay evaluating remote cluster URL until runtime --- .../gradle/test/ClusterConfiguration.groovy | 9 --------- .../gradle/test/ClusterFormationTasks.groovy | 2 -- qa/multi-cluster-search/build.gradle | 6 ++---- 3 files changed, 2 insertions(+), 15 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy index 2c5159bc2d8..ca4957f7a6c 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterConfiguration.groovy @@ -96,15 +96,6 @@ class ClusterConfiguration { return seedNode.transportUri() } - /** - * A closure to call which returns a map of settings. - * - * This can be used to pass settings to a cluster that are not available at evaluation time ie. - * the address of a remote cluster etc. - */ - @Input - Closure dynamicSettings = { -> Collections.emptyMap() } - /** * A closure to call before the cluster is considered ready. The closure is passed the node info, * as well as a groovy AntBuilder, to enable running ant condition checks. The default wait diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index b911a7b5ea8..73d7a3c7cd9 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -294,8 +294,6 @@ class ClusterFormationTasks { if (unicastTransportUri != null) { esConfig['discovery.zen.ping.unicast.hosts'] = "\"${unicastTransportUri}\"" } - Map dynamicSettings = node.config.dynamicSettings(); - esConfig.putAll(dynamicSettings) File configFile = new File(node.confDir, 'elasticsearch.yml') logger.info("Configuring ${configFile}") configFile.setText(esConfig.collect { key, value -> "${key}: ${value}" }.join('\n'), 'UTF-8') diff --git a/qa/multi-cluster-search/build.gradle b/qa/multi-cluster-search/build.gradle index 7721ed660f0..fe4e9524165 100644 --- a/qa/multi-cluster-search/build.gradle +++ b/qa/multi-cluster-search/build.gradle @@ -35,10 +35,8 @@ task mixedClusterTest(type: RestIntegTestTask) { dependsOn(remoteClusterTest) cluster { distribution = 'zip' - dynamicSettings = { -> - Collections.singletonMap("action.search.remote.my_remote_cluster", - "\"${remoteClusterTest.nodes.get(0).transportUri()}\"") - } + setting 'action.search.remote.my_remote_cluster', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" + } systemProperty 'tests.rest.suite', 'multi_cluster' finalizedBy 'remoteClusterTest#stop' From 32eeaef6cfe2c0bb49860fc641fd3c65c5602023 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 25 Nov 2016 00:17:50 +0100 Subject: [PATCH 06/59] Add WIP support for alias filters and additional tests --- .../action/search/SearchTransportService.java | 11 ++-- .../action/search/TransportSearchAction.java | 39 ++++++++---- .../search/internal/AliasFilter.java | 9 +++ .../test/multi_cluster/10_basic.yaml | 61 +++++++++++++++++-- .../test/remote_cluster/10_basic.yaml | 25 ++++++-- 5 files changed, 119 insertions(+), 26 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index e26f248d081..0ccec9adc13 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -193,16 +193,19 @@ public class SearchTransportService extends AbstractComponent { try { //TODO we should call liveness api and get back an updated discovery node. that would have the updated version // and would make the search shards call more future-proof. Also validating the cluster name may be a thing. - connectToRemoteNode(node); + return connectToRemoteNode(node); } catch(ConnectTransportException e) { throw new ConnectTransportException(node, "unable to connect to remote cluster [" + clusterName + "]", e); } - return node; } - void connectToRemoteNode(DiscoveryNode remoteNode) { + DiscoveryNode connectToRemoteNode(DiscoveryNode remoteNode) { + DiscoveryNode discoveryNode = transportService.connectToNodeLightAndHandshake(remoteNode, 10000, false); + transportService.disconnectFromNode(remoteNode); // disconnect the light connection //TODO should the list of seeds get updated based on nodes that we get back from the remote cluster through search_shards? - transportService.connectToNode(remoteNode); + // now go and do a real connection with the updated version of the node + transportService.connectToNode(discoveryNode); + return discoveryNode; //TODO is it ok to connect and leave the node connected? It will be pinged from now on? } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 912c0d30ead..a7af67feef3 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -88,7 +88,8 @@ public class TransportSearchAction extends HandledTransportAction buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState, - Index[] concreteIndices, String[] remoteUUIDs) { + Index[] concreteIndices, String[] remoteUUIDs, + Map remoteAliasMap) { final Map aliasFilterMap = new HashMap<>(); for (Index index : concreteIndices) { clusterState.blocks().indexBlockedRaiseException(ClusterBlockLevel.READ, index.getName()); @@ -99,7 +100,9 @@ public class TransportSearchAction extends HandledTransportAction { List remoteShardIterators = new ArrayList<>(); Set remoteNodes = new HashSet<>(); Set remoteUUIDs = new HashSet<>(); - processRemoteShards(searchShardsResponses, remoteShardIterators, remoteNodes, remoteUUIDs); + Map remoteAliasFilters = new HashMap<>(); + processRemoteShards(searchShardsResponses, remoteShardIterators, remoteNodes, remoteUUIDs, remoteAliasFilters); executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, - remoteUUIDs.toArray(new String[remoteUUIDs.size()]), remoteShardIterators, remoteNodes, listener); + remoteUUIDs.toArray(new String[remoteUUIDs.size()]), remoteShardIterators, remoteNodes, remoteAliasFilters, listener); }, listener::onFailure)); } } private void processRemoteShards(Map searchShardsResponses, - List remoteShardIterators, Set remoteNodes, Set remoteUUIDs) { + List remoteShardIterators, Set remoteNodes, Set remoteUUIDs, + Map aliasFilterMap) { for (Map.Entry entry : searchShardsResponses.entrySet()) { String clusterName = entry.getKey(); ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); Collections.addAll(remoteNodes, searchShardsResponse.getNodes()); + Map indicesAndFilters = searchShardsResponse.getIndicesAndFilters(); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { //add the cluster name to the remote index names for indices disambiguation //this ends up in the hits returned with the search response + ShardId sid = clusterSearchShardsGroup.getShardId(); Index index = new Index(clusterName + REMOTE_CLUSTER_INDEX_SEPARATOR + - clusterSearchShardsGroup.getShardId().getIndex().getName(), - clusterSearchShardsGroup.getShardId().getIndex().getUUID()); - ShardId shardId = new ShardId(index, clusterSearchShardsGroup.getShardId().getId()); - ShardIterator shardIterator = new PlainShardIterator(shardId, + sid.getIndex().getName(), + sid.getIndex().getUUID()); + ShardIterator shardIterator = new PlainShardIterator(new ShardId(index, sid.getId()), Arrays.asList(clusterSearchShardsGroup.getShards())); remoteShardIterators.add(shardIterator); - remoteUUIDs.add(clusterSearchShardsGroup.getShardId().getIndex().getUUID()); + remoteUUIDs.add(sid.getIndex().getUUID()); + if (indicesAndFilters != null) { + AliasFilter aliasFilter = indicesAndFilters.get(sid.getIndexName()); + if (aliasFilter != null) { + // here we have to map the filters to the UUID since from now on we use the uuid for the lookup + aliasFilterMap.put(sid.getIndex().getUUID(), aliasFilter); + } + } } } } private void executeSearch(SearchTask task, long startTimeInMillis, SearchRequest searchRequest, String[] localIndices, String[] remoteUUIDs, List remoteShardIterators, Set remoteNodes, + Map remoteAliasMap, ActionListener listener) { ClusterState clusterState = clusterService.state(); @@ -190,7 +205,7 @@ public class TransportSearchAction extends HandledTransportAction aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteUUIDs); + Map aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteUUIDs, remoteAliasMap); Map> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(), searchRequest.indices()); String[] concreteIndices = new String[indices.length]; diff --git a/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java b/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java index 4424f204318..39de37db597 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java +++ b/core/src/main/java/org/elasticsearch/search/internal/AliasFilter.java @@ -118,4 +118,13 @@ public final class AliasFilter implements Writeable { public int hashCode() { return Objects.hash(aliases, filter, reparseAliases); } + + @Override + public String toString() { + return "AliasFilter{" + + "aliases=" + Arrays.toString(aliases) + + ", filter=" + filter + + ", reparseAliases=" + reparseAliases + + '}'; + } } diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index a6acdf7b98f..0a1af86e482 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -14,15 +14,15 @@ refresh: true body: - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' - - '{"f1": "v1_old", "f2": 0}' + - '{"f1": "local_cluster", "filter_field": 0}' - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' - - '{"f1": "v2_old", "f2": 1}' + - '{"f1": "local_cluster", "filter_field": 1}' - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' - - '{"f1": "v3_old", "f2": 2}' + - '{"f1": "local_cluster", "filter_field": 0}' - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' - - '{"f1": "v4_old", "f2": 3}' + - '{"f1": "local_cluster", "filter_field": 1}' - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' - - '{"f1": "v5_old", "f2": 4}' + - '{"f1": "local_cluster", "filter_field": 0}' - do: indices.flush: @@ -34,6 +34,18 @@ - match: { hits.total: 10 } + - do: + search: + index: test_index_1,my_remote_cluster|test_index + body: + query: + term: + f1: remote_cluster + + - match: { hits.total: 5} + - match: { hits.hits.0._index: "my_remote_cluster|test_index"} + + - do: search: index: my_remote_cluster|test_index @@ -45,3 +57,42 @@ index: test_index_1 - match: { hits.total: 5} + +--- +"Add transient remote cluster based on the preset cluster": + - do: + cluster.get_settings: + include_defaults: true + + - set: { defaults.action.search.remote.my_remote_cluster: remote_ip } + + - do: + cluster.put_settings: + body: + transient: + action.search.remote.test_remote_cluster: $remote_ip + flat_settings: true + + - match: {transient: {action.search.remote.test_remote_cluster: $remote_ip}} + + - do: + search: + index: test_remote_cluster|test_index + + - match: { hits.total: 5 } + - match: { hits.hits.0._index: "test_remote_cluster|test_index" } + + +--- +"Search an filtered alias on the remote cluster": + + - do: + search: + index: my_remote_cluster|aliased_test_index + + - match: { hits.total: 2 } + - match: { hits.hits.0._source.filter_field: 1 } + - match: { hits.hits.0._index: "my_remote_cluster|test_index" } + + + diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml index 5f55407d6a4..39e62d75cfa 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml @@ -8,21 +8,26 @@ settings: index: number_of_replicas: 0 + aliases: + aliased_test_index: # we use this alias in the multi cluster test to very filtered aliases work + filter: + term: + filter_field : 1 - do: bulk: refresh: true body: - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "v1_old", "f2": 0}' + - '{"f1": "remote_cluster", "filter_field": 0}' - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "v2_old", "f2": 1}' + - '{"f1": "remote_cluster", "filter_field": 1}' - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "v3_old", "f2": 2}' + - '{"f1": "remote_cluster", "filter_field": 0}' - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "v4_old", "f2": 3}' + - '{"f1": "remote_cluster", "filter_field": 1}' - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "v5_old", "f2": 4}' + - '{"f1": "remote_cluster", "filter_field": 0}' - do: indices.flush: @@ -33,3 +38,13 @@ index: test_index - match: { hits.total: 5 } + + - do: + search: + index: aliased_test_index + + - match: { hits.total: 2 } + - match: { hits.hits.0._source.filter_field: 1 } + - match: { hits.hits.0._index: "test_index" } + + From 8cb2cec0c2af20de914e85e9218e400632866855 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 00:20:48 +0100 Subject: [PATCH 07/59] Don't carry remoteUUIDs around, simplify remote alias filter handling Also added explanation of when and why alias filters don't work at the moment --- .../action/search/TransportSearchAction.java | 62 +++++++++---------- 1 file changed, 28 insertions(+), 34 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index a7af67feef3..d5d35de6fbb 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -88,8 +88,7 @@ public class TransportSearchAction extends HandledTransportAction buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState, - Index[] concreteIndices, String[] remoteUUIDs, - Map remoteAliasMap) { + Index[] concreteIndices, Map remoteAliasMap) { final Map aliasFilterMap = new HashMap<>(); for (Index index : concreteIndices) { clusterState.blocks().indexBlockedRaiseException(ClusterBlockLevel.READ, index.getName()); @@ -97,13 +96,7 @@ public class TransportSearchAction extends HandledTransportAction { List remoteShardIterators = new ArrayList<>(); Set remoteNodes = new HashSet<>(); - Set remoteUUIDs = new HashSet<>(); Map remoteAliasFilters = new HashMap<>(); - processRemoteShards(searchShardsResponses, remoteShardIterators, remoteNodes, remoteUUIDs, remoteAliasFilters); - executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, - remoteUUIDs.toArray(new String[remoteUUIDs.size()]), remoteShardIterators, remoteNodes, remoteAliasFilters, listener); + processRemoteShards(searchShardsResponses, remoteShardIterators, remoteNodes, remoteAliasFilters); + executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, remoteShardIterators, + remoteNodes, remoteAliasFilters, listener); }, listener::onFailure)); } } private void processRemoteShards(Map searchShardsResponses, - List remoteShardIterators, Set remoteNodes, Set remoteUUIDs, + List remoteShardIterators, Set remoteNodes, Map aliasFilterMap) { for (Map.Entry entry : searchShardsResponses.entrySet()) { String clusterName = entry.getKey(); @@ -169,29 +161,31 @@ public class TransportSearchAction extends HandledTransportAction remoteShardIterators, Set remoteNodes, - Map remoteAliasMap, - ActionListener listener) { + List remoteShardIterators, Set remoteNodes, + Map remoteAliasMap, ActionListener listener) { ClusterState clusterState = clusterService.state(); clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ); @@ -199,13 +193,13 @@ public class TransportSearchAction extends HandledTransportAction 0) { - indices = new Index[0]; // don't search on ALL if nothing is specified + if (localIndices.length == 0 && remoteShardIterators.size() > 0) { + indices = new Index[0]; // don't search on _all if only remote indices were specified } else { indices = indexNameExpressionResolver.concreteIndices(clusterState, searchRequest.indicesOptions(), startTimeInMillis, localIndices); } - Map aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteUUIDs, remoteAliasMap); + Map aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteAliasMap); Map> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(), searchRequest.indices()); String[] concreteIndices = new String[indices.length]; From f8d32c1bdf65a209597405dc6c28dc29f6e13a2b Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 00:23:32 +0100 Subject: [PATCH 08/59] remove outdated TODOs We will eagerly connect to all configured nodes, the liveness api call is not needed as we already do a light connect and get back the updated discovery node. --- .../elasticsearch/action/search/SearchTransportService.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 0ccec9adc13..f29c4068bf4 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -110,7 +110,7 @@ public class SearchTransportService extends AbstractComponent { } private static void validateRemoteClustersSeeds(Settings settings) { - //TODO add a static whitelist like in reindex from remote + //TODO do we need a static whitelist like in reindex from remote? for (String clusterName : settings.names()) { String[] remoteHosts = settings.getAsArray(clusterName); if (remoteHosts.length == 0) { @@ -191,8 +191,6 @@ public class SearchTransportService extends AbstractComponent { DiscoveryNode node = nodes.get(Randomness.get().nextInt(nodes.size())); //TODO we just take a random host for now, implement fallback in case of connect failure try { - //TODO we should call liveness api and get back an updated discovery node. that would have the updated version - // and would make the search shards call more future-proof. Also validating the cluster name may be a thing. return connectToRemoteNode(node); } catch(ConnectTransportException e) { throw new ConnectTransportException(node, "unable to connect to remote cluster [" + clusterName + "]", e); @@ -202,11 +200,9 @@ public class SearchTransportService extends AbstractComponent { DiscoveryNode connectToRemoteNode(DiscoveryNode remoteNode) { DiscoveryNode discoveryNode = transportService.connectToNodeLightAndHandshake(remoteNode, 10000, false); transportService.disconnectFromNode(remoteNode); // disconnect the light connection - //TODO should the list of seeds get updated based on nodes that we get back from the remote cluster through search_shards? // now go and do a real connection with the updated version of the node transportService.connectToNode(discoveryNode); return discoveryNode; - //TODO is it ok to connect and leave the node connected? It will be pinged from now on? } void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, From 247c332a571567b837043a162fb29bf64e735a3b Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 00:27:42 +0100 Subject: [PATCH 09/59] add TODO on validating cluster name of remote clusters --- .../org/elasticsearch/action/search/SearchTransportService.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index f29c4068bf4..a347d463a09 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -202,6 +202,8 @@ public class SearchTransportService extends AbstractComponent { transportService.disconnectFromNode(remoteNode); // disconnect the light connection // now go and do a real connection with the updated version of the node transportService.connectToNode(discoveryNode); + //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes + //belong to the same cluster, also validate the cluster name against the configured label and make sure they match return discoveryNode; } From cb97b91a6037980cd87ec5b6d90c80fea2079ff2 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 00:34:33 +0100 Subject: [PATCH 10/59] Don't light connect to discovery nodes received from search shards Those discovery nodes are already enough to go ahead with the remote search, no need to light connect and update the discovery node, it's already updated. --- .../action/search/SearchTransportService.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index a347d463a09..38bdea35631 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -188,23 +188,23 @@ public class SearchTransportService extends AbstractComponent { if (nodes == null) { throw new IllegalArgumentException("no remote cluster configured with name [" + clusterName + "]"); } - DiscoveryNode node = nodes.get(Randomness.get().nextInt(nodes.size())); + DiscoveryNode remoteNode = nodes.get(Randomness.get().nextInt(nodes.size())); //TODO we just take a random host for now, implement fallback in case of connect failure try { - return connectToRemoteNode(node); + DiscoveryNode discoveryNode = transportService.connectToNodeLightAndHandshake(remoteNode, 10000, false); + transportService.disconnectFromNode(remoteNode); // disconnect the light connection + // now go and do a real connection with the updated version of the node + connectToRemoteNode(discoveryNode); + //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes + //belong to the same cluster, also validate the cluster name against the configured label and make sure they match + return discoveryNode; } catch(ConnectTransportException e) { - throw new ConnectTransportException(node, "unable to connect to remote cluster [" + clusterName + "]", e); + throw new ConnectTransportException(remoteNode, "unable to connect to remote cluster [" + clusterName + "]", e); } } - DiscoveryNode connectToRemoteNode(DiscoveryNode remoteNode) { - DiscoveryNode discoveryNode = transportService.connectToNodeLightAndHandshake(remoteNode, 10000, false); - transportService.disconnectFromNode(remoteNode); // disconnect the light connection - // now go and do a real connection with the updated version of the node - transportService.connectToNode(discoveryNode); - //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes - //belong to the same cluster, also validate the cluster name against the configured label and make sure they match - return discoveryNode; + void connectToRemoteNode(DiscoveryNode remoteNode) { + transportService.connectToNode(remoteNode); } void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, @@ -214,6 +214,7 @@ public class SearchTransportService extends AbstractComponent { final AtomicReference transportException = new AtomicReference<>(); for (Map.Entry> entry : remoteIndicesByCluster.entrySet()) { final String clusterName = entry.getKey(); + //TODO we should rather eagerly connect to every configured remote node of all remote clusters final DiscoveryNode node = connectToRemoteCluster(clusterName); final List indices = entry.getValue(); //local true so we don't go to the master for each single remote search From 61029389897e268e78bde8d790e5dbb4f9209cee Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 00:49:04 +0100 Subject: [PATCH 11/59] adapt node connection to upstream changes --- .../elasticsearch/action/search/SearchTransportService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 38bdea35631..8b186be6af7 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -191,12 +191,12 @@ public class SearchTransportService extends AbstractComponent { DiscoveryNode remoteNode = nodes.get(Randomness.get().nextInt(nodes.size())); //TODO we just take a random host for now, implement fallback in case of connect failure try { - DiscoveryNode discoveryNode = transportService.connectToNodeLightAndHandshake(remoteNode, 10000, false); + //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes + //belong to the same cluster, also validate the cluster name against the configured label and make sure they match + DiscoveryNode discoveryNode = transportService.connectToNodeAndHandshake(remoteNode, 10000, false); transportService.disconnectFromNode(remoteNode); // disconnect the light connection // now go and do a real connection with the updated version of the node connectToRemoteNode(discoveryNode); - //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes - //belong to the same cluster, also validate the cluster name against the configured label and make sure they match return discoveryNode; } catch(ConnectTransportException e) { throw new ConnectTransportException(remoteNode, "unable to connect to remote cluster [" + clusterName + "]", e); From bd816f02eeaed55dfd8521c0bf80962d425722d7 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 01:00:20 +0100 Subject: [PATCH 12/59] [TEST] Remove flush calls that are not needed --- .../resources/rest-api-spec/test/multi_cluster/10_basic.yaml | 4 ---- .../resources/rest-api-spec/test/remote_cluster/10_basic.yaml | 4 ---- 2 files changed, 8 deletions(-) diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index 0a1af86e482..a4dbe9e35ae 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -24,10 +24,6 @@ - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 0}' - - do: - indices.flush: - index: test_index_1 - - do: search: index: test_index_1,my_remote_cluster|test_index diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml index 39e62d75cfa..6318a2b069d 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml @@ -29,10 +29,6 @@ - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "remote_cluster", "filter_field": 0}' - - do: - indices.flush: - index: test_index - - do: search: index: test_index From f9eaee3c9ce695638305a7bdebd2e6834aa454b4 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 01:02:37 +0100 Subject: [PATCH 13/59] [TEST] rename local index so that the name is the same as the remote one This way we make sure that index names are disambiguated. --- .../test/multi_cluster/10_basic.yaml | 21 ++++++++----------- 1 file changed, 9 insertions(+), 12 deletions(-) diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index a4dbe9e35ae..6f3a5542cb5 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -3,7 +3,7 @@ - do: indices.create: - index: test_index_1 + index: test_index body: settings: index: @@ -13,26 +13,26 @@ bulk: refresh: true body: - - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 0}' - - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 1}' - - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 0}' - - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 1}' - - '{"index": {"_index": "test_index_1", "_type": "test_type"}}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 0}' - do: search: - index: test_index_1,my_remote_cluster|test_index + index: test_index,my_remote_cluster|test_index - match: { hits.total: 10 } - do: search: - index: test_index_1,my_remote_cluster|test_index + index: test_index,my_remote_cluster|test_index body: query: term: @@ -50,7 +50,7 @@ - do: search: - index: test_index_1 + index: test_index - match: { hits.total: 5} @@ -89,6 +89,3 @@ - match: { hits.total: 2 } - match: { hits.hits.0._source.filter_field: 1 } - match: { hits.hits.0._index: "my_remote_cluster|test_index" } - - - From 7d4b1d94b93f3df735e09f2cc386052d34a78f7e Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 01:34:11 +0100 Subject: [PATCH 14/59] [TEST] test also aggregations and fix indendation --- .../test/multi_cluster/10_basic.yaml | 109 ++++++++++++------ .../test/remote_cluster/10_basic.yaml | 81 +++++++------ 2 files changed, 117 insertions(+), 73 deletions(-) diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index 6f3a5542cb5..a2750dafa34 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -2,17 +2,17 @@ "Index data and search on the mixed cluster": - do: - indices.create: - index: test_index - body: - settings: - index: - number_of_replicas: 0 + indices.create: + index: test_index + body: + settings: + index: + number_of_replicas: 0 - do: - bulk: - refresh: true - body: + bulk: + refresh: true + body: - '{"index": {"_index": "test_index", "_type": "test_type"}}' - '{"f1": "local_cluster", "filter_field": 0}' - '{"index": {"_index": "test_index", "_type": "test_type"}}' @@ -25,66 +25,99 @@ - '{"f1": "local_cluster", "filter_field": 0}' - do: - search: - index: test_index,my_remote_cluster|test_index + search: + index: test_index,my_remote_cluster|test_index + body: + aggs: + cluster: + terms: + field: f1.keyword - - match: { hits.total: 10 } + - match: { hits.total: 11 } + - length: { aggregations.cluster.buckets: 2 } + - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } + - match: { aggregations.cluster.buckets.0.doc_count: 6 } + - match: { aggregations.cluster.buckets.1.key: "local_cluster" } + - match: { aggregations.cluster.buckets.1.doc_count: 5 } - do: - search: - index: test_index,my_remote_cluster|test_index - body: - query: - term: - f1: remote_cluster + search: + index: test_index,my_remote_cluster|test_index + body: + query: + term: + f1: remote_cluster + aggs: + cluster: + terms: + field: f1.keyword - - match: { hits.total: 5} + - match: { hits.total: 6} - match: { hits.hits.0._index: "my_remote_cluster|test_index"} - + - length: { aggregations.cluster.buckets: 1 } + - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } + - match: { aggregations.cluster.buckets.0.doc_count: 6 } - do: - search: - index: my_remote_cluster|test_index + search: + index: my_remote_cluster|test_index + body: + aggs: + cluster: + terms: + field: f1.keyword - - match: { hits.total: 5} + - match: { hits.total: 6} + - match: { hits.hits.0._index: "my_remote_cluster|test_index"} + - length: { aggregations.cluster.buckets: 1 } + - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } + - match: { aggregations.cluster.buckets.0.doc_count: 6 } - do: - search: - index: test_index + search: + index: test_index + body: + aggs: + cluster: + terms: + field: f1.keyword - match: { hits.total: 5} + - match: { hits.hits.0._index: "test_index"} + - length: { aggregations.cluster.buckets: 1 } + - match: { aggregations.cluster.buckets.0.key: "local_cluster" } + - match: { aggregations.cluster.buckets.0.doc_count: 5 } --- "Add transient remote cluster based on the preset cluster": - do: - cluster.get_settings: - include_defaults: true + cluster.get_settings: + include_defaults: true - set: { defaults.action.search.remote.my_remote_cluster: remote_ip } - do: - cluster.put_settings: - body: - transient: - action.search.remote.test_remote_cluster: $remote_ip - flat_settings: true + cluster.put_settings: + flat_settings: true + body: + transient: + action.search.remote.test_remote_cluster: $remote_ip - match: {transient: {action.search.remote.test_remote_cluster: $remote_ip}} - do: - search: - index: test_remote_cluster|test_index + search: + index: test_remote_cluster|test_index - - match: { hits.total: 5 } + - match: { hits.total: 6 } - match: { hits.hits.0._index: "test_remote_cluster|test_index" } - --- "Search an filtered alias on the remote cluster": - do: - search: - index: my_remote_cluster|aliased_test_index + search: + index: my_remote_cluster|aliased_test_index - match: { hits.total: 2 } - match: { hits.hits.0._source.filter_field: 1 } diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml index 6318a2b069d..545729a699b 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml @@ -1,46 +1,57 @@ --- "Index data and search on the old cluster": - - do: + - do: indices.create: - index: test_index - body: - settings: - index: - number_of_replicas: 0 - aliases: - aliased_test_index: # we use this alias in the multi cluster test to very filtered aliases work - filter: - term: - filter_field : 1 + index: test_index + body: + settings: + index: + number_of_replicas: 0 + aliases: + aliased_test_index: # we use this alias in the multi cluster test to very filtered aliases work + filter: + term: + filter_field : 1 - - do: - bulk: - refresh: true - body: - - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "remote_cluster", "filter_field": 0}' - - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "remote_cluster", "filter_field": 1}' - - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "remote_cluster", "filter_field": 0}' - - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "remote_cluster", "filter_field": 1}' - - '{"index": {"_index": "test_index", "_type": "test_type"}}' - - '{"f1": "remote_cluster", "filter_field": 0}' + - do: + bulk: + refresh: true + body: + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "remote_cluster", "filter_field": 0}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "remote_cluster", "filter_field": 1}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "remote_cluster", "filter_field": 0}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "remote_cluster", "filter_field": 1}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "remote_cluster", "filter_field": 0}' + - '{"index": {"_index": "test_index", "_type": "test_type"}}' + - '{"f1": "remote_cluster", "filter_field": 0}' - - do: - search: - index: test_index - - match: { hits.total: 5 } - - - do: + - do: search: - index: aliased_test_index + index: test_index + body: + aggs: + cluster: + terms: + field: f1.keyword - - match: { hits.total: 2 } - - match: { hits.hits.0._source.filter_field: 1 } - - match: { hits.hits.0._index: "test_index" } + - match: { hits.total: 6 } + - length: { aggregations.cluster.buckets: 1 } + - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } + - match: { aggregations.cluster.buckets.0.doc_count: 6 } + + - do: + search: + index: aliased_test_index + + - match: { hits.total: 2 } + - match: { hits.hits.0._source.filter_field: 1 } + - match: { hits.hits.0._index: "test_index" } From 29d7c0d50d46b154f5148313c7906699ec237c2f Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 2 Dec 2016 01:38:19 +0100 Subject: [PATCH 15/59] [TEST] check _shards.total to make sure that cross cluster search hits the right number of shards --- .../rest-api-spec/test/multi_cluster/10_basic.yaml | 7 +++++++ .../rest-api-spec/test/remote_cluster/10_basic.yaml | 3 +++ 2 files changed, 10 insertions(+) diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index a2750dafa34..9166508db92 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -7,6 +7,7 @@ body: settings: index: + number_of_shards: 2 number_of_replicas: 0 - do: @@ -33,6 +34,7 @@ terms: field: f1.keyword + - match: { _shards.total: 5 } - match: { hits.total: 11 } - length: { aggregations.cluster.buckets: 2 } - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } @@ -52,6 +54,7 @@ terms: field: f1.keyword + - match: { _shards.total: 5 } - match: { hits.total: 6} - match: { hits.hits.0._index: "my_remote_cluster|test_index"} - length: { aggregations.cluster.buckets: 1 } @@ -67,6 +70,7 @@ terms: field: f1.keyword + - match: { _shards.total: 3 } - match: { hits.total: 6} - match: { hits.hits.0._index: "my_remote_cluster|test_index"} - length: { aggregations.cluster.buckets: 1 } @@ -82,6 +86,7 @@ terms: field: f1.keyword + - match: { _shards.total: 2 } - match: { hits.total: 5} - match: { hits.hits.0._index: "test_index"} - length: { aggregations.cluster.buckets: 1 } @@ -109,6 +114,7 @@ search: index: test_remote_cluster|test_index + - match: { _shards.total: 3 } - match: { hits.total: 6 } - match: { hits.hits.0._index: "test_remote_cluster|test_index" } @@ -119,6 +125,7 @@ search: index: my_remote_cluster|aliased_test_index + - match: { _shards.total: 3 } - match: { hits.total: 2 } - match: { hits.hits.0._source.filter_field: 1 } - match: { hits.hits.0._index: "my_remote_cluster|test_index" } diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml index 545729a699b..de4ae736f94 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/remote_cluster/10_basic.yaml @@ -7,6 +7,7 @@ body: settings: index: + number_of_shards: 3 number_of_replicas: 0 aliases: aliased_test_index: # we use this alias in the multi cluster test to very filtered aliases work @@ -41,6 +42,7 @@ terms: field: f1.keyword + - match: { _shards.total: 3 } - match: { hits.total: 6 } - length: { aggregations.cluster.buckets: 1 } - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } @@ -50,6 +52,7 @@ search: index: aliased_test_index + - match: { _shards.total: 3 } - match: { hits.total: 2 } - match: { hits.hits.0._source.filter_field: 1 } - match: { hits.hits.0._index: "test_index" } From 3515f782d140662bcc4dff127a7985728ad96282 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 20 Dec 2016 15:10:58 +0100 Subject: [PATCH 16/59] fix compile issue --- .../elasticsearch/action/search/SearchTransportService.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 8b186be6af7..bfe50425e40 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -193,11 +193,9 @@ public class SearchTransportService extends AbstractComponent { try { //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes //belong to the same cluster, also validate the cluster name against the configured label and make sure they match - DiscoveryNode discoveryNode = transportService.connectToNodeAndHandshake(remoteNode, 10000, false); - transportService.disconnectFromNode(remoteNode); // disconnect the light connection // now go and do a real connection with the updated version of the node - connectToRemoteNode(discoveryNode); - return discoveryNode; + connectToRemoteNode(remoteNode); + return remoteNode; } catch(ConnectTransportException e) { throw new ConnectTransportException(remoteNode, "unable to connect to remote cluster [" + clusterName + "]", e); } From 3625d64b7fcd1ee8d73d237666b21f8befffdbf2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 20 Dec 2016 22:23:11 +0100 Subject: [PATCH 17/59] Add remote cluster connections manager --- .../search/RemoteClusterConnection.java | 239 ++++++++++++++++++ .../action/search/SearchTransportService.java | 103 ++++---- .../java/org/elasticsearch/node/Node.java | 3 + .../transport/ConnectionProfile.java | 2 +- .../transport/TransportService.java | 22 +- .../search/RemoteClusterConnectionTests.java | 125 +++++++++ 6 files changed, 440 insertions(+), 54 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java create mode 100644 core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java new file mode 100644 index 00000000000..533f1c85f5e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -0,0 +1,239 @@ +/* + * 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.action.search; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.logging.log4j.util.Supplier; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +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.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.ConnectionProfile; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportConnectionListener; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; + +class RemoteClusterConnection extends AbstractComponent implements TransportConnectionListener { + + private final TransportService transportService; + private final ConnectionProfile remoteProfile; + private final CopyOnWriteArrayList clusterNodes = new CopyOnWriteArrayList(); + private final Supplier nodeSupplier; + private final String clusterName; + private final CountDownLatch connected; + private volatile List seedNodes; + + RemoteClusterConnection(Settings settings, String clusterName, List seedNodes, + TransportService transportService) { + super(settings); + this.connected = new CountDownLatch(1); + this.transportService = transportService; + this.clusterName = clusterName; + ConnectionProfile.Builder builder = new ConnectionProfile.Builder(); + builder.addConnections(6, TransportRequestOptions.Type.REG, TransportRequestOptions.Type.PING); // TODO make this configurable? + builder.addConnections(0, // we don't want this to be used for anything else but search + TransportRequestOptions.Type.BULK, + TransportRequestOptions.Type.STATE, + TransportRequestOptions.Type.RECOVERY); + remoteProfile = builder.build(); + nodeSupplier = new Supplier() { + private volatile Iterator current; + @Override + public DiscoveryNode get() { + if (current == null || current.hasNext() == false) { + current = clusterNodes.iterator(); + if (current.hasNext() == false) { + throw new IllegalStateException("No node available for cluster: " + clusterName + " nodes: " + clusterNodes ); + } + } + return current.next(); + } + }; + this.seedNodes = seedNodes; + } + + public synchronized void connectWithSeeds(ActionListener connectListener) { + if (clusterNodes.isEmpty()) { + TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable + Iterator iterator = Collections.synchronizedList(seedNodes).iterator(); + handshakeAndConnect(iterator, transportService, connectTimeout, connectListener, true); + } else { + connectListener.onResponse(null); + } + + } + + public synchronized void updateSeedNodes(List seedNodes) { + if (this.seedNodes.containsAll(seedNodes) == false || this.seedNodes.size() != seedNodes.size()) { + this.seedNodes = new ArrayList<>(seedNodes); + ActionListener listener = ActionListener.wrap(x -> {}, + e -> logger.error("failed to establish connection to remote cluster", e)); + connectWithSeeds(listener); + } + } + + private void handshakeAndConnect(Iterator seedNodes, + final TransportService transportService, TimeValue connectTimeout, ActionListener listener, + boolean connect) { + try { + if (seedNodes.hasNext()) { + final DiscoveryNode seedNode = seedNodes.next(); + final DiscoveryNode handshakeNode; + if (connect) { + try (Transport.Connection connection = transportService.openConnection(seedNode, ConnectionProfile.LIGHT_PROFILE)) { + handshakeNode = transportService.handshake(connection, connectTimeout.millis(), (c) -> true); + transportService.connectToNode(handshakeNode, remoteProfile); + clusterNodes.add(handshakeNode); + } + } else { + handshakeNode = seedNode; + } + ClusterStateRequest request = new ClusterStateRequest(); + request.clear(); + request.nodes(true); + transportService.sendRequest(transportService.getConnection(handshakeNode), + ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, + new TransportResponseHandler() { + + @Override + public ClusterStateResponse newInstance() { + return new ClusterStateResponse(); + } + + @Override + public void handleResponse(ClusterStateResponse response) { + DiscoveryNodes nodes = response.getState().nodes(); + Iterable nodesIter = nodes.getDataNodes()::valuesIt; + for (DiscoveryNode node : nodesIter) { + transportService.connectToNode(node); // noop if node is connected + clusterNodes.add(node); + } + listener.onResponse(null); + } + + @Override + public void handleException(TransportException exp) { + logger.warn((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), exp); + handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); + } + + @Override + public String executor() { + return ThreadPool.Names.MANAGEMENT; + } + }); + } else { + listener.onFailure(new IllegalStateException("no seed node left")); + } + } catch (IOException ex) { + if (seedNodes.hasNext()) { + logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), ex); + handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); + } else { + listener.onFailure(ex); + } + } + } + + @Override + public void onNodeDisconnected(DiscoveryNode node) { + boolean remove = clusterNodes.remove(node); + if (remove == true && clusterNodes.isEmpty()) { + // try to reconnect + ActionListener listener = ActionListener.wrap(x -> {}, + e -> logger.error("failed to establish connection to remote cluster", e)); + connectWithSeeds(listener); + } + } + + private void ensureConnected(DiscoveryNode[] nodes) { + boolean seenNotConnectedNode = false; + for (DiscoveryNode node : nodes) { + if (transportService.nodeConnected(node) == false) { + seenNotConnectedNode = true; + transportService.connectToNode(node, remoteProfile); + } + } + if (seenNotConnectedNode) { + final TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable + handshakeAndConnect(clusterNodes.iterator(), transportService, connectTimeout, + ActionListener.wrap((x) -> { + }, x -> { + }), false); // nocommit handle exceptions here what should we do + } + } + + public void fetchSearchShards(SearchRequest searchRequest, final List indices, + ActionListener listener) { + final DiscoveryNode node = nodeSupplier.get(); + ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices.toArray(new String[indices.size()])) + .indicesOptions(searchRequest.indicesOptions()).local(true).preference(searchRequest.preference()) + .routing(searchRequest.routing()); + transportService.sendRequest(node, ClusterSearchShardsAction.NAME, searchShardsRequest, + new TransportResponseHandler() { + + @Override + public ClusterSearchShardsResponse newInstance() { + return new ClusterSearchShardsResponse(); + } + + @Override + public void handleResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { + ensureConnected(clusterSearchShardsResponse.getNodes()); + listener.onResponse(clusterSearchShardsResponse); + } + + @Override + public void handleException(TransportException e) { + listener.onFailure(e); + } + + @Override + public String executor() { + return ThreadPool.Names.SEARCH; + } + }); + } + + public String getClusterName() { + return clusterName; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index bfe50425e40..740f4e6d23c 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -24,12 +24,9 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.Randomness; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -53,13 +50,11 @@ import org.elasticsearch.search.query.QuerySearchResultProvider; import org.elasticsearch.search.query.ScrollQuerySearchResult; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TaskAwareTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportResponse; -import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -67,10 +62,12 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; /** @@ -99,16 +96,35 @@ public class SearchTransportService extends AbstractComponent { Setting.Property.Dynamic); private final TransportService transportService; - private volatile Map> remoteClustersSeeds; + private volatile Map remoteClusters = Collections.emptyMap(); public SearchTransportService(Settings settings, ClusterSettings clusterSettings, TransportService transportService) { super(settings); this.transportService = transportService; - setRemoteClustersSeeds(REMOTE_CLUSTERS_SEEDS.get(settings)); - clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTERS_SEEDS, this::setRemoteClustersSeeds, + clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTERS_SEEDS, this::setRemoteClusters, SearchTransportService::validateRemoteClustersSeeds); } + public void setupRemoteClusters() { + // nocommit we have to figure out a good way to set-up these connections + setRemoteClusters(REMOTE_CLUSTERS_SEEDS.get(settings)); + } + + private void connect() { + int size = remoteClusters.size(); + CountDownLatch latch = new CountDownLatch(size); + for (RemoteClusterConnection connection : remoteClusters.values()) { + connection.connectWithSeeds(ActionListener.wrap(x -> latch.countDown(), ex -> { + throw new Error("failed to connect to to remote cluster " + connection.getClusterName(), ex); + })); + } + try { + latch.await(); // NOCOMMIT timeout? + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + private static void validateRemoteClustersSeeds(Settings settings) { //TODO do we need a static whitelist like in reindex from remote? for (String clusterName : settings.names()) { @@ -171,34 +187,31 @@ public class SearchTransportService extends AbstractComponent { return remoteClustersNodes; } - private void setRemoteClustersSeeds(Settings settings) { - remoteClustersSeeds = buildRemoteClustersSeeds(settings); + private void setRemoteClusters(Settings settings) { + Map> seeds = buildRemoteClustersSeeds(settings); + Map remoteClusters = new HashMap<>(); + for (Map.Entry> entry : seeds.entrySet()) { + RemoteClusterConnection remote = this.remoteClusters.get(entry.getKey()); + if (remote == null) { + remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService); + remoteClusters.put(entry.getKey(), remote); + } else { + remote.updateSeedNodes(entry.getValue()); + } + } + if (remoteClusters.isEmpty() == false) { + remoteClusters.putAll(this.remoteClusters); + this.remoteClusters = Collections.unmodifiableMap(remoteClusters); + connect(); //nocommit this sucks as it's executed on the state update thread + } } boolean isCrossClusterSearchEnabled() { - return remoteClustersSeeds.isEmpty() == false; + return remoteClusters.isEmpty() == false; } boolean isRemoteClusterRegistered(String clusterName) { - return remoteClustersSeeds.containsKey(clusterName); - } - - private DiscoveryNode connectToRemoteCluster(String clusterName) { - List nodes = remoteClustersSeeds.get(clusterName); - if (nodes == null) { - throw new IllegalArgumentException("no remote cluster configured with name [" + clusterName + "]"); - } - DiscoveryNode remoteNode = nodes.get(Randomness.get().nextInt(nodes.size())); - //TODO we just take a random host for now, implement fallback in case of connect failure - try { - //TODO at the moment the configured cluster names are really just labels. We should validate that all the nodes - //belong to the same cluster, also validate the cluster name against the configured label and make sure they match - // now go and do a real connection with the updated version of the node - connectToRemoteNode(remoteNode); - return remoteNode; - } catch(ConnectTransportException e) { - throw new ConnectTransportException(remoteNode, "unable to connect to remote cluster [" + clusterName + "]", e); - } + return remoteClusters.containsKey(clusterName); } void connectToRemoteNode(DiscoveryNode remoteNode) { @@ -212,24 +225,15 @@ public class SearchTransportService extends AbstractComponent { final AtomicReference transportException = new AtomicReference<>(); for (Map.Entry> entry : remoteIndicesByCluster.entrySet()) { final String clusterName = entry.getKey(); - //TODO we should rather eagerly connect to every configured remote node of all remote clusters - final DiscoveryNode node = connectToRemoteCluster(clusterName); + RemoteClusterConnection remoteClusterConnection = remoteClusters.get(clusterName); + if (remoteClusterConnection == null) { + throw new IllegalArgumentException("no such remote cluster: " + clusterName); + } final List indices = entry.getValue(); - //local true so we don't go to the master for each single remote search - ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices.toArray(new String[indices.size()])) - .indicesOptions(searchRequest.indicesOptions()).local(true).preference(searchRequest.preference()) - .routing(searchRequest.routing()); - - transportService.sendRequest(node, ClusterSearchShardsAction.NAME, searchShardsRequest, - new TransportResponseHandler() { - + remoteClusterConnection.fetchSearchShards(searchRequest, indices, + new ActionListener() { @Override - public ClusterSearchShardsResponse newInstance() { - return new ClusterSearchShardsResponse(); - } - - @Override - public void handleResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { + public void onResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { searchShardsResponses.put(clusterName, clusterSearchShardsResponse); if (responsesCountDown.countDown()) { TransportException exception = transportException.get(); @@ -242,7 +246,7 @@ public class SearchTransportService extends AbstractComponent { } @Override - public void handleException(TransportException e) { + public void onFailure(Exception e) { TransportException exception = new TransportException("unable to communicate with remote cluster [" + clusterName + "]", e); if (transportException.compareAndSet(null, exception) == false) { @@ -255,11 +259,6 @@ public class SearchTransportService extends AbstractComponent { listener.onFailure(exception); } } - - @Override - public String executor() { - return ThreadPool.Names.SEARCH; - } }); } } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 687ea85247a..12e9cdb097c 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -585,6 +585,8 @@ public class Node implements Closeable { // start after cluster service so the local disco is known discovery.start(); transportService.acceptIncomingRequests(); + SearchTransportService searchTransportService = injector.getInstance(SearchTransportService.class); + searchTransportService.setupRemoteClusters(); discovery.startInitialJoin(); // tribe nodes don't have a master so we shouldn't register an observer s final TimeValue initialStateTimeout = DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings); @@ -619,6 +621,7 @@ public class Node implements Closeable { } } + if (NetworkModule.HTTP_ENABLED.get(settings)) { injector.getInstance(HttpServer.class).start(); } diff --git a/core/src/main/java/org/elasticsearch/transport/ConnectionProfile.java b/core/src/main/java/org/elasticsearch/transport/ConnectionProfile.java index 92421adea6a..c5239b06b3d 100644 --- a/core/src/main/java/org/elasticsearch/transport/ConnectionProfile.java +++ b/core/src/main/java/org/elasticsearch/transport/ConnectionProfile.java @@ -188,7 +188,7 @@ public final class ConnectionProfile { */ T getChannel(T[] channels) { if (length == 0) { - throw new IllegalStateException("can't select channel size is 0"); + throw new IllegalStateException("can't select channel size is 0 for types: " + types); } assert channels.length >= offset + length : "illegal size: " + channels.length + " expected >= " + (offset + length); return channels[offset + Math.floorMod(counter.incrementAndGet(), length)]; diff --git a/core/src/main/java/org/elasticsearch/transport/TransportService.java b/core/src/main/java/org/elasticsearch/transport/TransportService.java index 8884177ba63..b526390b7e5 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportService.java @@ -64,6 +64,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +import java.util.function.Predicate; import java.util.function.Supplier; import static java.util.Collections.emptyList; @@ -369,6 +370,25 @@ public class TransportService extends AbstractLifecycleComponent { public DiscoveryNode handshake( final Transport.Connection connection, final long handshakeTimeout) throws ConnectTransportException { + return handshake(connection, handshakeTimeout, clusterName::equals); + } + + /** + * Executes a high-level handshake using the given connection + * and returns the discovery node of the node the connection + * was established with. The handshake will fail if the cluster + * name on the target node mismatches the local cluster name. + * + * @param connection the connection to a specific node + * @param handshakeTimeout handshake timeout + * @param clusterNamePredicate cluster name validation predicate + * @return the connected node + * @throws ConnectTransportException if the connection failed + * @throws IllegalStateException if the handshake failed + */ + public DiscoveryNode handshake( + final Transport.Connection connection, + final long handshakeTimeout, Predicate clusterNamePredicate) throws ConnectTransportException { final HandshakeResponse response; final DiscoveryNode node = connection.getNode(); try { @@ -386,7 +406,7 @@ public class TransportService extends AbstractLifecycleComponent { throw new IllegalStateException("handshake failed with " + node, e); } - if (!Objects.equals(clusterName, response.clusterName)) { + if (!clusterNamePredicate.test(response.clusterName)) { throw new IllegalStateException("handshake failed, mismatched cluster name [" + response.clusterName + "] - " + node); } else if (response.version.isCompatible((localNode != null ? localNode.getVersion() : Version.CURRENT)) == false) { throw new IllegalStateException("handshake failed, incompatible version [" + response.version + "] - " + node); diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java new file mode 100644 index 00000000000..4d0e0af4488 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -0,0 +1,125 @@ +/* + * 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.action.search; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.Arrays; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class RemoteClusterConnectionTests extends ESIntegTestCase { + + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Override + public void tearDown() throws Exception { + super.tearDown(); + ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); + } + + public void testConnect() throws InterruptedException { + ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); + ImmutableOpenMap dataNodes = clusterStateResponse.getState().getNodes().getDataNodes(); + DiscoveryNode node = dataNodes.valuesIt().next(); + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { + service.start(); + service.acceptIncomingRequests(); + RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.connectWithSeeds(listener); + latch.await(); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = dataNodes::valuesIt; + for (DiscoveryNode dataNode : nodesIterable) { + assertTrue(service.nodeConnected(dataNode)); + + } + assertNull(exceptionAtomicReference.get()); + } + } + + public void testFetchShards() throws InterruptedException { + + ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); + ImmutableOpenMap dataNodes = clusterStateResponse.getState().getNodes().getDataNodes(); + DiscoveryNode node = dataNodes.valuesIt().next(); + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { + service.start(); + service.acceptIncomingRequests(); + RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.connectWithSeeds(listener); + latch.await(); + + String newNode = internalCluster().startDataOnlyNode(); + createIndex("test-index"); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = dataNodes::valuesIt; + for (DiscoveryNode dataNode : nodesIterable) { + if (dataNode.getName().equals(newNode)) { + assertFalse(service.nodeConnected(dataNode)); + } else { + assertTrue(service.nodeConnected(dataNode)); + } + } + assertNull(exceptionAtomicReference.get()); + + SearchRequest request = new SearchRequest("test-index"); + CountDownLatch responseLatch = new CountDownLatch(1); + AtomicReference reference = new AtomicReference<>(); + AtomicReference failReference = new AtomicReference<>(); + + ActionListener shardsListener = ActionListener.wrap( + x -> {reference.set(x); responseLatch.countDown();}, + x -> {failReference.set(x); responseLatch.countDown();}); + connection.fetchSearchShards(request, Arrays.asList("test-index"), shardsListener); + responseLatch.await(); + assertNull(failReference.get()); + assertNotNull(reference.get()); + ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); + DiscoveryNode[] nodes = clusterSearchShardsResponse.getNodes(); + assertTrue(nodes.length != 0); + for (DiscoveryNode dataNode : nodes) { + assertTrue(service.nodeConnected(dataNode)); + } + } + } +} From dce24b5a1055074864506064a9b53c76d4b6426e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 21 Dec 2016 10:00:04 +0100 Subject: [PATCH 18/59] make connection to nodes async and ensure that if we are not fully connected a search will fork or a reconnect --- .../search/RemoteClusterConnection.java | 254 +++++++++++------- .../action/search/SearchTransportService.java | 25 +- .../action/search/TransportSearchAction.java | 16 +- .../transport/TransportService.java | 7 + .../search/RemoteClusterConnectionTests.java | 46 ++-- 5 files changed, 212 insertions(+), 136 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 533f1c85f5e..6f719239857 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.component.AbstractComponent; 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 org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.Transport; @@ -43,26 +44,29 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; -class RemoteClusterConnection extends AbstractComponent implements TransportConnectionListener { +final class RemoteClusterConnection extends AbstractComponent implements TransportConnectionListener { private final TransportService transportService; private final ConnectionProfile remoteProfile; private final CopyOnWriteArrayList clusterNodes = new CopyOnWriteArrayList(); private final Supplier nodeSupplier; private final String clusterName; - private final CountDownLatch connected; private volatile List seedNodes; + private final ConnectHandler connectHandler; RemoteClusterConnection(Settings settings, String clusterName, List seedNodes, TransportService transportService) { super(settings); - this.connected = new CountDownLatch(1); this.transportService = transportService; this.clusterName = clusterName; ConnectionProfile.Builder builder = new ConnectionProfile.Builder(); @@ -86,91 +90,12 @@ class RemoteClusterConnection extends AbstractComponent implements TransportConn } }; this.seedNodes = seedNodes; + this.connectHandler = new ConnectHandler(); } - public synchronized void connectWithSeeds(ActionListener connectListener) { - if (clusterNodes.isEmpty()) { - TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable - Iterator iterator = Collections.synchronizedList(seedNodes).iterator(); - handshakeAndConnect(iterator, transportService, connectTimeout, connectListener, true); - } else { - connectListener.onResponse(null); - } - - } - - public synchronized void updateSeedNodes(List seedNodes) { - if (this.seedNodes.containsAll(seedNodes) == false || this.seedNodes.size() != seedNodes.size()) { - this.seedNodes = new ArrayList<>(seedNodes); - ActionListener listener = ActionListener.wrap(x -> {}, - e -> logger.error("failed to establish connection to remote cluster", e)); - connectWithSeeds(listener); - } - } - - private void handshakeAndConnect(Iterator seedNodes, - final TransportService transportService, TimeValue connectTimeout, ActionListener listener, - boolean connect) { - try { - if (seedNodes.hasNext()) { - final DiscoveryNode seedNode = seedNodes.next(); - final DiscoveryNode handshakeNode; - if (connect) { - try (Transport.Connection connection = transportService.openConnection(seedNode, ConnectionProfile.LIGHT_PROFILE)) { - handshakeNode = transportService.handshake(connection, connectTimeout.millis(), (c) -> true); - transportService.connectToNode(handshakeNode, remoteProfile); - clusterNodes.add(handshakeNode); - } - } else { - handshakeNode = seedNode; - } - ClusterStateRequest request = new ClusterStateRequest(); - request.clear(); - request.nodes(true); - transportService.sendRequest(transportService.getConnection(handshakeNode), - ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, - new TransportResponseHandler() { - - @Override - public ClusterStateResponse newInstance() { - return new ClusterStateResponse(); - } - - @Override - public void handleResponse(ClusterStateResponse response) { - DiscoveryNodes nodes = response.getState().nodes(); - Iterable nodesIter = nodes.getDataNodes()::valuesIt; - for (DiscoveryNode node : nodesIter) { - transportService.connectToNode(node); // noop if node is connected - clusterNodes.add(node); - } - listener.onResponse(null); - } - - @Override - public void handleException(TransportException exp) { - logger.warn((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), exp); - handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); - } - - @Override - public String executor() { - return ThreadPool.Names.MANAGEMENT; - } - }); - } else { - listener.onFailure(new IllegalStateException("no seed node left")); - } - } catch (IOException ex) { - if (seedNodes.hasNext()) { - logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), ex); - handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); - } else { - listener.onFailure(ex); - } - } + synchronized void updateSeedNodes(List seedNodes, ActionListener connectListener) { + this.seedNodes = new ArrayList<>(seedNodes); + connectHandler.handshake(connectListener); } @Override @@ -180,7 +105,7 @@ class RemoteClusterConnection extends AbstractComponent implements TransportConn // try to reconnect ActionListener listener = ActionListener.wrap(x -> {}, e -> logger.error("failed to establish connection to remote cluster", e)); - connectWithSeeds(listener); + connectHandler.handshake(listener); } } @@ -193,8 +118,8 @@ class RemoteClusterConnection extends AbstractComponent implements TransportConn } } if (seenNotConnectedNode) { - final TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable - handshakeAndConnect(clusterNodes.iterator(), transportService, connectTimeout, + TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable + connectHandler.handshakeAndConnect(clusterNodes.iterator(), transportService, connectTimeout, ActionListener.wrap((x) -> { }, x -> { }), false); // nocommit handle exceptions here what should we do @@ -203,6 +128,14 @@ class RemoteClusterConnection extends AbstractComponent implements TransportConn public void fetchSearchShards(SearchRequest searchRequest, final List indices, ActionListener listener) { + if (clusterNodes.isEmpty()) { + connectHandler.handshake(ActionListener.wrap((x) -> fetchSearchShards(searchRequest, indices, listener), listener::onFailure)); + } else { + fetchShardsInternal(searchRequest, indices, listener); + } + } + + private void fetchShardsInternal(SearchRequest searchRequest, List indices, final ActionListener listener) { final DiscoveryNode node = nodeSupplier.get(); ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices.toArray(new String[indices.size()])) .indicesOptions(searchRequest.indicesOptions()).local(true).preference(searchRequest.preference()) @@ -236,4 +169,145 @@ class RemoteClusterConnection extends AbstractComponent implements TransportConn public String getClusterName() { return clusterName; } + + + private class ConnectHandler { + private Semaphore running = new Semaphore(1); + private BlockingQueue> queue = new ArrayBlockingQueue<>(100); + + public void handshake(ActionListener connectListener) { + final boolean runConnect; + final Collection> toNotify; + synchronized (queue) { + if (connectListener != null && queue.offer(connectListener) == false) { + throw new IllegalStateException("connect queue is full"); + } + if (queue.isEmpty()) { + return; + } + runConnect = running.tryAcquire(); + if (runConnect) { + toNotify = new ArrayList<>(); + queue.drainTo(toNotify); + } else { + toNotify = Collections.emptyList(); + } + } + if (runConnect) { + forkConnect(toNotify); + } + + } + + private void forkConnect(final Collection> toNotify) { + ThreadPool threadPool = transportService.getThreadPool(); + ExecutorService executor = threadPool.executor(ThreadPool.Names.MANAGEMENT); + executor.submit(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + synchronized (queue) { + running.release(); + } + for (ActionListener queuedListener : toNotify) { + queuedListener.onFailure(e); + } + } + + @Override + protected void doRun() throws Exception { + ActionListener listener = ActionListener.wrap((x) -> { + + synchronized (queue) { + running.release(); + } + for (ActionListener queuedListener : toNotify) { + queuedListener.onResponse(x); + } + handshake(null); + }, + (e) -> { + synchronized (queue) { + running.release(); + } + for (ActionListener queuedListener : toNotify) { + queuedListener.onFailure(e); + } + handshake(null); + }); + TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable + Iterator iterator = Collections.synchronizedList(seedNodes).iterator(); + handshakeAndConnect(iterator, transportService, connectTimeout, listener, true); + } + }); + + } + + + + void handshakeAndConnect(Iterator seedNodes, + final TransportService transportService, TimeValue connectTimeout, ActionListener listener, + boolean connect) { + try { + if (seedNodes.hasNext()) { + final DiscoveryNode seedNode = seedNodes.next(); + final DiscoveryNode handshakeNode; + if (connect) { + try (Transport.Connection connection = transportService.openConnection(seedNode, ConnectionProfile.LIGHT_PROFILE)) { + handshakeNode = transportService.handshake(connection, connectTimeout.millis(), (c) -> true); + transportService.connectToNode(handshakeNode, remoteProfile); + clusterNodes.add(handshakeNode); + } + } else { + handshakeNode = seedNode; + } + ClusterStateRequest request = new ClusterStateRequest(); + request.clear(); + request.nodes(true); + transportService.sendRequest(transportService.getConnection(handshakeNode), + ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, + new TransportResponseHandler() { + + @Override + public ClusterStateResponse newInstance() { + return new ClusterStateResponse(); + } + + @Override + public void handleResponse(ClusterStateResponse response) { + DiscoveryNodes nodes = response.getState().nodes(); + Iterable nodesIter = nodes.getDataNodes()::valuesIt; + for (DiscoveryNode node : nodesIter) { + transportService.connectToNode(node); // noop if node is connected + clusterNodes.add(node); + } + listener.onResponse(null); + } + + @Override + public void handleException(TransportException exp) { + logger.warn((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), exp); + handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); + } + + @Override + public String executor() { + return ThreadPool.Names.MANAGEMENT; + } + }); + } else { + listener.onFailure(new IllegalStateException("no seed node left")); + } + } catch (IOException ex) { + if (seedNodes.hasNext()) { + logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), ex); + handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); + } else { + listener.onFailure(ex); + } + } + } + } + } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 740f4e6d23c..cf44fb37b95 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -106,25 +106,9 @@ public class SearchTransportService extends AbstractComponent { } public void setupRemoteClusters() { - // nocommit we have to figure out a good way to set-up these connections setRemoteClusters(REMOTE_CLUSTERS_SEEDS.get(settings)); } - private void connect() { - int size = remoteClusters.size(); - CountDownLatch latch = new CountDownLatch(size); - for (RemoteClusterConnection connection : remoteClusters.values()) { - connection.connectWithSeeds(ActionListener.wrap(x -> latch.countDown(), ex -> { - throw new Error("failed to connect to to remote cluster " + connection.getClusterName(), ex); - })); - } - try { - latch.await(); // NOCOMMIT timeout? - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } - } - private static void validateRemoteClustersSeeds(Settings settings) { //TODO do we need a static whitelist like in reindex from remote? for (String clusterName : settings.names()) { @@ -195,14 +179,13 @@ public class SearchTransportService extends AbstractComponent { if (remote == null) { remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService); remoteClusters.put(entry.getKey(), remote); - } else { - remote.updateSeedNodes(entry.getValue()); } + remote.updateSeedNodes(entry.getValue(), ActionListener.wrap((x) -> {}, + e -> logger.error("failed to update seed list for cluster: " + entry.getKey(), e) )); } if (remoteClusters.isEmpty() == false) { remoteClusters.putAll(this.remoteClusters); this.remoteClusters = Collections.unmodifiableMap(remoteClusters); - connect(); //nocommit this sucks as it's executed on the state update thread } } @@ -214,10 +197,6 @@ public class SearchTransportService extends AbstractComponent { return remoteClusters.containsKey(clusterName); } - void connectToRemoteNode(DiscoveryNode remoteNode) { - transportService.connectToNode(remoteNode); - } - void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, ActionListener> listener) { final CountDown responsesCountDown = new CountDown(remoteIndicesByCluster.size()); diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index f655fadf5b7..9b2d967f65c 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -279,13 +279,21 @@ public class TransportSearchAction extends HandledTransportAction builder = ImmutableOpenMap.builder(nodes.getNodes()); + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); for (DiscoveryNode remoteNode : remoteNodes) { - //TODO shall we catch connect exceptions here? Otherwise we will return an error but we could rather return partial results? - searchTransportService.connectToRemoteNode(remoteNode); builder.put(remoteNode.getId(), remoteNode); } - return builder.build()::get; + ImmutableOpenMap remoteNodesMap = builder.build(); + return (nodeId) -> { + DiscoveryNode discoveryNode = nodes.get(nodeId); + if (discoveryNode == null) { + discoveryNode = remoteNodesMap.get(nodeId); + } + if (discoveryNode == null) { + throw new IllegalArgumentException("no node found for id: " + nodeId); + } + return discoveryNode; + }; } @Override diff --git a/core/src/main/java/org/elasticsearch/transport/TransportService.java b/core/src/main/java/org/elasticsearch/transport/TransportService.java index b526390b7e5..9ed0603bdf4 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportService.java @@ -1182,4 +1182,11 @@ public class TransportService extends AbstractLifecycleComponent { return "direct"; } } + + /** + * Returns the internal thread pool + */ + public ThreadPool getThreadPool() { + return threadPool; + } } diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index 4d0e0af4488..07f88134b22 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; +import java.util.Collections; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -59,7 +60,7 @@ public class RemoteClusterConnectionTests extends ESIntegTestCase { exceptionAtomicReference.set(x); latch.countDown(); }); - connection.connectWithSeeds(listener); + connection.updateSeedNodes(Arrays.asList(node),listener); latch.await(); assertTrue(service.nodeConnected(node)); Iterable nodesIterable = dataNodes::valuesIt; @@ -79,28 +80,35 @@ public class RemoteClusterConnectionTests extends ESIntegTestCase { try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { service.start(); service.acceptIncomingRequests(); - RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service); + final boolean hasInitialNodes = randomBoolean(); + RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", + hasInitialNodes ? Arrays.asList(node) : Collections.emptyList(), service); CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.connectWithSeeds(listener); - latch.await(); - String newNode = internalCluster().startDataOnlyNode(); - createIndex("test-index"); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = dataNodes::valuesIt; - for (DiscoveryNode dataNode : nodesIterable) { - if (dataNode.getName().equals(newNode)) { - assertFalse(service.nodeConnected(dataNode)); - } else { - assertTrue(service.nodeConnected(dataNode)); + if (hasInitialNodes == false) { + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.updateSeedNodes(Arrays.asList(node), listener); + latch.await(); + + String newNode = internalCluster().startDataOnlyNode(); + createIndex("test-index"); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = dataNodes::valuesIt; + for (DiscoveryNode dataNode : nodesIterable) { + if (dataNode.getName().equals(newNode)) { + assertFalse(service.nodeConnected(dataNode)); + } else { + assertTrue(service.nodeConnected(dataNode)); + } } + assertNull(exceptionAtomicReference.get()); + } else { + createIndex("test-index"); } - assertNull(exceptionAtomicReference.get()); SearchRequest request = new SearchRequest("test-index"); CountDownLatch responseLatch = new CountDownLatch(1); From 20e9e8b5605c56772ab37ad2859e80cfe82134f7 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 21 Dec 2016 20:52:42 +0100 Subject: [PATCH 19/59] Add TransportActionProxy infrastructure --- .../transport/TransportActionProxy.java | 123 +++++++++ .../transport/TransportActionProxyTests.java | 252 ++++++++++++++++++ 2 files changed, 375 insertions(+) create mode 100644 core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java create mode 100644 core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java diff --git a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java new file mode 100644 index 00000000000..38ed2bbad73 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -0,0 +1,123 @@ +package org.elasticsearch.transport; + +import org.apache.logging.log4j.util.Supplier; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.io.UncheckedIOException; + +// nocommit javadocs +public final class TransportActionProxy { + + private TransportActionProxy() {} // no instance + + private static class ProxyRequestHandler implements TransportRequestHandler { + + private final TransportService service; + private final String action; + private final Supplier responseFactory; + + public ProxyRequestHandler(TransportService service, String action, Supplier responseFactory) { + this.service = service; + this.action = action; + this.responseFactory = responseFactory; + } + + @Override + public void messageReceived(T request, TransportChannel channel) throws Exception { + DiscoveryNode targetNode = request.targetNode; + TransportRequest wrappedRequest = request.wrapped; + service.sendRequest(targetNode, action, wrappedRequest, new ProxyResponseHandler<>(channel, responseFactory)); + } + } + + private static class ProxyResponseHandler implements TransportResponseHandler { + + private final Supplier responseFactory; + private final TransportChannel channel; + + public ProxyResponseHandler(TransportChannel channel, Supplier responseFactory) { + this.responseFactory = responseFactory; + this.channel = channel; + + } + @Override + public T newInstance() { + return responseFactory.get(); + } + + @Override + public void handleResponse(T response) { + try { + channel.sendResponse(response); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public void handleException(TransportException exp) { + try { + channel.sendResponse(exp); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } + + static class ProxyRequest extends TransportRequest { + T wrapped; + Supplier supplier; + DiscoveryNode targetNode; + + public ProxyRequest(Supplier supplier) { + this.supplier = supplier; + } + + public ProxyRequest(T wrapped, DiscoveryNode targetNode) { + this.wrapped = wrapped; + this.targetNode = targetNode; + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + targetNode = new DiscoveryNode(in); + wrapped = supplier.get(); + wrapped.readFrom(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + targetNode.writeTo(out); + wrapped.writeTo(out); + } + } + + /** + * Registers a proxy request handler that allows to forward requests for the given action to another node. + */ + public static String registerProxyAction(TransportService service, String action, Supplier responseSupplier) { + RequestHandlerRegistry requestHandler = service.getRequestHandler(action); + String proxyAction = "internal:transport/proxy/" + action; + service.registerRequestHandler(proxyAction, () -> new ProxyRequest(requestHandler::newRequest), ThreadPool.Names.SAME, true, false + , new ProxyRequestHandler<>(service, action, responseSupplier)); + return proxyAction; + } + + //nocommit javadocs + public static void sendProxyRequest(TransportService service, DiscoveryNode proxyNode, DiscoveryNode targetNode, String action, + TransportRequest request, TransportResponseHandler handler) { + String proxyAction = "internal:transport/proxy/" + action; + service.sendRequest(proxyNode, proxyAction, new ProxyRequest(request, targetNode), handler); + } +} diff --git a/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java b/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java new file mode 100644 index 00000000000..be4e7fd3ae4 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java @@ -0,0 +1,252 @@ +/* + * 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.lucene.util.IOUtils; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.Before; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; + +public class TransportActionProxyTests extends ESTestCase { + protected ThreadPool threadPool; + // we use always a non-alpha or beta version here otherwise minimumCompatibilityVersion will be different for the two used versions + private static final Version CURRENT_VERSION = Version.fromString(String.valueOf(Version.CURRENT.major) + ".0.0"); + protected static final Version version0 = CURRENT_VERSION.minimumCompatibilityVersion(); + + protected DiscoveryNode nodeA; + protected MockTransportService serviceA; + + protected static final Version version1 = Version.fromId(CURRENT_VERSION.id + 1); + protected DiscoveryNode nodeB; + protected MockTransportService serviceB; + + protected DiscoveryNode nodeC; + protected MockTransportService serviceC; + + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(getClass().getName()); + serviceA = buildService(version0); // this one supports dynamic tracer updates + nodeA = new DiscoveryNode("TS_A", serviceA.boundAddress().publishAddress(), emptyMap(), emptySet(), version0); + serviceB = buildService(version1); // this one doesn't support dynamic tracer updates + nodeB = new DiscoveryNode("TS_B", serviceB.boundAddress().publishAddress(), emptyMap(), emptySet(), version1); + serviceC = buildService(version1); // this one doesn't support dynamic tracer updates + nodeC = new DiscoveryNode("TS_C", serviceC.boundAddress().publishAddress(), emptyMap(), emptySet(), version1); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + IOUtils.close(serviceA, serviceB, serviceC, () -> { + try { + terminate(threadPool); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + } + + private MockTransportService buildService(final Version version) { + MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, version, threadPool, null); + service.start(); + service.acceptIncomingRequests(); + return service; + + } + + + public void testSendMessage() throws InterruptedException { + serviceA.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + assertEquals(request.sourceNode, "TS_A"); + SimpleTestResponse response = new SimpleTestResponse(); + response.targetNode = "TS_A"; + channel.sendResponse(response); + }); + TransportActionProxy.registerProxyAction(serviceA, "/test", SimpleTestResponse::new); + serviceA.connectToNode(nodeB); + + serviceB.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + assertEquals(request.sourceNode, "TS_A"); + SimpleTestResponse response = new SimpleTestResponse(); + response.targetNode = "TS_B"; + channel.sendResponse(response); + }); + TransportActionProxy.registerProxyAction(serviceB, "/test", SimpleTestResponse::new); + serviceB.connectToNode(nodeC); + serviceC.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + assertEquals(request.sourceNode, "TS_A"); + SimpleTestResponse response = new SimpleTestResponse(); + response.targetNode = "TS_C"; + channel.sendResponse(response); + }); + TransportActionProxy.registerProxyAction(serviceC, "/test", SimpleTestResponse::new); + + CountDownLatch latch = new CountDownLatch(1); + TransportActionProxy.sendProxyRequest(serviceA, nodeB, nodeC, "/test", new SimpleTestRequest("TS_A"), + new TransportResponseHandler() { + @Override + public SimpleTestResponse newInstance() { + return new SimpleTestResponse(); + } + + @Override + public void handleResponse(SimpleTestResponse response) { + try { + assertEquals("TS_C", response.targetNode); + } finally { + latch.countDown(); + } + } + + @Override + public void handleException(TransportException exp) { + try { + throw new AssertionError(exp); + } finally { + latch.countDown(); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + latch.await(); + } + + public void testException() throws InterruptedException { + serviceA.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + assertEquals(request.sourceNode, "TS_A"); + SimpleTestResponse response = new SimpleTestResponse(); + response.targetNode = "TS_A"; + channel.sendResponse(response); + }); + TransportActionProxy.registerProxyAction(serviceA, "/test", SimpleTestResponse::new); + serviceA.connectToNode(nodeB); + + serviceB.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + assertEquals(request.sourceNode, "TS_A"); + SimpleTestResponse response = new SimpleTestResponse(); + response.targetNode = "TS_B"; + channel.sendResponse(response); + }); + TransportActionProxy.registerProxyAction(serviceB, "/test", SimpleTestResponse::new); + serviceB.connectToNode(nodeC); + serviceC.registerRequestHandler("/test", SimpleTestRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + throw new ElasticsearchException("greetings from TS_C"); + }); + TransportActionProxy.registerProxyAction(serviceC, "/test", SimpleTestResponse::new); + + CountDownLatch latch = new CountDownLatch(1); + TransportActionProxy.sendProxyRequest(serviceA, nodeB, nodeC, "/test", new SimpleTestRequest("TS_A"), + new TransportResponseHandler() { + @Override + public SimpleTestResponse newInstance() { + return new SimpleTestResponse(); + } + + @Override + public void handleResponse(SimpleTestResponse response) { + try { + fail("expected exception"); + } finally { + latch.countDown(); + } + } + + @Override + public void handleException(TransportException exp) { + try { + Throwable cause = ExceptionsHelper.unwrapCause(exp); + assertEquals("greetings from TS_C", cause.getMessage()); + } finally { + latch.countDown(); + } + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + }); + latch.await(); + } + + public static class SimpleTestRequest extends TransportRequest { + String sourceNode; + + public SimpleTestRequest(String sourceNode) { + this.sourceNode = sourceNode; + } + public SimpleTestRequest() {} + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + sourceNode = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(sourceNode); + } + } + + public static class SimpleTestResponse extends TransportResponse { + String targetNode; + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + targetNode = in.readString(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(targetNode); + } + } + +} From d89b3ba2cf7cfb02c0444324e78daa9641ccf521 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 21 Dec 2016 21:09:21 +0100 Subject: [PATCH 20/59] add current WIP --- .../action/search/SearchTransportService.java | 24 +++++++++++++++++-- .../transport/TransportActionProxy.java | 22 +++++++++-------- .../transport/TransportActionProxyTests.java | 8 +++---- 3 files changed, 38 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index cf44fb37b95..07307d790f9 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -50,6 +50,7 @@ import org.elasticsearch.search.query.QuerySearchResultProvider; import org.elasticsearch.search.query.ScrollQuerySearchResult; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TaskAwareTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; @@ -67,7 +68,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; /** @@ -429,6 +429,7 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(new SearchFreeContextResponse(freed)); } }); + TransportActionProxy.registerProxyAction(transportService, FREE_CONTEXT_SCROLL_ACTION_NAME, SearchFreeContextResponse::new); transportService.registerRequestHandler(FREE_CONTEXT_ACTION_NAME, SearchFreeContextRequest::new, ThreadPool.Names.SAME, new TaskAwareTransportRequestHandler() { @Override @@ -437,6 +438,7 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(new SearchFreeContextResponse(freed)); } }); + TransportActionProxy.registerProxyAction(transportService, FREE_CONTEXT_ACTION_NAME, SearchFreeContextResponse::new); transportService.registerRequestHandler(CLEAR_SCROLL_CONTEXTS_ACTION_NAME, () -> TransportRequest.Empty.INSTANCE, ThreadPool.Names.SAME, new TaskAwareTransportRequestHandler() { @@ -446,6 +448,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(TransportResponse.Empty.INSTANCE); } }); + TransportActionProxy.registerProxyAction(transportService, CLEAR_SCROLL_CONTEXTS_ACTION_NAME, () -> TransportResponse.Empty.INSTANCE); + transportService.registerRequestHandler(DFS_ACTION_NAME, ShardSearchTransportRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -455,6 +459,8 @@ public class SearchTransportService extends AbstractComponent { } }); + TransportActionProxy.registerProxyAction(transportService, DFS_ACTION_NAME, DfsSearchResult::new); + transportService.registerRequestHandler(QUERY_ACTION_NAME, ShardSearchTransportRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -463,6 +469,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, QUERY_ACTION_NAME, QuerySearchResult::new); + transportService.registerRequestHandler(QUERY_ID_ACTION_NAME, QuerySearchRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -471,6 +479,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, QUERY_ID_ACTION_NAME, QuerySearchResult::new); + transportService.registerRequestHandler(QUERY_SCROLL_ACTION_NAME, InternalScrollSearchRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -479,6 +489,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, QUERY_SCROLL_ACTION_NAME, ScrollQuerySearchResult::new); + transportService.registerRequestHandler(QUERY_FETCH_ACTION_NAME, ShardSearchTransportRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -487,6 +499,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, QUERY_FETCH_ACTION_NAME, QueryFetchSearchResult::new); + transportService.registerRequestHandler(QUERY_QUERY_FETCH_ACTION_NAME, QuerySearchRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -495,6 +509,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, QUERY_QUERY_FETCH_ACTION_NAME, QueryFetchSearchResult::new); + transportService.registerRequestHandler(QUERY_FETCH_SCROLL_ACTION_NAME, InternalScrollSearchRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -503,6 +519,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, QUERY_FETCH_SCROLL_ACTION_NAME, ScrollQueryFetchSearchResult::new); + transportService.registerRequestHandler(FETCH_ID_SCROLL_ACTION_NAME, ShardFetchRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -511,6 +529,8 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); + TransportActionProxy.registerProxyAction(transportService, FETCH_ID_SCROLL_ACTION_NAME, FetchSearchResult::new); + transportService.registerRequestHandler(FETCH_ID_ACTION_NAME, ShardFetchSearchRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { @Override @@ -519,6 +539,6 @@ public class SearchTransportService extends AbstractComponent { channel.sendResponse(result); } }); - + TransportActionProxy.registerProxyAction(transportService, FETCH_ID_ACTION_NAME, FetchSearchResult::new); } } diff --git a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index 38ed2bbad73..0588c8bd7d0 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -106,18 +106,20 @@ public final class TransportActionProxy { /** * Registers a proxy request handler that allows to forward requests for the given action to another node. */ - public static String registerProxyAction(TransportService service, String action, Supplier responseSupplier) { + public static void registerProxyAction(TransportService service, String action, Supplier responseSupplier) { RequestHandlerRegistry requestHandler = service.getRequestHandler(action); - String proxyAction = "internal:transport/proxy/" + action; - service.registerRequestHandler(proxyAction, () -> new ProxyRequest(requestHandler::newRequest), ThreadPool.Names.SAME, true, false - , new ProxyRequestHandler<>(service, action, responseSupplier)); - return proxyAction; + service.registerRequestHandler(getProxyAction(action), () -> new ProxyRequest(requestHandler::newRequest), ThreadPool.Names.SAME, + true, false, new ProxyRequestHandler<>(service, action, responseSupplier)); } - //nocommit javadocs - public static void sendProxyRequest(TransportService service, DiscoveryNode proxyNode, DiscoveryNode targetNode, String action, - TransportRequest request, TransportResponseHandler handler) { - String proxyAction = "internal:transport/proxy/" + action; - service.sendRequest(proxyNode, proxyAction, new ProxyRequest(request, targetNode), handler); + /** + * Returns the corresponding proxy action for the given action + */ + public static String getProxyAction(String action) { + return "internal:transport/proxy/" + action; + } + + public static TransportRequest wrapRequest(DiscoveryNode node, TransportRequest request) { + return new ProxyRequest<>(request, node); } } diff --git a/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java b/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java index be4e7fd3ae4..68dffa1ded9 100644 --- a/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java +++ b/core/src/test/java/org/elasticsearch/transport/TransportActionProxyTests.java @@ -119,8 +119,8 @@ public class TransportActionProxyTests extends ESTestCase { TransportActionProxy.registerProxyAction(serviceC, "/test", SimpleTestResponse::new); CountDownLatch latch = new CountDownLatch(1); - TransportActionProxy.sendProxyRequest(serviceA, nodeB, nodeC, "/test", new SimpleTestRequest("TS_A"), - new TransportResponseHandler() { + serviceA.sendRequest(nodeB, TransportActionProxy.getProxyAction("/test"), TransportActionProxy.wrapRequest(nodeC, + new SimpleTestRequest("TS_A")), new TransportResponseHandler() { @Override public SimpleTestResponse newInstance() { return new SimpleTestResponse(); @@ -179,8 +179,8 @@ public class TransportActionProxyTests extends ESTestCase { TransportActionProxy.registerProxyAction(serviceC, "/test", SimpleTestResponse::new); CountDownLatch latch = new CountDownLatch(1); - TransportActionProxy.sendProxyRequest(serviceA, nodeB, nodeC, "/test", new SimpleTestRequest("TS_A"), - new TransportResponseHandler() { + serviceA.sendRequest(nodeB, TransportActionProxy.getProxyAction("/test"), TransportActionProxy.wrapRequest(nodeC, + new SimpleTestRequest("TS_A")), new TransportResponseHandler() { @Override public SimpleTestResponse newInstance() { return new SimpleTestResponse(); From 422cd1ef779c269bc69590dde9908c024a9b92a2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 3 Jan 2017 17:24:32 +0100 Subject: [PATCH 21/59] Add support for proxy nodes this commit adds full support for proxy nodes on the search layer. This allows to connection only to a small set of nodes on a remote cluster to exectue the search. The nodes will proxy the request to the correct node in the cluster while the coordinting node doesn't need to be connected to the target node. --- .../search/AbstractSearchAsyncAction.java | 31 ++--- .../search/RemoteClusterConnection.java | 62 +++++++--- .../SearchDfsQueryAndFetchAsyncAction.java | 19 +-- .../SearchDfsQueryThenFetchAsyncAction.java | 27 +++-- .../SearchQueryAndFetchAsyncAction.java | 9 +- .../SearchQueryThenFetchAsyncAction.java | 17 +-- .../action/search/SearchTransportService.java | 68 +++++++---- .../action/search/TransportSearchAction.java | 75 ++++++------ .../common/settings/ClusterSettings.java | 1 + .../transport/TransportService.java | 9 +- .../search/RemoteClusterConnectionTests.java | 40 ++++++- .../action/search/SearchAsyncActionTests.java | 46 ++++++-- .../search/simple/SimpleSearchIT.java | 28 ++++- qa/multi-cluster-search/build.gradle | 5 +- .../MultiClusterSearchYamlTestSuiteIT.java | 3 +- .../elasticsearch/test/ESIntegTestCase.java | 111 ++++++++++++++++++ 16 files changed, 408 insertions(+), 143 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 2479ff86750..4b57dd5f59b 100644 --- a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -41,6 +41,7 @@ import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.ShardSearchTransportRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResultProvider; +import org.elasticsearch.transport.Transport; import java.util.List; import java.util.Map; @@ -59,7 +60,7 @@ abstract class AbstractSearchAsyncAction private final GroupShardsIterator shardsIts; protected final SearchRequest request; /** Used by subclasses to resolve node ids to DiscoveryNodes. **/ - protected final Function nodeIdToDiscoveryNode; + protected final Function nodeIdToConnection; protected final SearchTask task; protected final int expectedSuccessfulOps; private final int expectedTotalOps; @@ -74,7 +75,7 @@ abstract class AbstractSearchAsyncAction protected volatile ScoreDoc[] sortedShardDocs; protected AbstractSearchAsyncAction(Logger logger, SearchTransportService searchTransportService, - Function nodeIdToDiscoveryNode, + Function nodeIdToConnection, Map aliasFilter, Map concreteIndexBoosts, Executor executor, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, long startTime, long clusterStateVersion, SearchTask task) { @@ -85,7 +86,7 @@ abstract class AbstractSearchAsyncAction this.request = request; this.task = task; this.listener = listener; - this.nodeIdToDiscoveryNode = nodeIdToDiscoveryNode; + this.nodeIdToConnection = nodeIdToConnection; this.clusterStateVersion = clusterStateVersion; this.shardsIts = shardsIts; expectedSuccessfulOps = shardsIts.size(); @@ -122,8 +123,8 @@ abstract class AbstractSearchAsyncAction // no more active shards... (we should not really get here, but just for safety) onFirstPhaseResult(shardIndex, null, null, shardIt, new NoShardAvailableActionException(shardIt.shardId())); } else { - final DiscoveryNode node = nodeIdToDiscoveryNode.apply(shard.currentNodeId()); - if (node == null) { + final Transport.Connection connection = nodeIdToConnection.apply(shard.currentNodeId()); + if (connection == null) { onFirstPhaseResult(shardIndex, shard, null, shardIt, new NoShardAvailableActionException(shardIt.shardId())); } else { AliasFilter filter = this.aliasFilter.get(shard.index().getUUID()); @@ -132,7 +133,7 @@ abstract class AbstractSearchAsyncAction float indexBoost = concreteIndexBoosts.getOrDefault(shard.index().getUUID(), DEFAULT_INDEX_BOOST); ShardSearchTransportRequest transportRequest = new ShardSearchTransportRequest(request, shardIt.shardId(), shardsIts.size(), filter, indexBoost, startTime()); - sendExecuteFirstPhase(node, transportRequest , new ActionListener() { + sendExecuteFirstPhase(connection, transportRequest , new ActionListener() { @Override public void onResponse(FirstResult result) { onFirstPhaseResult(shardIndex, shard.currentNodeId(), result, shardIt); @@ -140,7 +141,7 @@ abstract class AbstractSearchAsyncAction @Override public void onFailure(Exception t) { - onFirstPhaseResult(shardIndex, shard, node.getId(), shardIt, t); + onFirstPhaseResult(shardIndex, shard, connection.getNode().getId(), shardIt, t); } }); } @@ -292,8 +293,8 @@ abstract class AbstractSearchAsyncAction private void raiseEarlyFailure(Exception e) { for (AtomicArray.Entry entry : firstResults.asList()) { try { - DiscoveryNode node = nodeIdToDiscoveryNode.apply(entry.value.shardTarget().nodeId()); - sendReleaseSearchContext(entry.value.id(), node); + Transport.Connection connection = nodeIdToConnection.apply(entry.value.shardTarget().nodeId()); + sendReleaseSearchContext(entry.value.id(), connection); } catch (Exception inner) { inner.addSuppressed(e); logger.trace("failed to release context", inner); @@ -317,8 +318,8 @@ abstract class AbstractSearchAsyncAction if (queryResult.hasHits() && docIdsToLoad.get(entry.index) == null) { // but none of them made it to the global top docs try { - DiscoveryNode node = nodeIdToDiscoveryNode.apply(entry.value.queryResult().shardTarget().nodeId()); - sendReleaseSearchContext(entry.value.queryResult().id(), node); + Transport.Connection connection = nodeIdToConnection.apply(entry.value.queryResult().shardTarget().nodeId()); + sendReleaseSearchContext(entry.value.queryResult().id(), connection); } catch (Exception e) { logger.trace("failed to release context", e); } @@ -327,9 +328,9 @@ abstract class AbstractSearchAsyncAction } } - protected void sendReleaseSearchContext(long contextId, DiscoveryNode node) { - if (node != null) { - searchTransportService.sendFreeContext(node, contextId, request); + protected void sendReleaseSearchContext(long contextId, Transport.Connection connection) { + if (connection != null) { + searchTransportService.sendFreeContext(connection, contextId, request); } } @@ -339,7 +340,7 @@ abstract class AbstractSearchAsyncAction return new ShardFetchSearchRequest(request, queryResult.id(), entry.value, lastEmittedDoc); } - protected abstract void sendExecuteFirstPhase(DiscoveryNode node, ShardSearchTransportRequest request, + protected abstract void sendExecuteFirstPhase(Transport.Connection connection, ShardSearchTransportRequest request, ActionListener listener); protected final void processFirstPhaseResult(int shardIndex, FirstResult result) { diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 6f719239857..8e3ffc737ac 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; @@ -33,11 +34,14 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.discovery.Discovery; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; @@ -53,6 +57,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Semaphore; +import java.util.function.Predicate; final class RemoteClusterConnection extends AbstractComponent implements TransportConnectionListener { @@ -61,17 +66,21 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private final CopyOnWriteArrayList clusterNodes = new CopyOnWriteArrayList(); private final Supplier nodeSupplier; private final String clusterName; + private final int numSeedNodesToDiscover; + private final Predicate nodePredicate; private volatile List seedNodes; private final ConnectHandler connectHandler; RemoteClusterConnection(Settings settings, String clusterName, List seedNodes, - TransportService transportService) { + TransportService transportService, int numSeedNodesToDiscover, Predicate nodePredicate) { super(settings); this.transportService = transportService; + this.numSeedNodesToDiscover = numSeedNodesToDiscover; + this.nodePredicate = nodePredicate; this.clusterName = clusterName; ConnectionProfile.Builder builder = new ConnectionProfile.Builder(); builder.addConnections(6, TransportRequestOptions.Type.REG, TransportRequestOptions.Type.PING); // TODO make this configurable? - builder.addConnections(0, // we don't want this to be used for anything else but search + builder.addConnections(0, // we don't want this to be used for anything else but search TransportRequestOptions.Type.BULK, TransportRequestOptions.Type.STATE, TransportRequestOptions.Type.RECOVERY); @@ -89,13 +98,13 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo return current.next(); } }; - this.seedNodes = seedNodes; + this.seedNodes = Collections.unmodifiableList(seedNodes); this.connectHandler = new ConnectHandler(); } synchronized void updateSeedNodes(List seedNodes, ActionListener connectListener) { - this.seedNodes = new ArrayList<>(seedNodes); - connectHandler.handshake(connectListener); + this.seedNodes = Collections.unmodifiableList(new ArrayList<>(seedNodes)); + connectHandler.connect(connectListener); } @Override @@ -105,7 +114,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo // try to reconnect ActionListener listener = ActionListener.wrap(x -> {}, e -> logger.error("failed to establish connection to remote cluster", e)); - connectHandler.handshake(listener); + connectHandler.connect(listener); } } @@ -129,12 +138,34 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo public void fetchSearchShards(SearchRequest searchRequest, final List indices, ActionListener listener) { if (clusterNodes.isEmpty()) { - connectHandler.handshake(ActionListener.wrap((x) -> fetchSearchShards(searchRequest, indices, listener), listener::onFailure)); + connectHandler.connect(ActionListener.wrap((x) -> fetchSearchShards(searchRequest, indices, listener), listener::onFailure)); } else { fetchShardsInternal(searchRequest, indices, listener); } } + public Transport.Connection getProxyConnection(DiscoveryNode nodeToProxyTo) { + DiscoveryNode discoveryNode = nodeSupplier.get(); + Transport.Connection connection = transportService.getConnection(discoveryNode); + return new Transport.Connection() { + @Override + public DiscoveryNode getNode() { + return nodeToProxyTo; + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException { + connection.sendRequest(requestId, TransportActionProxy.getProxyAction(action), + TransportActionProxy.wrapRequest(nodeToProxyTo, request), options); + } + + @Override + public void close() throws IOException { + throw new IllegalStateException("never close a proxy connection"); + } + }; + } + private void fetchShardsInternal(SearchRequest searchRequest, List indices, final ActionListener listener) { final DiscoveryNode node = nodeSupplier.get(); ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices.toArray(new String[indices.size()])) @@ -175,7 +206,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private Semaphore running = new Semaphore(1); private BlockingQueue> queue = new ArrayBlockingQueue<>(100); - public void handshake(ActionListener connectListener) { + public void connect(ActionListener connectListener) { final boolean runConnect; final Collection> toNotify; synchronized (queue) { @@ -223,7 +254,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo for (ActionListener queuedListener : toNotify) { queuedListener.onResponse(x); } - handshake(null); + connect(null); }, (e) -> { synchronized (queue) { @@ -232,7 +263,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo for (ActionListener queuedListener : toNotify) { queuedListener.onFailure(e); } - handshake(null); + connect(null); }); TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable Iterator iterator = Collections.synchronizedList(seedNodes).iterator(); @@ -252,7 +283,8 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo final DiscoveryNode seedNode = seedNodes.next(); final DiscoveryNode handshakeNode; if (connect) { - try (Transport.Connection connection = transportService.openConnection(seedNode, ConnectionProfile.LIGHT_PROFILE)) { + try (Transport.Connection connection = transportService.openConnection(seedNode, + ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null,null))) { handshakeNode = transportService.handshake(connection, connectTimeout.millis(), (c) -> true); transportService.connectToNode(handshakeNode, remoteProfile); clusterNodes.add(handshakeNode); @@ -275,10 +307,12 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public void handleResponse(ClusterStateResponse response) { DiscoveryNodes nodes = response.getState().nodes(); - Iterable nodesIter = nodes.getDataNodes()::valuesIt; + Iterable nodesIter = nodes.getNodes()::valuesIt; for (DiscoveryNode node : nodesIter) { - transportService.connectToNode(node); // noop if node is connected - clusterNodes.add(node); + if (nodePredicate.test(node) && clusterNodes.size() < numSeedNodesToDiscover) { + transportService.connectToNode(node); // noop if node is connected + clusterNodes.add(node); + } } listener.onResponse(null); } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryAndFetchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryAndFetchAsyncAction.java index 9db3a21c485..0c9a6b1cdf3 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryAndFetchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchDfsQueryAndFetchAsyncAction.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.ShardSearchTransportRequest; import org.elasticsearch.search.query.QuerySearchRequest; +import org.elasticsearch.transport.Transport; import java.io.IOException; import java.util.Map; @@ -46,12 +47,12 @@ class SearchDfsQueryAndFetchAsyncAction extends AbstractSearchAsyncAction queryFetchResults; private final SearchPhaseController searchPhaseController; SearchDfsQueryAndFetchAsyncAction(Logger logger, SearchTransportService searchTransportService, - Function nodeIdToDiscoveryNode, + Function nodeIdToConnection, Map aliasFilter, Map concreteIndexBoosts, SearchPhaseController searchPhaseController, Executor executor, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, long startTime, long clusterStateVersion, SearchTask task) { - super(logger, searchTransportService, nodeIdToDiscoveryNode, aliasFilter, concreteIndexBoosts, executor, + super(logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener, shardsIts, startTime, clusterStateVersion, task); this.searchPhaseController = searchPhaseController; queryFetchResults = new AtomicArray<>(firstResults.length()); @@ -63,9 +64,9 @@ class SearchDfsQueryAndFetchAsyncAction extends AbstractSearchAsyncAction listener) { - searchTransportService.sendExecuteDfs(node, request, task, listener); + searchTransportService.sendExecuteDfs(connection, request, task, listener); } @Override @@ -75,15 +76,15 @@ class SearchDfsQueryAndFetchAsyncAction extends AbstractSearchAsyncAction entry : firstResults.asList()) { DfsSearchResult dfsResult = entry.value; - DiscoveryNode node = nodeIdToDiscoveryNode.apply(dfsResult.shardTarget().nodeId()); + Transport.Connection connection = nodeIdToConnection.apply(dfsResult.shardTarget().nodeId()); QuerySearchRequest querySearchRequest = new QuerySearchRequest(request, dfsResult.id(), dfs); - executeSecondPhase(entry.index, dfsResult, counter, node, querySearchRequest); + executeSecondPhase(entry.index, dfsResult, counter, connection, querySearchRequest); } } void executeSecondPhase(final int shardIndex, final DfsSearchResult dfsResult, final AtomicInteger counter, - final DiscoveryNode node, final QuerySearchRequest querySearchRequest) { - searchTransportService.sendExecuteFetch(node, querySearchRequest, task, new ActionListener() { + final Transport.Connection connection, final QuerySearchRequest querySearchRequest) { + searchTransportService.sendExecuteFetch(connection, querySearchRequest, task, new ActionListener() { @Override public void onResponse(QueryFetchSearchResult result) { result.shardTarget(dfsResult.shardTarget()); @@ -101,7 +102,7 @@ class SearchDfsQueryAndFetchAsyncAction extends AbstractSearchAsyncAction nodeIdToDiscoveryNode, + Function nodeIdToConnection, Map aliasFilter, Map concreteIndexBoosts, SearchPhaseController searchPhaseController, Executor executor, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, long startTime, long clusterStateVersion, SearchTask task) { - super(logger, searchTransportService, nodeIdToDiscoveryNode, aliasFilter, concreteIndexBoosts, executor, + super(logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener, shardsIts, startTime, clusterStateVersion, task); this.searchPhaseController = searchPhaseController; queryResults = new AtomicArray<>(firstResults.length()); @@ -73,9 +74,9 @@ class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { - searchTransportService.sendExecuteDfs(node, request, task, listener); + searchTransportService.sendExecuteDfs(connection, request, task, listener); } @Override @@ -84,15 +85,15 @@ class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction entry : firstResults.asList()) { DfsSearchResult dfsResult = entry.value; - DiscoveryNode node = nodeIdToDiscoveryNode.apply(dfsResult.shardTarget().nodeId()); + Transport.Connection connection = nodeIdToConnection.apply(dfsResult.shardTarget().nodeId()); QuerySearchRequest querySearchRequest = new QuerySearchRequest(request, dfsResult.id(), dfs); - executeQuery(entry.index, dfsResult, counter, querySearchRequest, node); + executeQuery(entry.index, dfsResult, counter, querySearchRequest, connection); } } void executeQuery(final int shardIndex, final DfsSearchResult dfsResult, final AtomicInteger counter, - final QuerySearchRequest querySearchRequest, final DiscoveryNode node) { - searchTransportService.sendExecuteQuery(node, querySearchRequest, task, new ActionListener() { + final QuerySearchRequest querySearchRequest, final Transport.Connection connection) { + searchTransportService.sendExecuteQuery(connection, querySearchRequest, task, new ActionListener() { @Override public void onResponse(QuerySearchResult result) { result.shardTarget(dfsResult.shardTarget()); @@ -110,7 +111,7 @@ class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction entry : docIdsToLoad.asList()) { QuerySearchResult queryResult = queryResults.get(entry.index); - DiscoveryNode node = nodeIdToDiscoveryNode.apply(queryResult.shardTarget().nodeId()); + Transport.Connection connection = nodeIdToConnection.apply(queryResult.shardTarget().nodeId()); ShardFetchSearchRequest fetchSearchRequest = createFetchRequest(queryResult, entry, lastEmittedDocPerShard); - executeFetch(entry.index, queryResult.shardTarget(), counter, fetchSearchRequest, node); + executeFetch(entry.index, queryResult.shardTarget(), counter, fetchSearchRequest, connection); } } void executeFetch(final int shardIndex, final SearchShardTarget shardTarget, final AtomicInteger counter, - final ShardFetchSearchRequest fetchSearchRequest, DiscoveryNode node) { - searchTransportService.sendExecuteFetch(node, fetchSearchRequest, task, new ActionListener() { + final ShardFetchSearchRequest fetchSearchRequest, Transport.Connection connection) { + searchTransportService.sendExecuteFetch(connection, fetchSearchRequest, task, new ActionListener() { @Override public void onResponse(FetchSearchResult result) { result.shardTarget(shardTarget); diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchQueryAndFetchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/SearchQueryAndFetchAsyncAction.java index f597ede64bc..5b20299f98c 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchQueryAndFetchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchQueryAndFetchAsyncAction.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.fetch.QueryFetchSearchResult; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.ShardSearchTransportRequest; +import org.elasticsearch.transport.Transport; import java.io.IOException; import java.util.Map; @@ -39,13 +40,13 @@ class SearchQueryAndFetchAsyncAction extends AbstractSearchAsyncAction nodeIdToDiscoveryNode, + Function nodeIdToConnection, Map aliasFilter, Map concreteIndexBoosts, SearchPhaseController searchPhaseController, Executor executor, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, long startTime, long clusterStateVersion, SearchTask task) { - super(logger, searchTransportService, nodeIdToDiscoveryNode, aliasFilter, concreteIndexBoosts, executor, + super(logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener, shardsIts, startTime, clusterStateVersion, task); this.searchPhaseController = searchPhaseController; @@ -57,9 +58,9 @@ class SearchQueryAndFetchAsyncAction extends AbstractSearchAsyncAction listener) { - searchTransportService.sendExecuteFetch(node, request, task, listener); + searchTransportService.sendExecuteFetch(connection, request, task, listener); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 7b300063291..36bfbab1b23 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -36,6 +36,7 @@ import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.ShardSearchTransportRequest; import org.elasticsearch.search.query.QuerySearchResultProvider; +import org.elasticsearch.transport.Transport; import java.io.IOException; import java.util.Map; @@ -50,13 +51,13 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction nodeIdToDiscoveryNode, + Function nodeIdToConnection, Map aliasFilter, Map concreteIndexBoosts, SearchPhaseController searchPhaseController, Executor executor, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, long startTime, long clusterStateVersion, SearchTask task) { - super(logger, searchTransportService, nodeIdToDiscoveryNode, aliasFilter, concreteIndexBoosts, executor, request, listener, + super(logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener, shardsIts, startTime, clusterStateVersion, task); this.searchPhaseController = searchPhaseController; fetchResults = new AtomicArray<>(firstResults.length()); @@ -69,9 +70,9 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction listener) { - searchTransportService.sendExecuteQuery(node, request, task, listener); + searchTransportService.sendExecuteQuery(connection, request, task, listener); } @Override @@ -90,15 +91,15 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction entry : docIdsToLoad.asList()) { QuerySearchResultProvider queryResult = firstResults.get(entry.index); - DiscoveryNode node = nodeIdToDiscoveryNode.apply(queryResult.shardTarget().nodeId()); + Transport.Connection connection = nodeIdToConnection.apply(queryResult.shardTarget().nodeId()); ShardFetchSearchRequest fetchSearchRequest = createFetchRequest(queryResult.queryResult(), entry, lastEmittedDocPerShard); - executeFetch(entry.index, queryResult.shardTarget(), counter, fetchSearchRequest, node); + executeFetch(entry.index, queryResult.shardTarget(), counter, fetchSearchRequest, connection); } } void executeFetch(final int shardIndex, final SearchShardTarget shardTarget, final AtomicInteger counter, - final ShardFetchSearchRequest fetchSearchRequest, DiscoveryNode node) { - searchTransportService.sendExecuteFetch(node, fetchSearchRequest, task, new ActionListener() { + final ShardFetchSearchRequest fetchSearchRequest, Transport.Connection connection) { + searchTransportService.sendExecuteFetch(connection, fetchSearchRequest, task, new ActionListener() { @Override public void onResponse(FetchSearchResult result) { result.shardTarget(shardTarget); diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 07307d790f9..ce5a5eec350 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.discovery.Discovery; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult; @@ -50,11 +51,13 @@ import org.elasticsearch.search.query.QuerySearchResultProvider; import org.elasticsearch.search.query.ScrollQuerySearchResult; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TaskAwareTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; @@ -90,11 +93,19 @@ public class SearchTransportService extends AbstractComponent { public static final String FETCH_ID_ACTION_NAME = "indices:data/read/search[phase/fetch/id]"; //TODO what should the setting name be? + // TODO this should be an affix settings? public static final Setting REMOTE_CLUSTERS_SEEDS = Setting.groupSetting("action.search.remote.", SearchTransportService::validateRemoteClustersSeeds, Setting.Property.NodeScope, Setting.Property.Dynamic); + /** + * The maximum number of connections that will be established to a remote cluster. For instance if there is only a single + * seed node, other nodes will be discovered up to the given number of nodes in this setting. The default is 3. + */ + public static final Setting NUM_REMOTE_CONNECTIONS = Setting.intSetting("action.search.num_remote_connections", + 3, 1, Setting.Property.NodeScope); + private final TransportService transportService; private volatile Map remoteClusters = Collections.emptyMap(); @@ -102,7 +113,7 @@ public class SearchTransportService extends AbstractComponent { super(settings); this.transportService = transportService; clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTERS_SEEDS, this::setRemoteClusters, - SearchTransportService::validateRemoteClustersSeeds); + SearchTransportService::validateRemoteClustersSeeds); } public void setupRemoteClusters() { @@ -177,7 +188,8 @@ public class SearchTransportService extends AbstractComponent { for (Map.Entry> entry : seeds.entrySet()) { RemoteClusterConnection remote = this.remoteClusters.get(entry.getKey()); if (remote == null) { - remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService); + remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService, 3, + (node) -> Version.CURRENT.isCompatible(node.getVersion())); remoteClusters.put(entry.getKey(), remote); } remote.updateSeedNodes(entry.getValue(), ActionListener.wrap((x) -> {}, @@ -242,9 +254,9 @@ public class SearchTransportService extends AbstractComponent { } } - public void sendFreeContext(DiscoveryNode node, final long contextId, SearchRequest request) { - transportService.sendRequest(node, FREE_CONTEXT_ACTION_NAME, new SearchFreeContextRequest(request, contextId), - new ActionListenerResponseHandler<>(new ActionListener() { + public void sendFreeContext(Transport.Connection connection, final long contextId, SearchRequest request) { + transportService.sendRequest(connection, FREE_CONTEXT_ACTION_NAME, new SearchFreeContextRequest(request, contextId), + TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(new ActionListener() { @Override public void onResponse(SearchFreeContextResponse response) { // no need to respond if it was freed or not @@ -267,61 +279,61 @@ public class SearchTransportService extends AbstractComponent { new ActionListenerResponseHandler<>(listener, () -> TransportResponse.Empty.INSTANCE)); } - public void sendExecuteDfs(DiscoveryNode node, final ShardSearchTransportRequest request, SearchTask task, + public void sendExecuteDfs(Transport.Connection connection, final ShardSearchTransportRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, DFS_ACTION_NAME, request, task, + transportService.sendChildRequest(connection, DFS_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, DfsSearchResult::new)); } - public void sendExecuteQuery(DiscoveryNode node, final ShardSearchTransportRequest request, SearchTask task, + public void sendExecuteQuery(Transport.Connection connection, final ShardSearchTransportRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, QUERY_ACTION_NAME, request, task, + transportService.sendChildRequest(connection, QUERY_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, QuerySearchResult::new)); } - public void sendExecuteQuery(DiscoveryNode node, final QuerySearchRequest request, SearchTask task, + public void sendExecuteQuery(Transport.Connection connection, final QuerySearchRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, QUERY_ID_ACTION_NAME, request, task, + transportService.sendChildRequest(connection, QUERY_ID_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, QuerySearchResult::new)); } public void sendExecuteQuery(DiscoveryNode node, final InternalScrollSearchRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, QUERY_SCROLL_ACTION_NAME, request, task, + transportService.sendChildRequest(transportService.getConnection(node), QUERY_SCROLL_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, ScrollQuerySearchResult::new)); } - public void sendExecuteFetch(DiscoveryNode node, final ShardSearchTransportRequest request, SearchTask task, + public void sendExecuteFetch(Transport.Connection connection, final ShardSearchTransportRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, QUERY_FETCH_ACTION_NAME, request, task, + transportService.sendChildRequest(connection, QUERY_FETCH_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, QueryFetchSearchResult::new)); } - public void sendExecuteFetch(DiscoveryNode node, final QuerySearchRequest request, SearchTask task, + public void sendExecuteFetch(Transport.Connection connection, final QuerySearchRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, QUERY_QUERY_FETCH_ACTION_NAME, request, task, + transportService.sendChildRequest(connection, QUERY_QUERY_FETCH_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, QueryFetchSearchResult::new)); } public void sendExecuteFetch(DiscoveryNode node, final InternalScrollSearchRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, QUERY_FETCH_SCROLL_ACTION_NAME, request, task, + transportService.sendChildRequest(transportService.getConnection(node), QUERY_FETCH_SCROLL_ACTION_NAME, request, task, new ActionListenerResponseHandler<>(listener, ScrollQueryFetchSearchResult::new)); } - public void sendExecuteFetch(DiscoveryNode node, final ShardFetchSearchRequest request, SearchTask task, + public void sendExecuteFetch(Transport.Connection connection, final ShardFetchSearchRequest request, SearchTask task, final ActionListener listener) { - sendExecuteFetch(node, FETCH_ID_ACTION_NAME, request, task, listener); + sendExecuteFetch(connection, FETCH_ID_ACTION_NAME, request, task, listener); } public void sendExecuteFetchScroll(DiscoveryNode node, final ShardFetchRequest request, SearchTask task, final ActionListener listener) { - sendExecuteFetch(node, FETCH_ID_SCROLL_ACTION_NAME, request, task, listener); + sendExecuteFetch(transportService.getConnection(node), FETCH_ID_SCROLL_ACTION_NAME, request, task, listener); } - private void sendExecuteFetch(DiscoveryNode node, String action, final ShardFetchRequest request, SearchTask task, + private void sendExecuteFetch(Transport.Connection connection, String action, final ShardFetchRequest request, SearchTask task, final ActionListener listener) { - transportService.sendChildRequest(node, action, request, task, + transportService.sendChildRequest(connection, action, request, task, new ActionListenerResponseHandler<>(listener, FetchSearchResult::new)); } @@ -541,4 +553,16 @@ public class SearchTransportService extends AbstractComponent { }); TransportActionProxy.registerProxyAction(transportService, FETCH_ID_ACTION_NAME, FetchSearchResult::new); } + + Transport.Connection getConnection(DiscoveryNode node) { + return transportService.getConnection(node); + } + + Transport.Connection getRemoteConnection(DiscoveryNode node, String cluster) { + RemoteClusterConnection connection = remoteClusters.get(cluster); + if (connection == null) { + throw new IllegalArgumentException("no such remote cluster: " + cluster); + } + return connection.getProxyConnection(node); + } } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 9b2d967f65c..ebb3e16d4e9 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.search; +import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; @@ -46,6 +47,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; import java.util.ArrayList; @@ -159,7 +161,7 @@ public class TransportSearchAction extends HandledTransportAction null, Collections.emptyMap(), listener); } else { // nocommit we have to extract this logic to add unittests ideally with manually prepared searchShardsResponses etc. searchTransportService.sendSearchShards(searchRequest, remoteIndicesByCluster, @@ -167,20 +169,24 @@ public class TransportSearchAction extends HandledTransportAction remoteShardIterators = new ArrayList<>(); Set remoteNodes = new HashSet<>(); Map remoteAliasFilters = new HashMap<>(); - processRemoteShards(searchShardsResponses, remoteShardIterators, remoteNodes, remoteAliasFilters); + Function connectionFunction = processRemoteShards(searchShardsResponses, + remoteShardIterators, remoteAliasFilters); executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, remoteShardIterators, - remoteNodes, remoteAliasFilters, listener); + connectionFunction, remoteAliasFilters, listener); }, listener::onFailure)); } } - private void processRemoteShards(Map searchShardsResponses, - List remoteShardIterators, Set remoteNodes, + private Function processRemoteShards(Map searchShardsResponses, + List remoteShardIterators, Map aliasFilterMap) { + Map> nodeToCluster = new HashMap<>(); for (Map.Entry entry : searchShardsResponses.entrySet()) { String clusterName = entry.getKey(); ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); - Collections.addAll(remoteNodes, searchShardsResponse.getNodes()); + for (DiscoveryNode remoteNode : searchShardsResponse.getNodes()) { + nodeToCluster.put(remoteNode.getId(), () -> searchTransportService.getRemoteConnection(remoteNode, clusterName)); + } Map indicesAndFilters = searchShardsResponse.getIndicesAndFilters(); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { //add the cluster name to the remote index names for indices disambiguation @@ -205,10 +211,17 @@ public class TransportSearchAction extends HandledTransportAction { + Supplier supplier = nodeToCluster.get(nodeId); + if (supplier == null) { + throw new IllegalArgumentException("unknown remote node: " + nodeId); + } + return supplier.get(); + }; } private void executeSearch(SearchTask task, long startTimeInMillis, SearchRequest searchRequest, String[] localIndices, - List remoteShardIterators, Set remoteNodes, + List remoteShardIterators, Function remoteConnections, Map remoteAliasMap, ActionListener listener) { ClusterState clusterState = clusterService.state(); @@ -255,8 +268,21 @@ public class TransportSearchAction extends HandledTransportAction nodesLookup = mergeNodesLookup(clusterState.nodes(), remoteNodes); - searchAsyncAction(task, searchRequest, shardIterators, startTimeInMillis, nodesLookup,clusterState.version(), + final DiscoveryNodes nodes = clusterState.nodes(); + Function connectionLookup = (nodeId) -> { + final DiscoveryNode discoveryNode = nodes.get(nodeId); + final Transport.Connection connection; + if (discoveryNode != null) { + connection = searchTransportService.getConnection(discoveryNode); + } else { + connection = remoteConnections.apply(nodeId); + } + if (connection == null) { + throw new IllegalArgumentException("no node found for id: " + nodeId); + } + return connection; + }; + searchAsyncAction(task, searchRequest, shardIterators, startTimeInMillis, connectionLookup, clusterState.version(), Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, listener).start(); } @@ -275,26 +301,7 @@ public class TransportSearchAction extends HandledTransportAction mergeNodesLookup(DiscoveryNodes nodes, Set remoteNodes) { - if (remoteNodes.isEmpty()) { - return nodes::get; - } - ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); - for (DiscoveryNode remoteNode : remoteNodes) { - builder.put(remoteNode.getId(), remoteNode); - } - ImmutableOpenMap remoteNodesMap = builder.build(); - return (nodeId) -> { - DiscoveryNode discoveryNode = nodes.get(nodeId); - if (discoveryNode == null) { - discoveryNode = remoteNodesMap.get(nodeId); - } - if (discoveryNode == null) { - throw new IllegalArgumentException("no node found for id: " + nodeId); - } - return discoveryNode; - }; - } + @Override protected final void doExecute(SearchRequest searchRequest, ActionListener listener) { @@ -302,7 +309,7 @@ public class TransportSearchAction extends HandledTransportAction nodesLookup, + long startTime, Function connectionLookup, long clusterStateVersion, Map aliasFilter, Map concreteIndexBoosts, ActionListener listener) { @@ -310,22 +317,22 @@ public class TransportSearchAction extends HandledTransportAction void sendChildRequest(final DiscoveryNode node, final String action, + public void sendChildRequest(final Transport.Connection connection, final String action, final TransportRequest request, final Task parentTask, final TransportResponseHandler handler) { - sendChildRequest(node, action, request, parentTask, TransportRequestOptions.EMPTY, handler); + sendChildRequest(connection, action, request, parentTask, TransportRequestOptions.EMPTY, handler); } - public void sendChildRequest(final DiscoveryNode node, final String action, + public void sendChildRequest(final Transport.Connection connection, final String action, final TransportRequest request, final Task parentTask, final TransportRequestOptions options, final TransportResponseHandler handler) { request.setParentTask(localNode.getId(), parentTask.getId()); try { - taskManager.registerChildTask(parentTask, node.getId()); - final Transport.Connection connection = getConnection(node); + taskManager.registerChildTask(parentTask, connection.getNode().getId()); sendRequest(connection, action, request, options, handler); } catch (TaskCancelledException ex) { // The parent task is already cancelled - just fail the request diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index 07f88134b22..ce919bac11e 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -48,12 +48,13 @@ public class RemoteClusterConnectionTests extends ESIntegTestCase { public void testConnect() throws InterruptedException { ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap dataNodes = clusterStateResponse.getState().getNodes().getDataNodes(); - DiscoveryNode node = dataNodes.valuesIt().next(); + ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getDataNodes(); + DiscoveryNode node = nodes.valuesIt().next(); try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { service.start(); service.acceptIncomingRequests(); - RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service); + RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service, + Integer.MAX_VALUE, n -> true); CountDownLatch latch = new CountDownLatch(1); AtomicReference exceptionAtomicReference = new AtomicReference<>(); ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { @@ -63,10 +64,39 @@ public class RemoteClusterConnectionTests extends ESIntegTestCase { connection.updateSeedNodes(Arrays.asList(node),listener); latch.await(); assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = dataNodes::valuesIt; + Iterable nodesIterable = nodes::valuesIt; for (DiscoveryNode dataNode : nodesIterable) { assertTrue(service.nodeConnected(dataNode)); + } + assertNull(exceptionAtomicReference.get()); + } + } + public void testConnectToSingleSeed() throws InterruptedException { + ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); + ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getNodes(); + DiscoveryNode node = nodes.valuesIt().next(); + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { + service.start(); + service.acceptIncomingRequests(); + RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service, + 1, n -> true); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.updateSeedNodes(Arrays.asList(node),listener); + latch.await(); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = nodes::valuesIt; + for (DiscoveryNode aNode : nodesIterable) { + if (aNode.equals(node)) { + assertTrue(service.nodeConnected(aNode)); + } else { + assertFalse(service.nodeConnected(aNode)); + } } assertNull(exceptionAtomicReference.get()); } @@ -82,7 +112,7 @@ public class RemoteClusterConnectionTests extends ESIntegTestCase { service.acceptIncomingRequests(); final boolean hasInitialNodes = randomBoolean(); RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", - hasInitialNodes ? Arrays.asList(node) : Collections.emptyList(), service); + hasInitialNodes ? Arrays.asList(node) : Collections.emptyList(), service, Integer.MAX_VALUE, n -> true); CountDownLatch latch = new CountDownLatch(1); if (hasInitialNodes == false) { diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index df6a1ffb4e8..8c473c0b0ca 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -39,6 +39,10 @@ import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchTransportRequest; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; import java.io.IOException; import java.util.ArrayList; @@ -80,24 +84,24 @@ public class SearchAsyncActionTests extends ESTestCase { SearchTransportService transportService = new SearchTransportService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, Collections.singleton(SearchTransportService.REMOTE_CLUSTERS_SEEDS)), null) { @Override - public void sendFreeContext(DiscoveryNode node, long contextId, SearchRequest request) { + public void sendFreeContext(Transport.Connection connection, long contextId, SearchRequest request) { numFreedContext.incrementAndGet(); - assertTrue(nodeToContextMap.containsKey(node)); - assertTrue(nodeToContextMap.get(node).remove(contextId)); + assertTrue(nodeToContextMap.containsKey(connection.getNode())); + assertTrue(nodeToContextMap.get(connection.getNode()).remove(contextId)); } }; - Map lookup = new HashMap<>(); - lookup.put(primaryNode.getId(), primaryNode); + Map lookup = new HashMap<>(); + lookup.put(primaryNode.getId(), new MockConnection(primaryNode)); Map aliasFilters = Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)); AbstractSearchAsyncAction asyncAction = new AbstractSearchAsyncAction(logger, transportService, lookup::get, aliasFilters, Collections.emptyMap(), null, request, responseListener, shardsIter, 0, 0, null) { TestSearchResponse response = new TestSearchResponse(); @Override - protected void sendExecuteFirstPhase(DiscoveryNode node, ShardSearchTransportRequest request, ActionListener listener) { + protected void sendExecuteFirstPhase(Transport.Connection connection, ShardSearchTransportRequest request, ActionListener listener) { assertTrue("shard: " + request.shardId() + " has been queried twice", response.queried.add(request.shardId())); - TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(contextIdGenerator.incrementAndGet(), node); - Set ids = nodeToContextMap.computeIfAbsent(node, (n) -> new HashSet<>()); + TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(contextIdGenerator.incrementAndGet(), connection.getNode()); + Set ids = nodeToContextMap.computeIfAbsent(connection.getNode(), (n) -> new HashSet<>()); ids.add(testSearchPhaseResult.id); if (randomBoolean()) { listener.onResponse(testSearchPhaseResult); @@ -111,7 +115,7 @@ public class SearchAsyncActionTests extends ESTestCase { for (int i = 0; i < firstResults.length(); i++) { TestSearchPhaseResult result = firstResults.get(i); assertEquals(result.node.getId(), result.shardTarget().getNodeId()); - sendReleaseSearchContext(result.id(), result.node); + sendReleaseSearchContext(result.id(), new MockConnection(result.node)); } responseListener.onResponse(response); latch.countDown(); @@ -213,4 +217,28 @@ public class SearchAsyncActionTests extends ESTestCase { } } + + public final class MockConnection implements Transport.Connection { + + private final DiscoveryNode node; + + public MockConnection(DiscoveryNode node) { + this.node = node; + } + + @Override + public DiscoveryNode getNode() { + return node; + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException { + throw new UnsupportedOperationException(); + } + } } diff --git a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java index 95978991046..6cc07a92dca 100644 --- a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -20,19 +20,38 @@ package org.elasticsearch.search.simple; import org.apache.lucene.util.Constants; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.discovery.zen.PublishClusterStateActionTests; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.node.MockNode; +import org.elasticsearch.node.Node; +import org.elasticsearch.node.NodeValidationException; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.rescore.QueryRescorerBuilder; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.discovery.TestZenDiscovery; +import org.elasticsearch.transport.MockTcpTransportPlugin; +import java.nio.file.Path; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.concurrent.ExecutionException; @@ -52,6 +71,12 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFa import static org.hamcrest.Matchers.containsString; public class SimpleSearchIT extends ESIntegTestCase { + + @Override + protected boolean useSearchProxyNode() { + return true; + } + public void testSearchNullIndex() { expectThrows(NullPointerException.class, () -> client().prepareSearch((String) null).setQuery(QueryBuilders.termQuery("_id", "XXX1")).get()); @@ -417,7 +442,8 @@ public class SimpleSearchIT extends ESIntegTestCase { client().prepareSearch("idx").setQuery(QueryBuilders.regexpQuery("num", "34")).get(); fail("SearchPhaseExecutionException should have been thrown"); } catch (SearchPhaseExecutionException ex) { - assertThat(ex.getCause().getCause().getMessage(), containsString("Can only use regexp queries on keyword and text fields")); + assertThat(ex.getRootCause().getMessage(), + containsString("Can only use regexp queries on keyword and text fields")); } } diff --git a/qa/multi-cluster-search/build.gradle b/qa/multi-cluster-search/build.gradle index fe4e9524165..68e6cf414c0 100644 --- a/qa/multi-cluster-search/build.gradle +++ b/qa/multi-cluster-search/build.gradle @@ -25,7 +25,7 @@ task remoteClusterTest(type: RestIntegTestTask) { mustRunAfter(precommit) cluster { distribution = 'zip' - numNodes = 1 + numNodes = 2 clusterName = 'remote-cluster' } systemProperty 'tests.rest.suite', 'remote_cluster' @@ -36,10 +36,11 @@ task mixedClusterTest(type: RestIntegTestTask) { cluster { distribution = 'zip' setting 'action.search.remote.my_remote_cluster', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" + setting 'action.search.num_remote_connections', 1 } systemProperty 'tests.rest.suite', 'multi_cluster' - finalizedBy 'remoteClusterTest#stop' + finalizedBy 'remoteClusterTest#node0.stop','remoteClusterTest#node1.stop' } task integTest { diff --git a/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java b/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java index ee140da23e7..e8c3592e214 100644 --- a/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java +++ b/qa/multi-cluster-search/src/test/java/org/elasticsearch/upgrades/MultiClusterSearchYamlTestSuiteIT.java @@ -24,7 +24,6 @@ import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite; import org.apache.lucene.util.TimeUnits; import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; -import org.elasticsearch.test.rest.yaml.parser.ClientYamlTestParseException; import java.io.IOException; @@ -41,7 +40,7 @@ public class MultiClusterSearchYamlTestSuiteIT extends ESClientYamlSuiteTestCase } @ParametersFactory - public static Iterable parameters() throws IOException, ClientYamlTestParseException { + public static Iterable parameters() throws IOException { return createParameters(); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 038477cc3aa..9f59508bca4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -30,6 +30,7 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.ShardOperationFailedException; @@ -54,19 +55,24 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollResponse; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; +import org.elasticsearch.client.FilterClient; import org.elasticsearch.client.Requests; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.cluster.ClusterModule; +import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -77,8 +83,10 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkModule; @@ -90,6 +98,7 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; @@ -117,7 +126,10 @@ import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.store.IndicesStore; +import org.elasticsearch.node.MockNode; +import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeMocksPlugin; +import org.elasticsearch.node.NodeValidationException; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.script.ScriptService; @@ -138,6 +150,7 @@ import org.junit.BeforeClass; import java.io.IOException; import java.io.InputStream; +import java.io.UncheckedIOException; import java.lang.annotation.Annotation; import java.lang.annotation.ElementType; import java.lang.annotation.Inherited; @@ -160,6 +173,7 @@ import java.util.IdentityHashMap; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; @@ -336,6 +350,7 @@ public abstract class ESIntegTestCase extends ESTestCase { private static ESIntegTestCase INSTANCE = null; // see @SuiteScope private static Long SUITE_SEED = null; + private static Node searchProxyNode; @BeforeClass public static void beforeClass() throws Exception { @@ -366,6 +381,9 @@ public abstract class ESIntegTestCase extends ESTestCase { cluster().beforeTest(random(), getPerTestTransportClientRatio()); cluster().wipe(excludeTemplates()); randomIndexTemplate(); + if (useSearchProxyNode()) { + searchProxyNode = startSearchProxyNode(); + } } private void printTestMessage(String message) { @@ -538,6 +556,7 @@ public abstract class ESIntegTestCase extends ESTestCase { protected final void afterInternal(boolean afterClass) throws Exception { boolean success = false; try { + final Scope currentClusterScope = getCurrentClusterScope(); clearDisruptionScheme(); try { @@ -581,6 +600,8 @@ public abstract class ESIntegTestCase extends ESTestCase { if (currentClusterScope == Scope.TEST) { clearClusters(); // it is ok to leave persistent / transient cluster state behind if scope is TEST } + IOUtils.close(searchProxyNode); + searchProxyNode = null; } success = true; } finally { @@ -634,6 +655,53 @@ public abstract class ESIntegTestCase extends ESTestCase { if (frequently()) { client = new RandomizingClient(client, random()); } + if (searchProxyNode != null && randomBoolean()) { + client = new FilterClient(client) { + @Override + public SearchRequestBuilder prepareSearch(String... indices) { + return searchProxyNode.client().prepareSearch(convertToRemoteIndices(indices)); + } + + private String[] convertToRemoteIndices(String[] indices) { + if (Objects.requireNonNull(indices).length == 0) { + return new String[] {"test_remote_cluster|_all"}; + } + String [] remoteIndices = new String[indices.length]; + for (int i = 0; i < indices.length; i++) { + remoteIndices[i] = "test_remote_cluster|"+ Objects.requireNonNull(indices[i]); + } + return remoteIndices; + } + + @Override + public ActionFuture search(SearchRequest request) { + // we copy the request to ensure we never modify the original request + try (BytesStreamOutput out = new BytesStreamOutput()) { + request.writeTo(out); + SearchRequest copy = new SearchRequest(); + copy.readFrom(out.bytes().streamInput()); + copy.indices(convertToRemoteIndices(request.indices())); + return searchProxyNode.client().search(copy); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + + @Override + public void search(SearchRequest request, ActionListener listener) { + // we copy the request to ensure we never modify the original request + try (BytesStreamOutput out = new BytesStreamOutput()) { + request.writeTo(out); + SearchRequest copy = new SearchRequest(); + copy.readFrom(out.bytes().streamInput()); + copy.indices(convertToRemoteIndices(request.indices())); + searchProxyNode.client().search(copy, listener); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } + }; + } return client; } @@ -2203,4 +2271,47 @@ public abstract class ESIntegTestCase extends ESTestCase { String uuid = getIndexResponse.getSettings().get(index).get(IndexMetaData.SETTING_INDEX_UUID); return new Index(index, uuid); } + + protected boolean useSearchProxyNode() { + return false; // nocommit - lets enable this globally + } + + private synchronized Node startSearchProxyNode() { + if (isInternalCluster()) { + final DiscoveryNode seedNode = internalCluster().getInstance(ClusterService.class).localNode(); + final Path tempDir = createTempDir(); + Settings settings = Settings.builder() + .put(ClusterName.CLUSTER_NAME_SETTING.getKey(), "search_proxy_" + internalCluster().getClusterName()) + .put(Environment.PATH_HOME_SETTING.getKey(), tempDir) + .put(Environment.PATH_REPO_SETTING.getKey(), tempDir.resolve("repo")) + .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), createTempDir().getParent()) + .put("action.search.remote.test_remote_cluster", seedNode.getAddress().toString()) + .put("node.name", "node_prx_0") + .put(EsExecutors.PROCESSORS_SETTING.getKey(), 1) // limit the number of threads created + .put(NetworkModule.HTTP_ENABLED.getKey(), false) + .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) + .put(Node.NODE_DATA_SETTING.getKey(), false) + .put(Node.NODE_MASTER_SETTING.getKey(), true) + .put(Node.NODE_INGEST_SETTING.getKey(), false) + .build(); + Collection> plugins = nodePlugins(); + if (plugins.contains(MockTcpTransportPlugin.class) == false) { + plugins = new ArrayList<>(plugins); + plugins.add(MockTcpTransportPlugin.class); + } + if (plugins.contains(TestZenDiscovery.TestPlugin.class) == false) { + plugins = new ArrayList<>(plugins); + plugins.add(TestZenDiscovery.TestPlugin.class); + } + Node build = new MockNode(settings, plugins); + try { + build.start(); + } catch (NodeValidationException e) { + throw new RuntimeException(e); + } + return build; + } else { + return null; + } + } } From 31499a12482897ad7aa319b77a4cb37affcd5d5a Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 4 Jan 2017 11:23:33 +0100 Subject: [PATCH 22/59] handle nodes that are not connected early in AbstractSearchAsyncAction --- .../search/AbstractSearchAsyncAction.java | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 4b57dd5f59b..2aed6ad3f2d 100644 --- a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -41,6 +41,7 @@ import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.search.internal.ShardSearchTransportRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.query.QuerySearchResultProvider; +import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.Transport; import java.util.List; @@ -120,30 +121,33 @@ abstract class AbstractSearchAsyncAction void performFirstPhase(final int shardIndex, final ShardIterator shardIt, final ShardRouting shard) { if (shard == null) { + // TODO upgrade this to an assert... // no more active shards... (we should not really get here, but just for safety) onFirstPhaseResult(shardIndex, null, null, shardIt, new NoShardAvailableActionException(shardIt.shardId())); } else { - final Transport.Connection connection = nodeIdToConnection.apply(shard.currentNodeId()); - if (connection == null) { - onFirstPhaseResult(shardIndex, shard, null, shardIt, new NoShardAvailableActionException(shardIt.shardId())); - } else { + try { + final Transport.Connection connection = nodeIdToConnection.apply(shard.currentNodeId()); AliasFilter filter = this.aliasFilter.get(shard.index().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shard.index().getUUID(), DEFAULT_INDEX_BOOST); ShardSearchTransportRequest transportRequest = new ShardSearchTransportRequest(request, shardIt.shardId(), shardsIts.size(), filter, indexBoost, startTime()); - sendExecuteFirstPhase(connection, transportRequest , new ActionListener() { - @Override - public void onResponse(FirstResult result) { - onFirstPhaseResult(shardIndex, shard.currentNodeId(), result, shardIt); - } + sendExecuteFirstPhase(connection, transportRequest, new ActionListener() { + @Override + public void onResponse(FirstResult result) { + onFirstPhaseResult(shardIndex, shard.currentNodeId(), result, shardIt); + } - @Override - public void onFailure(Exception t) { - onFirstPhaseResult(shardIndex, shard, connection.getNode().getId(), shardIt, t); - } - }); + @Override + public void onFailure(Exception t) { + onFirstPhaseResult(shardIndex, shard, connection.getNode().getId(), shardIt, t); + } + }); + } catch (ConnectTransportException | IllegalArgumentException ex) { + // we are getting the connection early here so we might run into nodes that are not connected. in that case we move on to + // the next shard. + onFirstPhaseResult(shardIndex, shard, shard.currentNodeId(), shardIt, ex); } } } From dd0331144a710f568a2045af69453a32bf3b503c Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 4 Jan 2017 13:31:59 +0100 Subject: [PATCH 23/59] [TEST] Also register replica node in node map --- .../org/elasticsearch/action/search/SearchAsyncActionTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index 8c473c0b0ca..60ebe6f0ae2 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -92,6 +92,7 @@ public class SearchAsyncActionTests extends ESTestCase { }; Map lookup = new HashMap<>(); lookup.put(primaryNode.getId(), new MockConnection(primaryNode)); + lookup.put(replicaNode.getId(), new MockConnection(replicaNode)); Map aliasFilters = Collections.singletonMap("_na_", new AliasFilter(null, Strings.EMPTY_ARRAY)); AbstractSearchAsyncAction asyncAction = new AbstractSearchAsyncAction(logger, transportService, lookup::get, aliasFilters, Collections.emptyMap(), null, request, responseListener, shardsIter, 0, 0, null) { From e6429658045890de26c45f57927db77147637853 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 4 Jan 2017 17:26:00 +0100 Subject: [PATCH 24/59] Cleanup lots of code, add javadocs and tests --- .../search/RemoteClusterConnection.java | 305 ++++++++++------ .../action/search/RemoteClusterService.java | 284 +++++++++++++++ .../action/search/SearchTransportService.java | 205 ++--------- .../action/search/TransportSearchAction.java | 12 +- .../common/settings/ClusterSettings.java | 6 +- .../java/org/elasticsearch/node/Node.java | 8 +- .../transport/TransportService.java | 2 +- .../search/RemoteClusterConnectionIT.java | 330 ++++++++++++++++++ .../search/RemoteClusterConnectionTests.java | 163 --------- ...ts.java => RemoteClusterServiceTests.java} | 20 +- .../action/search/SearchAsyncActionTests.java | 2 +- qa/multi-cluster-search/build.gradle | 4 +- .../test/multi_cluster/10_basic.yaml | 6 +- .../elasticsearch/test/ESIntegTestCase.java | 2 +- .../test/transport/MockTransportService.java | 4 + 15 files changed, 867 insertions(+), 486 deletions(-) create mode 100644 core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java create mode 100644 core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java delete mode 100644 core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java rename core/src/test/java/org/elasticsearch/action/search/{SearchTransportServiceTests.java => RemoteClusterServiceTests.java} (75%) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 8e3ffc737ac..e9065a110f2 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -20,7 +20,8 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; -import org.elasticsearch.Version; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; @@ -32,11 +33,11 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.discovery.Discovery; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.ConnectionProfile; +import org.elasticsearch.transport.TcpTransport; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TransportConnectionListener; @@ -46,6 +47,7 @@ import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -57,28 +59,53 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; -final class RemoteClusterConnection extends AbstractComponent implements TransportConnectionListener { +/** + * Represents a connection to a single remote cluster. In contrast to a local cluster a remote cluster is not joined such that the + * current node is part of the cluster and it won't receive cluster state updated from the remote cluster. Remote clusters are also not + * fully connected with the current node. From a connectin perspective a local cluster forms a bi-directional star network while in the + * remote case we only connect to a subset of the nodes in the cluster in an uni-directional fashion. + * + * This class also handles the discovery of nodes from the remote cluster. The initial list of seed nodes is only used to discover all nodes + * in the remote cluster and connects to all eligible nodes, for details see {@link RemoteClusterService#REMOTE_NODE_ATTRIBUTE}. + * + * In the case of a disconnection, this class will issue a re-connect task to establish at most + * {@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 extends AbstractComponent implements TransportConnectionListener, Closeable { private final TransportService transportService; private final ConnectionProfile remoteProfile; - private final CopyOnWriteArrayList clusterNodes = new CopyOnWriteArrayList(); + private final CopyOnWriteArrayList connectedNodes = new CopyOnWriteArrayList(); private final Supplier nodeSupplier; private final String clusterName; - private final int numSeedNodesToDiscover; + private final int maxNumRemoteConnections; private final Predicate nodePredicate; private volatile List seedNodes; private final ConnectHandler connectHandler; + /** + * Creates a new {@link RemoteClusterConnection} + * @param settings the nodes settings object + * @param clusterName the configured name 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 eligable remote nodes to connect to + */ RemoteClusterConnection(Settings settings, String clusterName, List seedNodes, - TransportService transportService, int numSeedNodesToDiscover, Predicate nodePredicate) { + TransportService transportService, int maxNumRemoteConnections, Predicate nodePredicate) { super(settings); this.transportService = transportService; - this.numSeedNodesToDiscover = numSeedNodesToDiscover; + this.maxNumRemoteConnections = maxNumRemoteConnections; this.nodePredicate = nodePredicate; this.clusterName = clusterName; ConnectionProfile.Builder builder = new ConnectionProfile.Builder(); + builder.setConnectTimeout(TcpTransport.TCP_CONNECT_TIMEOUT.get(settings)); + builder.setHandshakeTimeout(TcpTransport.TCP_CONNECT_TIMEOUT.get(settings)); builder.addConnections(6, TransportRequestOptions.Type.REG, TransportRequestOptions.Type.PING); // TODO make this configurable? builder.addConnections(0, // we don't want this to be used for anything else but search TransportRequestOptions.Type.BULK, @@ -90,9 +117,9 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public DiscoveryNode get() { if (current == null || current.hasNext() == false) { - current = clusterNodes.iterator(); + current = connectedNodes.iterator(); if (current.hasNext() == false) { - throw new IllegalStateException("No node available for cluster: " + clusterName + " nodes: " + clusterNodes ); + throw new IllegalStateException("No node available for cluster: " + clusterName + " nodes: " + connectedNodes); } } return current.next(); @@ -100,8 +127,12 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo }; this.seedNodes = Collections.unmodifiableList(seedNodes); this.connectHandler = new ConnectHandler(); + transportService.addConnectionListener(this); } + /** + * Updates the list of seed nodes for this cluster connection + */ synchronized void updateSeedNodes(List seedNodes, ActionListener connectListener) { this.seedNodes = Collections.unmodifiableList(new ArrayList<>(seedNodes)); connectHandler.connect(connectListener); @@ -109,64 +140,32 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public void onNodeDisconnected(DiscoveryNode node) { - boolean remove = clusterNodes.remove(node); - if (remove == true && clusterNodes.isEmpty()) { - // try to reconnect - ActionListener listener = ActionListener.wrap(x -> {}, - e -> logger.error("failed to establish connection to remote cluster", e)); - connectHandler.connect(listener); - } - } - - private void ensureConnected(DiscoveryNode[] nodes) { - boolean seenNotConnectedNode = false; - for (DiscoveryNode node : nodes) { - if (transportService.nodeConnected(node) == false) { - seenNotConnectedNode = true; - transportService.connectToNode(node, remoteProfile); - } - } - if (seenNotConnectedNode) { - TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable - connectHandler.handshakeAndConnect(clusterNodes.iterator(), transportService, connectTimeout, - ActionListener.wrap((x) -> { - }, x -> { - }), false); // nocommit handle exceptions here what should we do + boolean remove = connectedNodes.remove(node); + if (remove == true && connectedNodes.size() < maxNumRemoteConnections) { + // try to reconnect and fill up the slot of the disconnected node + connectHandler.maybeConnect(); } } + /** + * Fetches all shards for the search request from this remote connection. This is used to later run the search on the remote end. + */ public void fetchSearchShards(SearchRequest searchRequest, final List indices, ActionListener listener) { - if (clusterNodes.isEmpty()) { - connectHandler.connect(ActionListener.wrap((x) -> fetchSearchShards(searchRequest, indices, listener), listener::onFailure)); + if (connectedNodes.isEmpty()) { + // just in case if we are not connected for some reason we try to connect and if we fail we to and notify the listener + // this will cause some back pressure on the search end and eventually will cause rejections but that's fine + // we can't proceed with a search on a cluster level. + // in the future we might want to just skip the remote nodes in such a case but that can already be implemented on the caller + // end since they provide the listener. + connectHandler.connect(ActionListener.wrap((x) -> fetchShardsInternal(searchRequest, indices, listener), listener::onFailure)); } else { fetchShardsInternal(searchRequest, indices, listener); } } - public Transport.Connection getProxyConnection(DiscoveryNode nodeToProxyTo) { - DiscoveryNode discoveryNode = nodeSupplier.get(); - Transport.Connection connection = transportService.getConnection(discoveryNode); - return new Transport.Connection() { - @Override - public DiscoveryNode getNode() { - return nodeToProxyTo; - } - - @Override - public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException { - connection.sendRequest(requestId, TransportActionProxy.getProxyAction(action), - TransportActionProxy.wrapRequest(nodeToProxyTo, request), options); - } - - @Override - public void close() throws IOException { - throw new IllegalStateException("never close a proxy connection"); - } - }; - } - - private void fetchShardsInternal(SearchRequest searchRequest, List indices, final ActionListener listener) { + private void fetchShardsInternal(SearchRequest searchRequest, List indices, + final ActionListener listener) { final DiscoveryNode node = nodeSupplier.get(); ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices.toArray(new String[indices.size()])) .indicesOptions(searchRequest.indicesOptions()).local(true).preference(searchRequest.preference()) @@ -181,7 +180,6 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public void handleResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { - ensureConnected(clusterSearchShardsResponse.getNodes()); listener.onResponse(clusterSearchShardsResponse); } @@ -197,14 +195,54 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo }); } + + /** + * Returns a connection to the remote cluster. This connection might be a proxy connection that redirects internally to the + * given node. + */ + Transport.Connection getConnection(DiscoveryNode remoteClusterNode) { + DiscoveryNode discoveryNode = nodeSupplier.get(); + Transport.Connection connection = transportService.getConnection(discoveryNode); + return new Transport.Connection() { + @Override + public DiscoveryNode getNode() { + return remoteClusterNode; + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws IOException, TransportException { + connection.sendRequest(requestId, TransportActionProxy.getProxyAction(action), + TransportActionProxy.wrapRequest(remoteClusterNode, request), options); + } + + @Override + public void close() throws IOException { + assert false: "proxy connections must not be closed"; + } + }; + } + + /** + * Returns the name of the remote cluster + */ public String getClusterName() { return clusterName; } + @Override + public void close() throws IOException { + connectHandler.close(); + } - private class ConnectHandler { - private Semaphore running = new Semaphore(1); - private BlockingQueue> queue = new ArrayBlockingQueue<>(100); + private class ConnectHandler implements Closeable { + private final Semaphore running = new Semaphore(1); + private final AtomicBoolean closed = new AtomicBoolean(false); + private final BlockingQueue> queue = new ArrayBlockingQueue<>(100); + + public void maybeConnect() { + connect(null); + } public void connect(ActionListener connectListener) { final boolean runConnect; @@ -213,13 +251,19 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo if (connectListener != null && queue.offer(connectListener) == false) { throw new IllegalStateException("connect queue is full"); } - if (queue.isEmpty()) { + if (connectListener != null && queue.isEmpty()) { return; } runConnect = running.tryAcquire(); + if (runConnect) { toNotify = new ArrayList<>(); queue.drainTo(toNotify); + if (closed.get()) { + for (ActionListener listener : toNotify) { + listener.onFailure(new AlreadyClosedException("connecte handler is already closed")); + } + } } else { toNotify = Collections.emptyList(); } @@ -227,7 +271,6 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo if (runConnect) { forkConnect(toNotify); } - } private void forkConnect(final Collection> toNotify) { @@ -247,14 +290,13 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override protected void doRun() throws Exception { ActionListener listener = ActionListener.wrap((x) -> { - synchronized (queue) { running.release(); } for (ActionListener queuedListener : toNotify) { queuedListener.onResponse(x); } - connect(null); + maybeConnect(); }, (e) -> { synchronized (queue) { @@ -263,72 +305,92 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo for (ActionListener queuedListener : toNotify) { queuedListener.onFailure(e); } - connect(null); + maybeConnect(); }); - TimeValue connectTimeout = TimeValue.timeValueSeconds(10); // TODO make configurable Iterator iterator = Collections.synchronizedList(seedNodes).iterator(); - handshakeAndConnect(iterator, transportService, connectTimeout, listener, true); + collectRemoteNodes(iterator, transportService, listener); } }); } - - - void handshakeAndConnect(Iterator seedNodes, - final TransportService transportService, TimeValue connectTimeout, ActionListener listener, - boolean connect) { + void collectRemoteNodes(Iterator seedNodes, + final TransportService transportService, ActionListener listener) { try { if (seedNodes.hasNext()) { final DiscoveryNode seedNode = seedNodes.next(); final DiscoveryNode handshakeNode; - if (connect) { - try (Transport.Connection connection = transportService.openConnection(seedNode, - ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null,null))) { - handshakeNode = transportService.handshake(connection, connectTimeout.millis(), (c) -> true); + Transport.Connection connection = transportService.openConnection(seedNode, + ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null,null)); + boolean success = false; + try { + handshakeNode = transportService.handshake(connection, remoteProfile.getHandshakeTimeout().millis(), + (c) -> true); + if (nodePredicate.test(handshakeNode) && connectedNodes.size() < maxNumRemoteConnections) { transportService.connectToNode(handshakeNode, remoteProfile); - clusterNodes.add(handshakeNode); + connectedNodes.add(handshakeNode); } - } else { - handshakeNode = seedNode; - } - ClusterStateRequest request = new ClusterStateRequest(); - request.clear(); - request.nodes(true); - transportService.sendRequest(transportService.getConnection(handshakeNode), - ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, - new TransportResponseHandler() { + ClusterStateRequest request = new ClusterStateRequest(); + request.clear(); + request.nodes(true); + transportService.sendRequest(connection, + ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, + new TransportResponseHandler() { - @Override - public ClusterStateResponse newInstance() { - return new ClusterStateResponse(); - } + @Override + public ClusterStateResponse newInstance() { + return new ClusterStateResponse(); + } - @Override - public void handleResponse(ClusterStateResponse response) { - DiscoveryNodes nodes = response.getState().nodes(); - Iterable nodesIter = nodes.getNodes()::valuesIt; - for (DiscoveryNode node : nodesIter) { - if (nodePredicate.test(node) && clusterNodes.size() < numSeedNodesToDiscover) { - transportService.connectToNode(node); // noop if node is connected - clusterNodes.add(node); + @Override + public void handleResponse(ClusterStateResponse response) { + try { + DiscoveryNodes nodes = response.getState().nodes(); + Iterable nodesIter = nodes.getNodes()::valuesIt; + for (DiscoveryNode node : nodesIter) { + if (nodePredicate.test(node) && connectedNodes.size() < maxNumRemoteConnections) { + try { + transportService.connectToNode(node, remoteProfile); // noop if node is connected + connectedNodes.add(node); + } catch (ConnectTransportException ex) { + // fair enough we can't connect just move on + logger.debug((Supplier) + () -> new ParameterizedMessage("failed to connect to node {}", node), ex); + } + + } + } + listener.onResponse(null); + + } catch (Exception ex) { + logger.warn((Supplier) + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), ex); + collectRemoteNodes(seedNodes, transportService, listener); + } finally { + IOUtils.closeWhileHandlingException(connection); } } - listener.onResponse(null); - } - @Override - public void handleException(TransportException exp) { - logger.warn((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), exp); - handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); - } + @Override + public void handleException(TransportException exp) { + logger.warn((Supplier) + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), exp); + collectRemoteNodes(seedNodes, transportService, listener); + IOUtils.closeWhileHandlingException(connection); + } + + @Override + public String executor() { + return ThreadPool.Names.MANAGEMENT; + } + }); + success = true; + } finally { + if (success == false) { + connection.close(); + } + } - @Override - public String executor() { - return ThreadPool.Names.MANAGEMENT; - } - }); } else { listener.onFailure(new IllegalStateException("no seed node left")); } @@ -336,12 +398,27 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo if (seedNodes.hasNext()) { logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), ex); - handshakeAndConnect(seedNodes, transportService, connectTimeout, listener, connect); + collectRemoteNodes(seedNodes, transportService, listener); } else { listener.onFailure(ex); } } } + + @Override + public void close() throws IOException { + try { + closed.compareAndSet(false, true); + running.acquire(); + running.release(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + synchronized (queue) { + + running.release(); + } + } } } diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java new file mode 100644 index 00000000000..b9af31f755c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -0,0 +1,284 @@ +/* + * 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.action.search; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportService; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; + + +public final class RemoteClusterService extends AbstractComponent { + + /** + * A list of initial seed nodes to discover eligibale nodes from the remote cluster + */ + //TODO this should be an affix settings? + public static final Setting REMOTE_CLUSTERS_SEEDS = Setting.groupSetting("search.remote.seeds.", + RemoteClusterService::validateRemoteClustersSeeds, + Setting.Property.NodeScope, + Setting.Property.Dynamic); + /** + * The maximum number of connections that will be established to a remote cluster. For instance if there is only a single + * seed node, other nodes will be discovered up to the given number of nodes in this setting. The default is 3. + */ + public static final Setting REMOTE_CONNECTIONS_PER_CLUSTER = Setting.intSetting("search.remote.connections_per_cluster", + 3, 1, Setting.Property.NodeScope); + + /** + * The initial connect timeout for remote cluster connections + */ + public static final Setting REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING = + Setting.positiveTimeSetting("search.remote.initial_connect_timeout", TimeValue.timeValueSeconds(30), Setting.Property.NodeScope); + + /** + * The name of a node attribute to filter out nodes that should not be connected to in the remote cluster. + * For instance a node can be configured with node.node_attr.gateway: true in order to be eligable as a gateway node between + * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster + */ + public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", + Setting.Property.NodeScope); + private final TransportService transportService; + private final int numRemoteConnections; + private volatile Map remoteClusters = Collections.emptyMap(); + + RemoteClusterService(Settings settings, TransportService transportService) { + super(settings); + this.transportService = transportService; + numRemoteConnections = REMOTE_CONNECTIONS_PER_CLUSTER.get(settings); + } + + void updateRemoteClusters(Settings seedSettings, ActionListener connectionListener) { + Map remoteClusters = new HashMap<>(); + Map> seeds = buildRemoteClustersSeeds(seedSettings); + if (seeds.isEmpty()) { + connectionListener.onResponse(null); + } else { + CountDown countDown = new CountDown(seeds.size()); + Predicate nodePredicate = (node) -> Version.CURRENT.isCompatible(node.getVersion()); + 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); + nodePredicate = nodePredicate.and((node) -> Boolean.getBoolean(node.getAttributes().getOrDefault(attribute, "false"))); + } + for (Map.Entry> entry : seeds.entrySet()) { + RemoteClusterConnection remote = this.remoteClusters.get(entry.getKey()); + if (remote == null) { + remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService, numRemoteConnections, + nodePredicate); + remoteClusters.put(entry.getKey(), remote); + } + remote.updateSeedNodes(entry.getValue(), ActionListener.wrap( + x -> { + if (countDown.countDown()) { + connectionListener.onResponse(x); + } + }, + e -> { + if (countDown.fastForward()) { + connectionListener.onFailure(e); + } + logger.error("failed to update seed list for cluster: " + entry.getKey(), e); + })); + } + } + if (remoteClusters.isEmpty() == false) { + remoteClusters.putAll(this.remoteClusters); + this.remoteClusters = Collections.unmodifiableMap(remoteClusters); + } + } + + /** + * Returns true if at least one remote cluster is configured + */ + boolean isCrossClusterSearchEnabled() { + return remoteClusters.isEmpty() == false; + } + + /** + * Returns true iff the given cluster is configured as a remote cluster. Otherwise false + */ + boolean isRemoteClusterRegistered(String clusterName) { + return remoteClusters.containsKey(clusterName); + } + + void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, + ActionListener> listener) { + final CountDown responsesCountDown = new CountDown(remoteIndicesByCluster.size()); + final Map searchShardsResponses = new ConcurrentHashMap<>(); + final AtomicReference transportException = new AtomicReference<>(); + for (Map.Entry> entry : remoteIndicesByCluster.entrySet()) { + final String clusterName = entry.getKey(); + RemoteClusterConnection remoteClusterConnection = remoteClusters.get(clusterName); + if (remoteClusterConnection == null) { + throw new IllegalArgumentException("no such remote cluster: " + clusterName); + } + final List indices = entry.getValue(); + remoteClusterConnection.fetchSearchShards(searchRequest, indices, + new ActionListener() { + @Override + public void onResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { + searchShardsResponses.put(clusterName, clusterSearchShardsResponse); + if (responsesCountDown.countDown()) { + TransportException exception = transportException.get(); + if (exception == null) { + listener.onResponse(searchShardsResponses); + } else { + listener.onFailure(transportException.get()); + } + } + } + + @Override + public void onFailure(Exception e) { + TransportException exception = new TransportException("unable to communicate with remote cluster [" + + clusterName + "]", e); + if (transportException.compareAndSet(null, exception) == false) { + exception = transportException.accumulateAndGet(exception, (previous, current) -> { + current.addSuppressed(previous); + return current; + }); + } + if (responsesCountDown.countDown()) { + listener.onFailure(exception); + } + } + }); + } + } + + /** + * Returns a connection to the given node on the given remote cluster + * @throws IllegalArgumentException if the remote cluster is unknown + */ + Transport.Connection getConnection(DiscoveryNode node, String cluster) { + RemoteClusterConnection connection = remoteClusters.get(cluster); + if (connection == null) { + throw new IllegalArgumentException("no such remote cluster: " + cluster); + } + return connection.getConnection(node); + } + + + static Map> buildRemoteClustersSeeds(Settings settings) { + Map> remoteClustersNodes = new HashMap<>(); + for (String clusterName : settings.names()) { + String[] remoteHosts = settings.getAsArray(clusterName); + for (String remoteHost : remoteHosts) { + int portSeparator = remoteHost.lastIndexOf(':'); // in case we have a IPv6 address ie. [::1]:9300 + String host = remoteHost.substring(0, portSeparator); + InetAddress hostAddress; + try { + hostAddress = InetAddress.getByName(host); + } catch (UnknownHostException e) { + throw new IllegalArgumentException("unknown host [" + host + "]", e); + } + int port = Integer.valueOf(remoteHost.substring(portSeparator + 1)); + DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, + new TransportAddress(new InetSocketAddress(hostAddress, port)), + Version.CURRENT.minimumCompatibilityVersion()); + //don't connect yet as that would require the remote node to be up and would fail the local node startup otherwise + List nodes = remoteClustersNodes.get(clusterName); + if (nodes == null) { + nodes = new ArrayList<>(); + remoteClustersNodes.put(clusterName, nodes); + } + nodes.add(node); + } + } + return remoteClustersNodes; + } + + static void validateRemoteClustersSeeds(Settings settings) { + //TODO do we need a static whitelist like in reindex from remote? + for (String clusterName : settings.names()) { + String[] remoteHosts = settings.getAsArray(clusterName); + if (remoteHosts.length == 0) { + throw new IllegalArgumentException("no hosts set for remote cluster [" + clusterName + "], at least one host is required"); + } + for (String remoteHost : remoteHosts) { + 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 for remote cluster [" + clusterName + "]"); + } + String host = remoteHost.substring(0, portSeparator); + try { + InetAddress.getByName(host); + } catch (UnknownHostException e) { + throw new IllegalArgumentException("unknown host [" + host + "]", e); + } + String port = remoteHost.substring(portSeparator + 1); + try { + Integer portValue = Integer.valueOf(port); + if (portValue <= 0) { + throw new IllegalArgumentException("port number must be > 0 but was: [" + portValue + "]"); + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("port must be a number, found [" + port + "] instead for remote cluster [" + + clusterName + "]"); + } + } + } + } + + /** + * Connects to all remote clusters in a blocking fashion. This should be called on node startup to establish an initial connection + * to all configured seed nodes. + */ + void initializeRemoteClusters() { + final TimeValue timeValue = REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING.get(settings); + final PlainActionFuture future = new PlainActionFuture<>(); + updateRemoteClusters(REMOTE_CLUSTERS_SEEDS.get(settings), future); + try { + future.get(timeValue.millis(), TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (TimeoutException ex) { + logger.warn("failed to connect to remote clusters within {}", timeValue.toString()); + } catch (Exception e) { + throw new IllegalStateException("failed to connect to remote clusters", e); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index ce5a5eec350..308b7e02c39 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -19,23 +19,17 @@ package org.elasticsearch.action.search; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.ClusterSettings; -import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.discovery.Discovery; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.dfs.DfsSearchResult; import org.elasticsearch.search.fetch.FetchSearchResult; @@ -55,29 +49,19 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportActionProxy; import org.elasticsearch.transport.TaskAwareTransportRequestHandler; import org.elasticsearch.transport.TransportChannel; -import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; /** * An encapsulation of {@link org.elasticsearch.search.SearchService} operations exposed through * transport. */ -public class SearchTransportService extends AbstractComponent { +public class SearchTransportService extends AbstractLifecycleComponent { public static final String FREE_CONTEXT_SCROLL_ACTION_NAME = "indices:data/read/search[free_context/scroll]"; public static final String FREE_CONTEXT_ACTION_NAME = "indices:data/read/search[free_context]"; @@ -92,166 +76,17 @@ public class SearchTransportService extends AbstractComponent { public static final String FETCH_ID_SCROLL_ACTION_NAME = "indices:data/read/search[phase/fetch/id/scroll]"; public static final String FETCH_ID_ACTION_NAME = "indices:data/read/search[phase/fetch/id]"; - //TODO what should the setting name be? - // TODO this should be an affix settings? - public static final Setting REMOTE_CLUSTERS_SEEDS = Setting.groupSetting("action.search.remote.", - SearchTransportService::validateRemoteClustersSeeds, - Setting.Property.NodeScope, - Setting.Property.Dynamic); - - /** - * The maximum number of connections that will be established to a remote cluster. For instance if there is only a single - * seed node, other nodes will be discovered up to the given number of nodes in this setting. The default is 3. - */ - public static final Setting NUM_REMOTE_CONNECTIONS = Setting.intSetting("action.search.num_remote_connections", - 3, 1, Setting.Property.NodeScope); - private final TransportService transportService; - private volatile Map remoteClusters = Collections.emptyMap(); + private final RemoteClusterService remoteClusterService; public SearchTransportService(Settings settings, ClusterSettings clusterSettings, TransportService transportService) { super(settings); this.transportService = transportService; - clusterSettings.addSettingsUpdateConsumer(REMOTE_CLUSTERS_SEEDS, this::setRemoteClusters, - SearchTransportService::validateRemoteClustersSeeds); - } - - public void setupRemoteClusters() { - setRemoteClusters(REMOTE_CLUSTERS_SEEDS.get(settings)); - } - - private static void validateRemoteClustersSeeds(Settings settings) { - //TODO do we need a static whitelist like in reindex from remote? - for (String clusterName : settings.names()) { - String[] remoteHosts = settings.getAsArray(clusterName); - if (remoteHosts.length == 0) { - throw new IllegalArgumentException("no hosts set for remote cluster [" + clusterName + "], at least one host is required"); - } - for (String remoteHost : remoteHosts) { - 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 for remote cluster [" + clusterName + "]"); - } - String host = remoteHost.substring(0, portSeparator); - try { - InetAddress.getByName(host); - } catch (UnknownHostException e) { - throw new IllegalArgumentException("unknown host [" + host + "]", e); - } - String port = remoteHost.substring(portSeparator + 1); - try { - Integer portValue = Integer.valueOf(port); - if (portValue <= 0) { - throw new IllegalArgumentException("port number must be > 0 but was: [" + portValue + "]"); - } - } catch(NumberFormatException e) { - throw new IllegalArgumentException("port must be a number, found [" + port + "] instead for remote cluster [" + - clusterName + "]"); - } - } - } - } - - static Map> buildRemoteClustersSeeds(Settings settings) { - Map> remoteClustersNodes = new HashMap<>(); - for (String clusterName : settings.names()) { - String[] remoteHosts = settings.getAsArray(clusterName); - for (String remoteHost : remoteHosts) { - int portSeparator = remoteHost.lastIndexOf(':'); // in case we have a IPv6 address ie. [::1]:9300 - String host = remoteHost.substring(0, portSeparator); - InetAddress hostAddress; - try { - hostAddress = InetAddress.getByName(host); - } catch (UnknownHostException e) { - throw new IllegalArgumentException("unknown host [" + host + "]", e); - } - int port = Integer.valueOf(remoteHost.substring(portSeparator + 1)); - DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, - new TransportAddress(new InetSocketAddress(hostAddress, port)), - Version.CURRENT.minimumCompatibilityVersion()); - //don't connect yet as that would require the remote node to be up and would fail the local node startup otherwise - List nodes = remoteClustersNodes.get(clusterName); - if (nodes == null) { - nodes = new ArrayList<>(); - remoteClustersNodes.put(clusterName, nodes); - } - nodes.add(node); - } - } - return remoteClustersNodes; - } - - private void setRemoteClusters(Settings settings) { - Map> seeds = buildRemoteClustersSeeds(settings); - Map remoteClusters = new HashMap<>(); - for (Map.Entry> entry : seeds.entrySet()) { - RemoteClusterConnection remote = this.remoteClusters.get(entry.getKey()); - if (remote == null) { - remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService, 3, - (node) -> Version.CURRENT.isCompatible(node.getVersion())); - remoteClusters.put(entry.getKey(), remote); - } - remote.updateSeedNodes(entry.getValue(), ActionListener.wrap((x) -> {}, - e -> logger.error("failed to update seed list for cluster: " + entry.getKey(), e) )); - } - if (remoteClusters.isEmpty() == false) { - remoteClusters.putAll(this.remoteClusters); - this.remoteClusters = Collections.unmodifiableMap(remoteClusters); - } - } - - boolean isCrossClusterSearchEnabled() { - return remoteClusters.isEmpty() == false; - } - - boolean isRemoteClusterRegistered(String clusterName) { - return remoteClusters.containsKey(clusterName); - } - - void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, - ActionListener> listener) { - final CountDown responsesCountDown = new CountDown(remoteIndicesByCluster.size()); - final Map searchShardsResponses = new ConcurrentHashMap<>(); - final AtomicReference transportException = new AtomicReference<>(); - for (Map.Entry> entry : remoteIndicesByCluster.entrySet()) { - final String clusterName = entry.getKey(); - RemoteClusterConnection remoteClusterConnection = remoteClusters.get(clusterName); - if (remoteClusterConnection == null) { - throw new IllegalArgumentException("no such remote cluster: " + clusterName); - } - final List indices = entry.getValue(); - remoteClusterConnection.fetchSearchShards(searchRequest, indices, - new ActionListener() { - @Override - public void onResponse(ClusterSearchShardsResponse clusterSearchShardsResponse) { - searchShardsResponses.put(clusterName, clusterSearchShardsResponse); - if (responsesCountDown.countDown()) { - TransportException exception = transportException.get(); - if (exception == null) { - listener.onResponse(searchShardsResponses); - } else { - listener.onFailure(transportException.get()); - } - } - } - - @Override - public void onFailure(Exception e) { - TransportException exception = new TransportException("unable to communicate with remote cluster [" + - clusterName + "]", e); - if (transportException.compareAndSet(null, exception) == false) { - exception = transportException.accumulateAndGet(exception, (previous, current) -> { - current.addSuppressed(previous); - return current; - }); - } - if (responsesCountDown.countDown()) { - listener.onFailure(exception); - } - } - }); - } + this.remoteClusterService = new RemoteClusterService(settings, transportService); + final Consumer clusterUpdateConsumer = (s) -> remoteClusterService.updateRemoteClusters(s, + ActionListener.wrap((x) -> {}, (x) -> {})); + clusterSettings.addSettingsUpdateConsumer(RemoteClusterService.REMOTE_CLUSTERS_SEEDS, clusterUpdateConsumer, + RemoteClusterService::validateRemoteClustersSeeds); } public void sendFreeContext(Transport.Connection connection, final long contextId, SearchRequest request) { @@ -337,6 +172,10 @@ public class SearchTransportService extends AbstractComponent { new ActionListenerResponseHandler<>(listener, FetchSearchResult::new)); } + public RemoteClusterService getRemoteClusterService() { + return remoteClusterService; + } + static class ScrollFreeContextRequest extends TransportRequest { private long id; @@ -558,11 +397,17 @@ public class SearchTransportService extends AbstractComponent { return transportService.getConnection(node); } - Transport.Connection getRemoteConnection(DiscoveryNode node, String cluster) { - RemoteClusterConnection connection = remoteClusters.get(cluster); - if (connection == null) { - throw new IllegalArgumentException("no such remote cluster: " + cluster); - } - return connection.getProxyConnection(node); + @Override + protected void doStart() { + // here we start to connect to the remote clusters + remoteClusterService.initializeRemoteClusters(); } + + @Override + protected void doStop() {} + + @Override + protected void doClose() throws IOException {} + + } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index ebb3e16d4e9..189b54a039d 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -74,6 +74,7 @@ public class TransportSearchAction extends HandledTransportAction> remoteIndicesByCluster = new HashMap<>(); - if (searchTransportService.isCrossClusterSearchEnabled()) { + if (remoteClusterService.isCrossClusterSearchEnabled()) { List localIndicesList = new ArrayList<>(); for (String index : searchRequest.indices()) { int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); if (i >= 0) { String remoteCluster = index.substring(0, i); - if (searchTransportService.isRemoteClusterRegistered(remoteCluster)) { + if (remoteClusterService.isRemoteClusterRegistered(remoteCluster)) { String remoteIndex = index.substring(i + 1); List indices = remoteIndicesByCluster.get(remoteCluster); if (indices == null) { @@ -163,11 +165,9 @@ public class TransportSearchAction extends HandledTransportAction null, Collections.emptyMap(), listener); } else { - // nocommit we have to extract this logic to add unittests ideally with manually prepared searchShardsResponses etc. - searchTransportService.sendSearchShards(searchRequest, remoteIndicesByCluster, + remoteClusterService.sendSearchShards(searchRequest, remoteIndicesByCluster, ActionListener.wrap((searchShardsResponses) -> { List remoteShardIterators = new ArrayList<>(); - Set remoteNodes = new HashSet<>(); Map remoteAliasFilters = new HashMap<>(); Function connectionFunction = processRemoteShards(searchShardsResponses, remoteShardIterators, remoteAliasFilters); @@ -185,7 +185,7 @@ public class TransportSearchAction extends HandledTransportAction searchTransportService.getRemoteConnection(remoteNode, clusterName)); + nodeToCluster.put(remoteNode.getId(), () -> remoteClusterService.getConnection(remoteNode, clusterName)); } Map indicesAndFilters = searchShardsResponse.getIndicesAndFilters(); for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 77adc984c06..a8b4fa54459 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -19,7 +19,7 @@ package org.elasticsearch.common.settings; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; -import org.elasticsearch.action.search.SearchTransportService; +import org.elasticsearch.action.search.RemoteClusterService; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.AutoCreateIndex; import org.elasticsearch.action.support.DestructiveOperations; @@ -254,8 +254,8 @@ public final class ClusterSettings extends AbstractScopedSettings { SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING, ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, - SearchTransportService.REMOTE_CLUSTERS_SEEDS, - SearchTransportService.NUM_REMOTE_CONNECTIONS, + RemoteClusterService.REMOTE_CLUSTERS_SEEDS, + RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER, TransportService.TRACE_LOG_EXCLUDE_SETTING, TransportService.TRACE_LOG_INCLUDE_SETTING, TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING, diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 94f0d37aa36..1ee95471a3e 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -596,8 +596,6 @@ public class Node implements Closeable { // start after cluster service so the local disco is known discovery.start(); transportService.acceptIncomingRequests(); - SearchTransportService searchTransportService = injector.getInstance(SearchTransportService.class); - searchTransportService.setupRemoteClusters(); discovery.startInitialJoin(); // tribe nodes don't have a master so we shouldn't register an observer s final TimeValue initialStateTimeout = DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings); @@ -640,6 +638,9 @@ public class Node implements Closeable { // start nodes now, after the http server, because it may take some time tribeService.startNodes(); + // starts connecting to remote clusters if any cluster is configured + SearchTransportService searchTransportService = injector.getInstance(SearchTransportService.class); + searchTransportService.start(); if (WRITE_PORTS_FIELD_SETTING.get(settings)) { if (NetworkModule.HTTP_ENABLED.get(settings)) { @@ -683,6 +684,7 @@ public class Node implements Closeable { injector.getInstance(GatewayService.class).stop(); injector.getInstance(SearchService.class).stop(); injector.getInstance(TransportService.class).stop(); + injector.getInstance(SearchTransportService.class).stop(); pluginLifecycleComponents.forEach(LifecycleComponent::stop); // we should stop this last since it waits for resources to get released @@ -744,6 +746,8 @@ public class Node implements Closeable { toClose.add(injector.getInstance(SearchService.class)); toClose.add(() -> stopWatch.stop().start("transport")); toClose.add(injector.getInstance(TransportService.class)); + toClose.add(() -> stopWatch.stop().start("search_transport_service")); + toClose.add(injector.getInstance(SearchTransportService.class)); for (LifecycleComponent plugin : pluginLifecycleComponents) { toClose.add(() -> stopWatch.stop().start("plugin(" + plugin.getClass().getName() + ")")); diff --git a/core/src/main/java/org/elasticsearch/transport/TransportService.java b/core/src/main/java/org/elasticsearch/transport/TransportService.java index a1494ea81cd..28527d938fb 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportService.java @@ -171,7 +171,7 @@ public class TransportService extends AbstractLifecycleComponent { } // for testing - DiscoveryNode getLocalNode() { + protected DiscoveryNode getLocalNode() { return localNode; } diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java new file mode 100644 index 00000000000..da3abe0f4a8 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java @@ -0,0 +1,330 @@ +/* + * 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.action.search; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +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.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportConnectionListener; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class RemoteClusterConnectionIT extends ESIntegTestCase { + + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Override + public void tearDown() throws Exception { + super.tearDown(); + ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); + } + + public MockTransportService startTransport(String id, List knownNodes) { + boolean success = false; + MockTransportService newService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null); + try { + newService.registerRequestHandler(ClusterSearchShardsAction.NAME, ClusterSearchShardsRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + channel.sendResponse(new ClusterSearchShardsResponse()); + }); + newService.registerRequestHandler(ClusterStateAction.NAME, ClusterStateRequest::new, ThreadPool.Names.SAME, + (request, channel) -> { + DiscoveryNodes.Builder builder = DiscoveryNodes.builder(); + for (DiscoveryNode node : knownNodes) { + builder.add(node); + } + ClusterState build = ClusterState.builder(ClusterName.DEFAULT).nodes(builder.build()).build(); + channel.sendResponse(new ClusterStateResponse(ClusterName.DEFAULT, build)); + }); + newService.start(); + newService.setLocalNode(new DiscoveryNode(id, newService.boundAddress().publishAddress(), Version.CURRENT)); + newService.acceptIncomingRequests(); + success = true; + return newService; + } finally { + if (success == false) { + newService.close(); + } + } + } + + public void testDiscoverSingleNode() throws Exception { + List knownNodes = new CopyOnWriteArrayList<>(); + try (MockTransportService seedTransport = startTransport("seed_node", knownNodes); + MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes)) { + DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); + DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(discoverableTransport.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(seedNode), service, Integer.MAX_VALUE, n -> true)) { + updateSeedNodes(connection, Arrays.asList(seedNode)); + assertTrue(service.nodeConnected(seedNode)); + assertTrue(service.nodeConnected(discoverableNode)); + } + } + } + } + + public void testNodeDisconnected() throws Exception { + List knownNodes = new CopyOnWriteArrayList<>(); + try (MockTransportService seedTransport = startTransport("seed_node", knownNodes); + MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes); + MockTransportService spareTransport = startTransport("spare_node", knownNodes)) { + DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); + DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode(); + DiscoveryNode spareNode = spareTransport.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(discoverableTransport.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(seedNode), service, Integer.MAX_VALUE, n -> true)) { + updateSeedNodes(connection, Arrays.asList(seedNode)); + assertTrue(service.nodeConnected(seedNode)); + assertTrue(service.nodeConnected(discoverableNode)); + assertFalse(service.nodeConnected(spareNode)); + knownNodes.add(spareNode); + CountDownLatch latchDisconnect = new CountDownLatch(1); + CountDownLatch latchConnected = new CountDownLatch(1); + service.addConnectionListener(new TransportConnectionListener() { + @Override + public void onNodeDisconnected(DiscoveryNode node) { + if (node.equals(discoverableNode)) { + latchDisconnect.countDown(); + } + } + + @Override + public void onNodeConnected(DiscoveryNode node) { + 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(service.nodeConnected(spareNode)); + } + } + } + } + + public void testFilterDiscoveredNodes() throws Exception { + List knownNodes = new CopyOnWriteArrayList<>(); + try (MockTransportService seedTransport = startTransport("seed_node", knownNodes); + MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes)) { + DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); + DiscoveryNode discoverableNode = discoverableTransport.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(discoverableTransport.getLocalDiscoNode()); + DiscoveryNode rejectedNode = randomBoolean() ? seedNode : discoverableNode; + + 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(seedNode), service, Integer.MAX_VALUE, n -> n.equals(rejectedNode) == false)) { + updateSeedNodes(connection, Arrays.asList(seedNode)); + if (rejectedNode.equals(seedNode)) { + assertFalse(service.nodeConnected(seedNode)); + assertTrue(service.nodeConnected(discoverableNode)); + } else { + assertTrue(service.nodeConnected(seedNode)); + assertFalse(service.nodeConnected(discoverableNode)); + } + } + } + } + } + + private void updateSeedNodes(RemoteClusterConnection connection, List seedNodes) 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(seedNodes, listener); + latch.await(); + if (exceptionAtomicReference.get() != null) { + throw exceptionAtomicReference.get(); + } + } + + + public void testConnect() throws InterruptedException, IOException { + ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); + ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getDataNodes(); + DiscoveryNode node = nodes.valuesIt().next(); + 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(node), + service, Integer.MAX_VALUE, n -> true)) { + CountDownLatch latch = new CountDownLatch(1); + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.updateSeedNodes(Arrays.asList(node), listener); + latch.await(); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = nodes::valuesIt; + for (DiscoveryNode dataNode : nodesIterable) { + assertTrue(service.nodeConnected(dataNode)); + } + assertNull(exceptionAtomicReference.get()); + } + } + } + + public void testConnectToSingleSeed() throws InterruptedException, IOException { + ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); + ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getNodes(); + DiscoveryNode node = nodes.valuesIt().next(); + 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(node), + service, 1, n -> true)) { + CountDownLatch latch = new CountDownLatch(1); + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.updateSeedNodes(Arrays.asList(node), listener); + latch.await(); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = nodes::valuesIt; + for (DiscoveryNode aNode : nodesIterable) { + if (aNode.equals(node)) { + assertTrue(service.nodeConnected(aNode)); + } else { + assertFalse(service.nodeConnected(aNode)); + } + } + assertNull(exceptionAtomicReference.get()); + } + } + } + + public void testFetchShards() throws Exception { + ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); + ImmutableOpenMap clusterNodes = clusterStateResponse.getState().getNodes().getNodes(); + DiscoveryNode node = clusterNodes.valuesIt().next(); + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { + service.start(); + service.acceptIncomingRequests(); + final boolean hasInitialNodes = randomBoolean(); + try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", + hasInitialNodes ? Arrays.asList(node) : Collections.emptyList(), service, Integer.MAX_VALUE, n -> true)) { + CountDownLatch latch = new CountDownLatch(1); + String newNode = null; + if (hasInitialNodes == false) { + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { + exceptionAtomicReference.set(x); + latch.countDown(); + }); + connection.updateSeedNodes(Arrays.asList(node), listener); + latch.await(); + + newNode = internalCluster().startDataOnlyNode(); + createIndex("test-index"); + assertTrue(service.nodeConnected(node)); + Iterable nodesIterable = clusterNodes::valuesIt; + for (DiscoveryNode dataNode : nodesIterable) { + if (dataNode.getName().equals(newNode)) { + assertFalse(service.nodeConnected(dataNode)); + } else { + assertTrue(service.nodeConnected(dataNode)); + } + } + assertNull(exceptionAtomicReference.get()); + } else { + createIndex("test-index"); + } + + SearchRequest request = new SearchRequest("test-index"); + CountDownLatch responseLatch = new CountDownLatch(1); + AtomicReference reference = new AtomicReference<>(); + AtomicReference failReference = new AtomicReference<>(); + + ActionListener shardsListener = ActionListener.wrap( + x -> { + reference.set(x); + responseLatch.countDown(); + }, + x -> { + failReference.set(x); + responseLatch.countDown(); + }); + connection.fetchSearchShards(request, Arrays.asList("test-index"), shardsListener); + responseLatch.await(); + assertNull(failReference.get()); + assertNotNull(reference.get()); + ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); + DiscoveryNode[] nodes = clusterSearchShardsResponse.getNodes(); + assertTrue(nodes.length != 0); + for (DiscoveryNode dataNode : nodes) { + assertNotNull(connection.getConnection(dataNode)); + if (dataNode.getName().equals(newNode)) { + assertFalse(service.nodeConnected(dataNode)); + } else { + assertTrue(service.nodeConnected(dataNode)); + } + } + } + } + } +} diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java deleted file mode 100644 index ce919bac11e..00000000000 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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.action.search; - -import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.transport.MockTransportService; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; - -import java.util.Arrays; -import java.util.Collections; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -public class RemoteClusterConnectionTests extends ESIntegTestCase { - - private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); - - @Override - public void tearDown() throws Exception { - super.tearDown(); - ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); - } - - public void testConnect() throws InterruptedException { - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getDataNodes(); - DiscoveryNode node = nodes.valuesIt().next(); - try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { - service.start(); - service.acceptIncomingRequests(); - RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service, - Integer.MAX_VALUE, n -> true); - CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.updateSeedNodes(Arrays.asList(node),listener); - latch.await(); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = nodes::valuesIt; - for (DiscoveryNode dataNode : nodesIterable) { - assertTrue(service.nodeConnected(dataNode)); - } - assertNull(exceptionAtomicReference.get()); - } - } - - public void testConnectToSingleSeed() throws InterruptedException { - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getNodes(); - DiscoveryNode node = nodes.valuesIt().next(); - try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { - service.start(); - service.acceptIncomingRequests(); - RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(node), service, - 1, n -> true); - CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.updateSeedNodes(Arrays.asList(node),listener); - latch.await(); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = nodes::valuesIt; - for (DiscoveryNode aNode : nodesIterable) { - if (aNode.equals(node)) { - assertTrue(service.nodeConnected(aNode)); - } else { - assertFalse(service.nodeConnected(aNode)); - } - } - assertNull(exceptionAtomicReference.get()); - } - } - - public void testFetchShards() throws InterruptedException { - - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap dataNodes = clusterStateResponse.getState().getNodes().getDataNodes(); - DiscoveryNode node = dataNodes.valuesIt().next(); - try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { - service.start(); - service.acceptIncomingRequests(); - final boolean hasInitialNodes = randomBoolean(); - RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", - hasInitialNodes ? Arrays.asList(node) : Collections.emptyList(), service, Integer.MAX_VALUE, n -> true); - CountDownLatch latch = new CountDownLatch(1); - - if (hasInitialNodes == false) { - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.updateSeedNodes(Arrays.asList(node), listener); - latch.await(); - - String newNode = internalCluster().startDataOnlyNode(); - createIndex("test-index"); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = dataNodes::valuesIt; - for (DiscoveryNode dataNode : nodesIterable) { - if (dataNode.getName().equals(newNode)) { - assertFalse(service.nodeConnected(dataNode)); - } else { - assertTrue(service.nodeConnected(dataNode)); - } - } - assertNull(exceptionAtomicReference.get()); - } else { - createIndex("test-index"); - } - - SearchRequest request = new SearchRequest("test-index"); - CountDownLatch responseLatch = new CountDownLatch(1); - AtomicReference reference = new AtomicReference<>(); - AtomicReference failReference = new AtomicReference<>(); - - ActionListener shardsListener = ActionListener.wrap( - x -> {reference.set(x); responseLatch.countDown();}, - x -> {failReference.set(x); responseLatch.countDown();}); - connection.fetchSearchShards(request, Arrays.asList("test-index"), shardsListener); - responseLatch.await(); - assertNull(failReference.get()); - assertNotNull(reference.get()); - ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); - DiscoveryNode[] nodes = clusterSearchShardsResponse.getNodes(); - assertTrue(nodes.length != 0); - for (DiscoveryNode dataNode : nodes) { - assertTrue(service.nodeConnected(dataNode)); - } - } - } -} diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java similarity index 75% rename from core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java rename to core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index def29de9688..6e29c720893 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchTransportServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -29,24 +29,24 @@ import java.net.InetSocketAddress; import java.util.List; import java.util.Map; -public class SearchTransportServiceTests extends ESTestCase { +public class RemoteClusterServiceTests extends ESTestCase { public void testRemoteClusterSeedSetting() { // simple validation - SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() - .put("action.search.remote.foo", "192.168.0.1:8080") - .put("action.search.remote.bar", "[::1]:9090").build()); + RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() + .put("search.remote.seeds.foo", "192.168.0.1:8080") + .put("search.remote.seeds.bar", "[::1]:9090").build()); expectThrows(IllegalArgumentException.class, () -> - SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() - .put("action.search.remote.foo", "192.168.0.1").build())); + RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() + .put("search.remote.seeds.foo", "192.168.0.1").build())); } public void testBuiltRemoteClustersSeeds() throws Exception { - Map> map = SearchTransportService.buildRemoteClustersSeeds( - SearchTransportService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() - .put("action.search.remote.foo", "192.168.0.1:8080") - .put("action.search.remote.bar", "[::1]:9090").build())); + Map> map = RemoteClusterService.buildRemoteClustersSeeds( + RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() + .put("search.remote.seeds.foo", "192.168.0.1:8080") + .put("search.remote.seeds.bar", "[::1]:9090").build())); assertEquals(2, map.size()); assertTrue(map.containsKey("foo")); assertTrue(map.containsKey("bar")); diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index 60ebe6f0ae2..5db18833af8 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -82,7 +82,7 @@ public class SearchAsyncActionTests extends ESTestCase { GroupShardsIterator shardsIter = getShardsIter("idx", randomIntBetween(1, 10), randomBoolean(), primaryNode, replicaNode); AtomicInteger numFreedContext = new AtomicInteger(); SearchTransportService transportService = new SearchTransportService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, - Collections.singleton(SearchTransportService.REMOTE_CLUSTERS_SEEDS)), null) { + Collections.singleton(RemoteClusterService.REMOTE_CLUSTERS_SEEDS)), null) { @Override public void sendFreeContext(Transport.Connection connection, long contextId, SearchRequest request) { numFreedContext.incrementAndGet(); diff --git a/qa/multi-cluster-search/build.gradle b/qa/multi-cluster-search/build.gradle index 68e6cf414c0..1abe6e984c3 100644 --- a/qa/multi-cluster-search/build.gradle +++ b/qa/multi-cluster-search/build.gradle @@ -35,8 +35,8 @@ task mixedClusterTest(type: RestIntegTestTask) { dependsOn(remoteClusterTest) cluster { distribution = 'zip' - setting 'action.search.remote.my_remote_cluster', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" - setting 'action.search.num_remote_connections', 1 + setting 'search.remote.seeds.my_remote_cluster', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" + setting 'search.remote.connections_per_cluster', 1 } systemProperty 'tests.rest.suite', 'multi_cluster' diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index 9166508db92..586b3764ea8 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -99,16 +99,16 @@ cluster.get_settings: include_defaults: true - - set: { defaults.action.search.remote.my_remote_cluster: remote_ip } + - set: { defaults.search.remote.seeds.my_remote_cluster: remote_ip } - do: cluster.put_settings: flat_settings: true body: transient: - action.search.remote.test_remote_cluster: $remote_ip + search.remote.seeds.test_remote_cluster: $remote_ip - - match: {transient: {action.search.remote.test_remote_cluster: $remote_ip}} + - match: {transient: {search.remote.seeds.test_remote_cluster: $remote_ip}} - do: search: diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 9f59508bca4..c433c6d2abc 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -2285,7 +2285,7 @@ public abstract class ESIntegTestCase extends ESTestCase { .put(Environment.PATH_HOME_SETTING.getKey(), tempDir) .put(Environment.PATH_REPO_SETTING.getKey(), tempDir.resolve("repo")) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), createTempDir().getParent()) - .put("action.search.remote.test_remote_cluster", seedNode.getAddress().toString()) + .put("search.remote.seeds.test_remote_cluster", seedNode.getAddress().toString()) .put("node.name", "node_prx_0") .put(EsExecutors.PROCESSORS_SETTING.getKey(), 1) // limit the number of threads created .put(NetworkModule.HTTP_ENABLED.getKey(), false) 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 a0344a6f86d..e04d3963bf4 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 @@ -740,4 +740,8 @@ public final class MockTransportService extends TransportService { super.doClose(); assert openConnections.size() == 0 : "still open connections: " + openConnections; } + + public DiscoveryNode getLocalDiscoNode() { + return this.getLocalNode(); + } } From daf1f53c39efa59252b64485f2346787209d3660 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 4 Jan 2017 21:36:53 +0100 Subject: [PATCH 25/59] Make RemoteClusterConnectionIT a unit test --- .../shards/ClusterSearchShardsResponse.java | 2 +- .../search/RemoteClusterConnection.java | 181 +++++++------- .../elasticsearch/transport/TcpTransport.java | 20 +- ...java => RemoteClusterConnectionTests.java} | 230 ++++++++---------- 4 files changed, 215 insertions(+), 218 deletions(-) rename core/src/test/java/org/elasticsearch/action/search/{RemoteClusterConnectionIT.java => RemoteClusterConnectionTests.java} (58%) diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java index 6f9a2ae55b1..8d97351e361 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java @@ -42,7 +42,7 @@ public class ClusterSearchShardsResponse extends ActionResponse implements ToXCo } - ClusterSearchShardsResponse(ClusterSearchShardsGroup[] groups, DiscoveryNode[] nodes, Map indicesAndFilters) { + public ClusterSearchShardsResponse(ClusterSearchShardsGroup[] groups, DiscoveryNode[] nodes, Map indicesAndFilters) { this.groups = groups; this.nodes = nodes; this.indicesAndFilters = indicesAndFilters; diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index e9065a110f2..95659c8f404 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -143,7 +143,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo boolean remove = connectedNodes.remove(node); if (remove == true && connectedNodes.size() < maxNumRemoteConnections) { // try to reconnect and fill up the slot of the disconnected node - connectHandler.maybeConnect(); + connectHandler.forceConnect(); } } @@ -235,23 +235,32 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo connectHandler.close(); } + // TODO document this class private class ConnectHandler implements Closeable { private final Semaphore running = new Semaphore(1); private final AtomicBoolean closed = new AtomicBoolean(false); private final BlockingQueue> queue = new ArrayBlockingQueue<>(100); - public void maybeConnect() { + void maybeConnect() { connect(null); } - public void connect(ActionListener connectListener) { + void connect(ActionListener connectListener) { + connect(connectListener, false); + } + + void forceConnect() { + connect(null, true); + } + + private void connect(ActionListener connectListener, boolean forceRun) { final boolean runConnect; final Collection> toNotify; synchronized (queue) { if (connectListener != null && queue.offer(connectListener) == false) { throw new IllegalStateException("connect queue is full"); } - if (connectListener != null && queue.isEmpty()) { + if (forceRun == false && queue.isEmpty()) { return; } runConnect = running.tryAcquire(); @@ -261,8 +270,9 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo queue.drainTo(toNotify); if (closed.get()) { for (ActionListener listener : toNotify) { - listener.onFailure(new AlreadyClosedException("connecte handler is already closed")); + listener.onFailure(new AlreadyClosedException("connect handler is already closed")); } + return; } } else { toNotify = Collections.emptyList(); @@ -316,93 +326,100 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo void collectRemoteNodes(Iterator seedNodes, final TransportService transportService, ActionListener listener) { - try { - if (seedNodes.hasNext()) { - final DiscoveryNode seedNode = seedNodes.next(); - final DiscoveryNode handshakeNode; - Transport.Connection connection = transportService.openConnection(seedNode, - ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null,null)); - boolean success = false; - try { - handshakeNode = transportService.handshake(connection, remoteProfile.getHandshakeTimeout().millis(), - (c) -> true); - if (nodePredicate.test(handshakeNode) && connectedNodes.size() < maxNumRemoteConnections) { - transportService.connectToNode(handshakeNode, remoteProfile); - connectedNodes.add(handshakeNode); - } - ClusterStateRequest request = new ClusterStateRequest(); - request.clear(); - request.nodes(true); - transportService.sendRequest(connection, - ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, - new TransportResponseHandler() { + try { + if (seedNodes.hasNext()) { + final DiscoveryNode seedNode = seedNodes.next(); + final DiscoveryNode handshakeNode; + Transport.Connection connection = transportService.openConnection(seedNode, + ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null,null)); + boolean success = false; + try { + handshakeNode = transportService.handshake(connection, remoteProfile.getHandshakeTimeout().millis(), + (c) -> true); + if (nodePredicate.test(handshakeNode) && connectedNodes.size() < maxNumRemoteConnections) { + transportService.connectToNode(handshakeNode, remoteProfile); + connectedNodes.add(handshakeNode); + } + ClusterStateRequest request = new ClusterStateRequest(); + request.clear(); + request.nodes(true); + transportService.sendRequest(connection, + ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, + new TransportResponseHandler() { - @Override - public ClusterStateResponse newInstance() { - return new ClusterStateResponse(); - } - - @Override - public void handleResponse(ClusterStateResponse response) { - try { - DiscoveryNodes nodes = response.getState().nodes(); - Iterable nodesIter = nodes.getNodes()::valuesIt; - for (DiscoveryNode node : nodesIter) { - if (nodePredicate.test(node) && connectedNodes.size() < maxNumRemoteConnections) { - try { - transportService.connectToNode(node, remoteProfile); // noop if node is connected - connectedNodes.add(node); - } catch (ConnectTransportException ex) { - // fair enough we can't connect just move on - logger.debug((Supplier) - () -> new ParameterizedMessage("failed to connect to node {}", node), ex); - } + @Override + public ClusterStateResponse newInstance() { + return new ClusterStateResponse(); + } + @Override + public void handleResponse(ClusterStateResponse response) { + try { + DiscoveryNodes nodes = response.getState().nodes(); + Iterable nodesIter = nodes.getNodes()::valuesIt; + for (DiscoveryNode node : nodesIter) { + if (nodePredicate.test(node) && connectedNodes.size() < maxNumRemoteConnections) { + try { + transportService.connectToNode(node, remoteProfile); // noop if node is connected + connectedNodes.add(node); + } catch (ConnectTransportException | IllegalStateException ex) { + // ISE if we fail the handshake with an version incompatible node + // fair enough we can't connect just move on + logger.debug((Supplier) + () -> new ParameterizedMessage("failed to connect to node {}", node), ex); } + } - listener.onResponse(null); - - } catch (Exception ex) { - logger.warn((Supplier) - () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), ex); - collectRemoteNodes(seedNodes, transportService, listener); - } finally { - IOUtils.closeWhileHandlingException(connection); } - } - - @Override - public void handleException(TransportException exp) { + connection.close(); + listener.onResponse(null); + } catch (Exception ex) { logger.warn((Supplier) - () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), exp); + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), ex); collectRemoteNodes(seedNodes, transportService, listener); + } finally { IOUtils.closeWhileHandlingException(connection); } + } - @Override - public String executor() { - return ThreadPool.Names.MANAGEMENT; + @Override + public void handleException(TransportException exp) { + logger.warn((Supplier) + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), + exp); + try { + IOUtils.closeWhileHandlingException(connection); + } finally { + collectRemoteNodes(seedNodes, transportService, listener); } - }); - success = true; - } finally { - if (success == false) { - connection.close(); - } - } + } - } else { - listener.onFailure(new IllegalStateException("no seed node left")); - } - } catch (IOException ex) { - if (seedNodes.hasNext()) { - logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), ex); - collectRemoteNodes(seedNodes, transportService, listener); - } else { - listener.onFailure(ex); + @Override + public String executor() { + return ThreadPool.Names.MANAGEMENT; + } + }); + success = true; + } finally { + if (success == false) { + connection.close(); + } } + + } else { + listener.onFailure(new IllegalStateException("no seed node left")); } + } catch (ConnectTransportException | IOException | IllegalStateException ex) { + // ISE if we fail the handshake with an version incompatible node + if (seedNodes.hasNext()) { + logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), ex); + collectRemoteNodes(seedNodes, transportService, listener); + } else { + listener.onFailure(ex); + } + } } @Override @@ -414,11 +431,11 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo } catch (InterruptedException e) { Thread.currentThread().interrupt(); } - synchronized (queue) { - - running.release(); - } } } + boolean isConnectRunning() { // for testing only + return connectHandler.running.availablePermits() == 0; + } + } diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index 2e8cb4f65ce..07aa8f62d1b 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -458,13 +458,6 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i "failed to connect to [{}], cleaning dangling connections", node), e); throw e; } - Channel channel = nodeChannels.channel(TransportRequestOptions.Type.PING); - final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ? - defaultConnectionProfile.getConnectTimeout() : - connectionProfile.getConnectTimeout(); - final TimeValue handshakeTimeout = connectionProfile.getHandshakeTimeout() == null ? - connectTimeout : connectionProfile.getHandshakeTimeout(); - Version version = executeHandshake(node, channel, handshakeTimeout); // we acquire a connection lock, so no way there is an existing connection connectedNodes.put(node, nodeChannels); if (logger.isDebugEnabled()) { @@ -483,11 +476,18 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i } @Override - public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile profile) throws IOException { + public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) throws IOException { try { - NodeChannels nodeChannels = connectToChannels(node, profile); + NodeChannels nodeChannels = connectToChannels(node, connectionProfile); + Channel channel = nodeChannels.getChannels().get(0); + final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ? + defaultConnectionProfile.getConnectTimeout() : + connectionProfile.getConnectTimeout(); + final TimeValue handshakeTimeout = connectionProfile.getHandshakeTimeout() == null ? + connectTimeout : connectionProfile.getHandshakeTimeout(); + Version version = executeHandshake(node, channel, handshakeTimeout); transportServiceAdapter.onConnectionOpened(node); - return nodeChannels; + return new NodeChannels(nodeChannels, version); } catch (ConnectTransportException e) { throw e; } catch (Exception e) { diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java similarity index 58% rename from core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java rename to core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index da3abe0f4a8..94a9778bdb9 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionIT.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -21,6 +21,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; @@ -30,15 +31,13 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportConnectionListener; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -47,7 +46,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -public class RemoteClusterConnectionIT extends ESIntegTestCase { +public class RemoteClusterConnectionTests extends ESTestCase { private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); @@ -57,13 +56,14 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); } - public MockTransportService startTransport(String id, List knownNodes) { + public MockTransportService startTransport(String id, List knownNodes, Version version) { boolean success = false; - MockTransportService newService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null); + MockTransportService newService = MockTransportService.createNewService(Settings.EMPTY, version, threadPool, null); try { newService.registerRequestHandler(ClusterSearchShardsAction.NAME, ClusterSearchShardsRequest::new, ThreadPool.Names.SAME, (request, channel) -> { - channel.sendResponse(new ClusterSearchShardsResponse()); + channel.sendResponse(new ClusterSearchShardsResponse(new ClusterSearchShardsGroup[0], + knownNodes.toArray(new DiscoveryNode[0]), Collections.emptyMap())); }); newService.registerRequestHandler(ClusterStateAction.NAME, ClusterStateRequest::new, ThreadPool.Names.SAME, (request, channel) -> { @@ -75,7 +75,7 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { channel.sendResponse(new ClusterStateResponse(ClusterName.DEFAULT, build)); }); newService.start(); - newService.setLocalNode(new DiscoveryNode(id, newService.boundAddress().publishAddress(), Version.CURRENT)); + newService.setLocalNode(new DiscoveryNode(id, newService.boundAddress().publishAddress(), version)); newService.acceptIncomingRequests(); success = true; return newService; @@ -88,12 +88,13 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { public void testDiscoverSingleNode() throws Exception { List knownNodes = new CopyOnWriteArrayList<>(); - try (MockTransportService seedTransport = startTransport("seed_node", knownNodes); - MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes)) { + 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(); @@ -103,6 +104,37 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { updateSeedNodes(connection, Arrays.asList(seedNode)); assertTrue(service.nodeConnected(seedNode)); assertTrue(service.nodeConnected(discoverableNode)); + assertFalse(connection.isConnectRunning()); + } + } + } + } + + public void testDiscoverSingleNodeWithIncompatibleSeed() throws Exception { + List knownNodes = new CopyOnWriteArrayList<>(); + try (MockTransportService seedTransport = startTransport("seed_node", knownNodes, Version.CURRENT); + MockTransportService incomaptibleTransport = 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 = incomaptibleTransport.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(discoverableTransport.getLocalDiscoNode()); + knownNodes.add(incomaptibleTransport.getLocalDiscoNode()); + Collections.shuffle(knownNodes, random()); + List seedNodes = Arrays.asList(incompatibleSeedNode, 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)) { + updateSeedNodes(connection, seedNodes); + assertTrue(service.nodeConnected(seedNode)); + assertTrue(service.nodeConnected(discoverableNode)); + assertFalse(service.nodeConnected(incompatibleSeedNode)); + assertFalse(connection.isConnectRunning()); } } } @@ -110,14 +142,15 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { public void testNodeDisconnected() throws Exception { List knownNodes = new CopyOnWriteArrayList<>(); - try (MockTransportService seedTransport = startTransport("seed_node", knownNodes); - MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes); - MockTransportService spareTransport = startTransport("spare_node", knownNodes)) { + 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(); @@ -159,13 +192,14 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { public void testFilterDiscoveredNodes() throws Exception { List knownNodes = new CopyOnWriteArrayList<>(); - try (MockTransportService seedTransport = startTransport("seed_node", knownNodes); - MockTransportService discoverableTransport = startTransport("discoverable_node", knownNodes)) { + 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(); @@ -180,6 +214,7 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { assertTrue(service.nodeConnected(seedNode)); assertFalse(service.nodeConnected(discoverableNode)); } + assertFalse(connection.isConnectRunning()); } } } @@ -199,130 +234,75 @@ public class RemoteClusterConnectionIT extends ESIntegTestCase { } } + 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()); - public void testConnect() throws InterruptedException, IOException { - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getDataNodes(); - DiscoveryNode node = nodes.valuesIt().next(); - 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(node), - service, Integer.MAX_VALUE, n -> true)) { - CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.updateSeedNodes(Arrays.asList(node), listener); - latch.await(); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = nodes::valuesIt; - for (DiscoveryNode dataNode : nodesIterable) { - assertTrue(service.nodeConnected(dataNode)); + 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(seedNode), service, Integer.MAX_VALUE, n -> true)) { + expectThrows(Exception.class, () -> updateSeedNodes(connection, Arrays.asList(seedNode))); + assertFalse(service.nodeConnected(seedNode)); + assertFalse(connection.isConnectRunning()); } - assertNull(exceptionAtomicReference.get()); - } - } - } - - public void testConnectToSingleSeed() throws InterruptedException, IOException { - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap nodes = clusterStateResponse.getState().getNodes().getNodes(); - DiscoveryNode node = nodes.valuesIt().next(); - 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(node), - service, 1, n -> true)) { - CountDownLatch latch = new CountDownLatch(1); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.updateSeedNodes(Arrays.asList(node), listener); - latch.await(); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = nodes::valuesIt; - for (DiscoveryNode aNode : nodesIterable) { - if (aNode.equals(node)) { - assertTrue(service.nodeConnected(aNode)); - } else { - assertFalse(service.nodeConnected(aNode)); - } - } - assertNull(exceptionAtomicReference.get()); } } } public void testFetchShards() throws Exception { - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().clear().setNodes(true).get(); - ImmutableOpenMap clusterNodes = clusterStateResponse.getState().getNodes().getNodes(); - DiscoveryNode node = clusterNodes.valuesIt().next(); - try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { - service.start(); - service.acceptIncomingRequests(); - final boolean hasInitialNodes = randomBoolean(); - try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", - hasInitialNodes ? Arrays.asList(node) : Collections.emptyList(), service, Integer.MAX_VALUE, n -> true)) { - CountDownLatch latch = new CountDownLatch(1); - String newNode = null; - if (hasInitialNodes == false) { - AtomicReference exceptionAtomicReference = new AtomicReference<>(); - ActionListener listener = ActionListener.wrap(x -> latch.countDown(), x -> { - exceptionAtomicReference.set(x); - latch.countDown(); - }); - connection.updateSeedNodes(Arrays.asList(node), listener); - latch.await(); + List knownNodes = new CopyOnWriteArrayList<>(); - newNode = internalCluster().startDataOnlyNode(); - createIndex("test-index"); - assertTrue(service.nodeConnected(node)); - Iterable nodesIterable = clusterNodes::valuesIt; - for (DiscoveryNode dataNode : nodesIterable) { + 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(); + try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", + Arrays.asList(seedNode), service, Integer.MAX_VALUE, n -> true)) { + CountDownLatch latch = new CountDownLatch(1); + String newNode = null; + if (randomBoolean()) { + updateSeedNodes(connection, Arrays.asList(seedNode)); + } + + SearchRequest request = new SearchRequest("test-index"); + CountDownLatch responseLatch = new CountDownLatch(1); + AtomicReference reference = new AtomicReference<>(); + AtomicReference failReference = new AtomicReference<>(); + + ActionListener shardsListener = ActionListener.wrap( + x -> { + reference.set(x); + responseLatch.countDown(); + }, + x -> { + failReference.set(x); + responseLatch.countDown(); + }); + connection.fetchSearchShards(request, Arrays.asList("test-index"), shardsListener); + responseLatch.await(); + assertNull(failReference.get()); + assertNotNull(reference.get()); + ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); + DiscoveryNode[] nodes = clusterSearchShardsResponse.getNodes(); + assertTrue(nodes.length != 0); + for (DiscoveryNode dataNode : nodes) { + assertNotNull(connection.getConnection(dataNode)); if (dataNode.getName().equals(newNode)) { assertFalse(service.nodeConnected(dataNode)); } else { assertTrue(service.nodeConnected(dataNode)); } } - assertNull(exceptionAtomicReference.get()); - } else { - createIndex("test-index"); - } - - SearchRequest request = new SearchRequest("test-index"); - CountDownLatch responseLatch = new CountDownLatch(1); - AtomicReference reference = new AtomicReference<>(); - AtomicReference failReference = new AtomicReference<>(); - - ActionListener shardsListener = ActionListener.wrap( - x -> { - reference.set(x); - responseLatch.countDown(); - }, - x -> { - failReference.set(x); - responseLatch.countDown(); - }); - connection.fetchSearchShards(request, Arrays.asList("test-index"), shardsListener); - responseLatch.await(); - assertNull(failReference.get()); - assertNotNull(reference.get()); - ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); - DiscoveryNode[] nodes = clusterSearchShardsResponse.getNodes(); - assertTrue(nodes.length != 0); - for (DiscoveryNode dataNode : nodes) { - assertNotNull(connection.getConnection(dataNode)); - if (dataNode.getName().equals(newNode)) { - assertFalse(service.nodeConnected(dataNode)); - } else { - assertTrue(service.nodeConnected(dataNode)); - } + assertFalse(connection.isConnectRunning()); } } } From 7b95c2f54cbe9b2f985e703cdd70775fbb591a38 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 5 Jan 2017 14:07:56 +0100 Subject: [PATCH 26/59] document and test concurrent remote cluster node discovery --- .../elasticsearch/action/ActionListener.java | 49 ++++ .../search/RemoteClusterConnection.java | 275 +++++++++++------- .../action/search/RemoteClusterService.java | 10 +- .../action/search/SearchTransportService.java | 4 +- .../search/RemoteClusterConnectionTests.java | 213 ++++++++++++-- 5 files changed, 422 insertions(+), 129 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/ActionListener.java b/core/src/main/java/org/elasticsearch/action/ActionListener.java index ef26867600e..19d33707ed7 100644 --- a/core/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/core/src/main/java/org/elasticsearch/action/ActionListener.java @@ -65,4 +65,53 @@ public interface ActionListener { } }; } + + /** + * Notifies every given listener with the response passed to {@link #onResponse(Object)}. If a listener itself throws an exception + * the exception is forwarded to {@link #onFailure(Exception)}. If in turn {@link #onFailure(Exception)} fails all remaining + * listeners will be processed and the caught exception will be re-thrown. + */ + static void onResponse(Iterable> listeners, Response response) { + RuntimeException exception = null; + for (ActionListener listener : listeners) { + try { + listener.onResponse(response); + } catch (Exception ex) { + try { + listener.onFailure(ex); + } catch (Exception ex1) { + if (exception != null) { + exception = new RuntimeException(ex1); + } else { + exception.addSuppressed(ex1); + } + } + } + } + if (exception != null) { + throw exception; + } + } + + /** + * Notifies every given listener with the failure passed to {@link #onFailure(Exception)}. If a listener itself throws an exception + * all remaining listeners will be processed and the caught exception will be re-thrown. + */ + static void onFailure(Iterable> listeners, Exception failure) { + RuntimeException exception = null; + for (ActionListener listener : listeners) { + try { + listener.onFailure(failure); + } catch (Exception ex) { + if (exception != null) { + exception = new RuntimeException(ex); + } else { + exception.addSuppressed(ex); + } + } + } + if (exception != null) { + throw exception; + } + } } diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 95659c8f404..e4549f1b529 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -32,7 +32,10 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; @@ -58,6 +61,7 @@ import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.Semaphore; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Predicate; @@ -65,7 +69,7 @@ import java.util.function.Predicate; /** * Represents a connection to a single remote cluster. In contrast to a local cluster a remote cluster is not joined such that the * current node is part of the cluster and it won't receive cluster state updated from the remote cluster. Remote clusters are also not - * fully connected with the current node. From a connectin perspective a local cluster forms a bi-directional star network while in the + * fully connected with the current node. From a connection perspective a local cluster forms a bi-directional star network while in the * remote case we only connect to a subset of the nodes in the cluster in an uni-directional fashion. * * This class also handles the discovery of nodes from the remote cluster. The initial list of seed nodes is only used to discover all nodes @@ -94,7 +98,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo * @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 eligable remote nodes to connect to + * @param nodePredicate a predicate to filter eligible remote nodes to connect to */ RemoteClusterConnection(Settings settings, String clusterName, List seedNodes, TransportService transportService, int maxNumRemoteConnections, Predicate nodePredicate) { @@ -235,20 +239,41 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo connectHandler.close(); } - // TODO document this class + /** + * 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 once the current running job returns. + * + * The handler has ab built-in queue that can hold up to 100 connect attempts and will reject request once the queue is full. + * In a scenario when a remote cluster becomes unavailable we will queue up immediate request 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 final Semaphore running = new Semaphore(1); private final AtomicBoolean closed = new AtomicBoolean(false); private final BlockingQueue> queue = new ArrayBlockingQueue<>(100); + private final CancellableThreads cancellableThreads = new CancellableThreads(); + /** + * Triggers a connect round iff there are pending requests queued up and if there is no + * connect round currently running. + */ void maybeConnect() { connect(null); } + /** + * 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) { connect(connectListener, false); } + /** + * Triggers a connect round unless there is one already running. In contrast to {@link #maybeConnect()} will this method also + * trigger a connect round if there is no listener queued up. + */ void forceConnect() { connect(null, true); } @@ -258,20 +283,19 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo final Collection> toNotify; synchronized (queue) { if (connectListener != null && queue.offer(connectListener) == false) { - throw new IllegalStateException("connect queue is full"); + connectListener.onFailure(new RejectedExecutionException("connect queue is full")); + return; } if (forceRun == false && queue.isEmpty()) { return; } runConnect = running.tryAcquire(); - if (runConnect) { toNotify = new ArrayList<>(); queue.drainTo(toNotify); if (closed.get()) { - for (ActionListener listener : toNotify) { - listener.onFailure(new AlreadyClosedException("connect handler is already closed")); - } + running.release(); + ActionListener.onFailure(toNotify, new AlreadyClosedException("connect handler is already closed")); return; } } else { @@ -292,33 +316,36 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo synchronized (queue) { running.release(); } - for (ActionListener queuedListener : toNotify) { - queuedListener.onFailure(e); + try { + ActionListener.onFailure(toNotify, e); + } finally { + maybeConnect(); } } @Override protected void doRun() throws Exception { ActionListener listener = ActionListener.wrap((x) -> { - synchronized (queue) { - running.release(); - } - for (ActionListener queuedListener : toNotify) { - queuedListener.onResponse(x); - } + synchronized (queue) { + running.release(); + } + try { + ActionListener.onResponse(toNotify, x); + } finally { maybeConnect(); - }, - (e) -> { - synchronized (queue) { - running.release(); - } - for (ActionListener queuedListener : toNotify) { - queuedListener.onFailure(e); - } + } + + }, (e) -> { + synchronized (queue) { + running.release(); + } + try { + ActionListener.onFailure(toNotify, e); + } finally { maybeConnect(); - }); - Iterator iterator = Collections.synchronizedList(seedNodes).iterator(); - collectRemoteNodes(iterator, transportService, listener); + } + }); + collectRemoteNodes(seedNodes.iterator(), transportService, listener); } }); @@ -328,88 +355,37 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo final TransportService transportService, ActionListener listener) { try { if (seedNodes.hasNext()) { - final DiscoveryNode seedNode = seedNodes.next(); - final DiscoveryNode handshakeNode; - Transport.Connection connection = transportService.openConnection(seedNode, - ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null,null)); - boolean success = false; - try { - handshakeNode = transportService.handshake(connection, remoteProfile.getHandshakeTimeout().millis(), - (c) -> true); - if (nodePredicate.test(handshakeNode) && connectedNodes.size() < maxNumRemoteConnections) { - transportService.connectToNode(handshakeNode, remoteProfile); - connectedNodes.add(handshakeNode); + cancellableThreads.executeIO(() -> { + final DiscoveryNode seedNode = seedNodes.next(); + final DiscoveryNode handshakeNode; + Transport.Connection connection = transportService.openConnection(seedNode, + ConnectionProfile.buildSingleChannelProfile(TransportRequestOptions.Type.REG, null, null)); + boolean success = false; + try { + handshakeNode = transportService.handshake(connection, remoteProfile.getHandshakeTimeout().millis(), + (c) -> true); + if (nodePredicate.test(handshakeNode) && connectedNodes.size() < maxNumRemoteConnections) { + transportService.connectToNode(handshakeNode, remoteProfile); + connectedNodes.add(handshakeNode); + } + ClusterStateRequest request = new ClusterStateRequest(); + request.clear(); + request.nodes(true); + transportService.sendRequest(connection, + ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, + new StateResponseHandler(transportService, connection, listener, seedNodes, cancellableThreads)); + success = true; + } finally { + if (success == false) { + connection.close(); + } } - ClusterStateRequest request = new ClusterStateRequest(); - request.clear(); - request.nodes(true); - transportService.sendRequest(connection, - ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, - new TransportResponseHandler() { - - @Override - public ClusterStateResponse newInstance() { - return new ClusterStateResponse(); - } - - @Override - public void handleResponse(ClusterStateResponse response) { - try { - DiscoveryNodes nodes = response.getState().nodes(); - Iterable nodesIter = nodes.getNodes()::valuesIt; - for (DiscoveryNode node : nodesIter) { - if (nodePredicate.test(node) && connectedNodes.size() < maxNumRemoteConnections) { - try { - transportService.connectToNode(node, remoteProfile); // noop if node is connected - connectedNodes.add(node); - } catch (ConnectTransportException | IllegalStateException ex) { - // ISE if we fail the handshake with an version incompatible node - // fair enough we can't connect just move on - logger.debug((Supplier) - () -> new ParameterizedMessage("failed to connect to node {}", node), ex); - } - - } - } - connection.close(); - listener.onResponse(null); - } catch (Exception ex) { - logger.warn((Supplier) - () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), ex); - collectRemoteNodes(seedNodes, transportService, listener); - } finally { - IOUtils.closeWhileHandlingException(connection); - } - } - - @Override - public void handleException(TransportException exp) { - logger.warn((Supplier) - () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), - exp); - try { - IOUtils.closeWhileHandlingException(connection); - } finally { - collectRemoteNodes(seedNodes, transportService, listener); - } - } - - @Override - public String executor() { - return ThreadPool.Names.MANAGEMENT; - } - }); - success = true; - } finally { - if (success == false) { - connection.close(); - } - } - + }); } else { listener.onFailure(new IllegalStateException("no seed node left")); } + } catch (CancellableThreads.ExecutionCancelledException ex) { + listener.onFailure(ex); // we got canceled - fail the listener and step out } catch (ConnectTransportException | IOException | IllegalStateException ex) { // ISE if we fail the handshake with an version incompatible node if (seedNodes.hasNext()) { @@ -425,17 +401,96 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public void close() throws IOException { try { - closed.compareAndSet(false, true); - running.acquire(); - running.release(); + if (closed.compareAndSet(false, true)) { + cancellableThreads.cancel("connect handler is closed"); + running.acquire(); // acquire the semaphore to ensure all connections are closed and all thread joined + running.release(); + } } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } + + private class StateResponseHandler implements TransportResponseHandler { + + private final TransportService transportService; + private final Transport.Connection connection; + private final ActionListener listener; + private final Iterator seedNodes; + private final CancellableThreads cancellableThreads; + + public StateResponseHandler(TransportService transportService, Transport.Connection connection, ActionListener listener + , Iterator seedNodes, CancellableThreads cancellableThreads) { + this.transportService = transportService; + this.connection = connection; + this.listener = listener; + this.seedNodes = seedNodes; + this.cancellableThreads = cancellableThreads; + } + + @Override + public ClusterStateResponse newInstance() { + return new ClusterStateResponse(); + } + + @Override + public void handleResponse(ClusterStateResponse response) { + try { + cancellableThreads.executeIO(() -> { + DiscoveryNodes nodes = response.getState().nodes(); + Iterable nodesIter = nodes.getNodes()::valuesIt; + for (DiscoveryNode node : nodesIter) { + if (nodePredicate.test(node) && connectedNodes.size() < maxNumRemoteConnections) { + try { + transportService.connectToNode(node, remoteProfile); // noop if node is connected + connectedNodes.add(node); + } catch (ConnectTransportException | IllegalStateException ex) { + // ISE if we fail the handshake with an version incompatible node + // fair enough we can't connect just move on + logger.debug((Supplier) + () -> new ParameterizedMessage("failed to connect to node {}", node), ex); + } + } + } + }); + connection.close(); + listener.onResponse(null); + } catch (CancellableThreads.ExecutionCancelledException ex) { + listener.onFailure(ex); // we got canceled - fail the listener and step out + } catch (Exception ex) { + logger.warn((Supplier) + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", + clusterName), ex); + collectRemoteNodes(seedNodes, transportService, listener); + } finally { + // just to make sure we don't leak anything we close the connection here again even if we managed to do so before + IOUtils.closeWhileHandlingException(connection); + } + } + + @Override + public void handleException(TransportException exp) { + logger.warn((Supplier) + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), + exp); + try { + IOUtils.closeWhileHandlingException(connection); + } finally { + // once the connection is closed lets try the next node + collectRemoteNodes(seedNodes, transportService, listener); + } + } + + @Override + public String executor() { + return ThreadPool.Names.MANAGEMENT; + } + } } - boolean isConnectRunning() { // for testing only - return connectHandler.running.availablePermits() == 0; + boolean assertNoRunningConnections() { // for testing only + assert connectHandler.running.availablePermits() == 1; + return true; } } diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index b9af31f755c..5a6e44df538 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.search; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; @@ -33,6 +34,8 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportService; +import java.io.Closeable; +import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; @@ -48,7 +51,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; -public final class RemoteClusterService extends AbstractComponent { +public final class RemoteClusterService extends AbstractComponent implements Closeable { /** * A list of initial seed nodes to discover eligibale nodes from the remote cluster @@ -281,4 +284,9 @@ public final class RemoteClusterService extends AbstractComponent { throw new IllegalStateException("failed to connect to remote clusters", e); } } + + @Override + public void close() throws IOException { + IOUtils.close(remoteClusters.values()); + } } diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 308b7e02c39..49cd1e2059d 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -407,7 +407,9 @@ public class SearchTransportService extends AbstractLifecycleComponent { protected void doStop() {} @Override - protected void doClose() throws IOException {} + protected void doClose() throws IOException { + remoteClusterService.close(); + } } diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index 94a9778bdb9..674cf706268 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.action.search; +import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; @@ -32,20 +33,38 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.common.util.CancellableThreads; +import org.elasticsearch.mocksocket.MockServerSocket; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportConnectionListener; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.UnknownHostException; +import java.nio.channels.AlreadyConnectedException; import java.util.Arrays; import java.util.Collections; import java.util.List; +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.AtomicReference; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; + public class RemoteClusterConnectionTests extends ESTestCase { private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); @@ -104,7 +123,7 @@ public class RemoteClusterConnectionTests extends ESTestCase { updateSeedNodes(connection, Arrays.asList(seedNode)); assertTrue(service.nodeConnected(seedNode)); assertTrue(service.nodeConnected(discoverableNode)); - assertFalse(connection.isConnectRunning()); + assertTrue(connection.assertNoRunningConnections()); } } } @@ -134,7 +153,7 @@ public class RemoteClusterConnectionTests extends ESTestCase { assertTrue(service.nodeConnected(seedNode)); assertTrue(service.nodeConnected(discoverableNode)); assertFalse(service.nodeConnected(incompatibleSeedNode)); - assertFalse(connection.isConnectRunning()); + assertTrue(connection.assertNoRunningConnections()); } } } @@ -214,7 +233,7 @@ public class RemoteClusterConnectionTests extends ESTestCase { assertTrue(service.nodeConnected(seedNode)); assertFalse(service.nodeConnected(discoverableNode)); } - assertFalse(connection.isConnectRunning()); + assertTrue(connection.assertNoRunningConnections()); } } } @@ -247,12 +266,57 @@ public class RemoteClusterConnectionTests extends ESTestCase { Arrays.asList(seedNode), service, Integer.MAX_VALUE, n -> true)) { expectThrows(Exception.class, () -> updateSeedNodes(connection, Arrays.asList(seedNode))); assertFalse(service.nodeConnected(seedNode)); - assertFalse(connection.isConnectRunning()); + assertTrue(connection.assertNoRunningConnections()); } } } } + public void testSlowNodeCanBeCanceled() throws IOException, InterruptedException { + try (ServerSocket socket = new MockServerSocket()) { + socket.bind(new InetSocketAddress(InetAddress.getLocalHost(), 0), 1); + socket.setReuseAddress(true); + DiscoveryNode seedNode = new DiscoveryNode("TEST", new TransportAddress(socket.getInetAddress(), + socket.getLocalPort()), emptyMap(), + emptySet(), Version.CURRENT); + CountDownLatch acceptedLatch = new CountDownLatch(1); + CountDownLatch closeRemote = new CountDownLatch(1); + Thread t = new Thread() { + @Override + public void run() { + try (Socket accept = socket.accept()) { + acceptedLatch.countDown(); + closeRemote.await(); + } catch (IOException e) { + // that's fine we might close + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + }; + t.start(); + + try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { + service.start(); + service.acceptIncomingRequests(); + AtomicReference exceptionAtomicReference = new AtomicReference<>(); + try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", + Arrays.asList(seedNode), service, Integer.MAX_VALUE, n -> true)) { + ActionListener listener = ActionListener.wrap(x -> {}, x -> { + exceptionAtomicReference.set(x); + }); + connection.updateSeedNodes(Arrays.asList(seedNode), listener); + acceptedLatch.await(); + connection.close(); // now close it, this should trigger an interrupt on the socket and we can move on + assertTrue(connection.assertNoRunningConnections()); + } + closeRemote.countDown(); + expectThrows(CancellableThreads.ExecutionCancelledException.class, () -> {throw exceptionAtomicReference.get();}); + + } + } + } + public void testFetchShards() throws Exception { List knownNodes = new CopyOnWriteArrayList<>(); @@ -267,8 +331,6 @@ public class RemoteClusterConnectionTests extends ESTestCase { service.acceptIncomingRequests(); try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(seedNode), service, Integer.MAX_VALUE, n -> true)) { - CountDownLatch latch = new CountDownLatch(1); - String newNode = null; if (randomBoolean()) { updateSeedNodes(connection, Arrays.asList(seedNode)); } @@ -277,7 +339,6 @@ public class RemoteClusterConnectionTests extends ESTestCase { CountDownLatch responseLatch = new CountDownLatch(1); AtomicReference reference = new AtomicReference<>(); AtomicReference failReference = new AtomicReference<>(); - ActionListener shardsListener = ActionListener.wrap( x -> { reference.set(x); @@ -292,17 +353,135 @@ public class RemoteClusterConnectionTests extends ESTestCase { assertNull(failReference.get()); assertNotNull(reference.get()); ClusterSearchShardsResponse clusterSearchShardsResponse = reference.get(); - DiscoveryNode[] nodes = clusterSearchShardsResponse.getNodes(); - assertTrue(nodes.length != 0); - for (DiscoveryNode dataNode : nodes) { - assertNotNull(connection.getConnection(dataNode)); - if (dataNode.getName().equals(newNode)) { - assertFalse(service.nodeConnected(dataNode)); - } else { - assertTrue(service.nodeConnected(dataNode)); - } + assertEquals(knownNodes, Arrays.asList(clusterSearchShardsResponse.getNodes())); + assertTrue(connection.assertNoRunningConnections()); + } + } + } + } + + 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 = Arrays.asList(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)) { + 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) { + // that's fine + } else { + throw new AssertionError(x); + } + }); + connection.updateSeedNodes(seedNodes, listener); + } + latch.await(); + } catch (Exception ex) { + throw new AssertionError(ex); + } + } + }; + threads[i].start(); } - assertFalse(connection.isConnectRunning()); + + for (int i = 0; i < threads.length; i++) { + threads[i].join(); + } + assertTrue(service.nodeConnected(seedNode)); + assertTrue(service.nodeConnected(discoverableNode)); + assertTrue(service.nodeConnected(seedNode1)); + assertTrue(connection.assertNoRunningConnections()); + } + } + } + } + + public void testCloseWhileConcurrentlyConnecting() throws IOException, InterruptedException, BrokenBarrierException { + 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 seedNode1 = seedTransport1.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(discoverableTransport.getLocalDiscoNode()); + knownNodes.add(seedTransport1.getLocalDiscoNode()); + Collections.shuffle(knownNodes, random()); + List seedNodes = Arrays.asList(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)) { + int numThreads = randomIntBetween(4, 10); + Thread[] threads = new Thread[numThreads]; + CyclicBarrier barrier = new CyclicBarrier(numThreads + 1); + for (int i = 0; i < threads.length; i++) { + final int numConnectionAttempts = randomIntBetween(10, 100); + 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 || x instanceof AlreadyClosedException) { + // that's fine + } else { + throw new AssertionError(x); + } + }); + connection.updateSeedNodes(seedNodes, listener); + } + latch.await(); + } catch (Exception ex) { + throw new AssertionError(ex); + } + } + }; + threads[i].start(); + } + barrier.await(); + connection.close(); } } } From 1ef7115bbd7109913145a40564003ea307812f9a Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 5 Jan 2017 14:13:08 +0100 Subject: [PATCH 27/59] Add javadocs to TransportActionProxy --- .../elasticsearch/transport/TransportActionProxy.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index 0588c8bd7d0..7e4927294d2 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -9,7 +9,11 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.io.UncheckedIOException; -// nocommit javadocs +/** + * TransportActionProxy allows an arbitrary action to be executed on a defined target node while the initial request is send to a second + * node that acts as a request proxy to the target node. This is useful if a node is not directly connected to a target node but is + * connected to an intermediate node that establishes a transitive connection. + */ public final class TransportActionProxy { private TransportActionProxy() {} // no instance @@ -119,6 +123,9 @@ public final class TransportActionProxy { return "internal:transport/proxy/" + action; } + /** + * Wraps the actual request in a proxy request object that encodes the target node. + */ public static TransportRequest wrapRequest(DiscoveryNode node, TransportRequest request) { return new ProxyRequest<>(request, node); } From 0183b0c5a8b01738d673189105919c5b01b92236 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 5 Jan 2017 15:23:55 +0100 Subject: [PATCH 28/59] More cleanups --- .../action/search/RemoteClusterService.java | 89 +++++++++++++- .../action/search/TransportSearchAction.java | 88 ++------------ .../common/settings/ClusterSettings.java | 2 + .../action/search/SearchAsyncActionTests.java | 9 +- .../search/simple/SimpleSearchIT.java | 24 ---- .../elasticsearch/test/ESIntegTestCase.java | 111 ------------------ 6 files changed, 101 insertions(+), 222 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 5a6e44df538..7af6905b6f6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -18,18 +18,26 @@ */ package org.elasticsearch.action.search; +import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.util.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.PlainShardIterator; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportService; @@ -40,6 +48,7 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.UnknownHostException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -48,9 +57,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.function.Predicate; - +//nocommit this class needs more javadocs and must be unittested public final class RemoteClusterService extends AbstractComponent implements Closeable { /** @@ -81,6 +91,9 @@ public final class RemoteClusterService extends AbstractComponent implements Clo */ public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", Setting.Property.NodeScope); + + private static final char REMOTE_CLUSTER_INDEX_SEPARATOR = '|'; + private final TransportService transportService; private final int numRemoteConnections; private volatile Map remoteClusters = Collections.emptyMap(); @@ -91,6 +104,11 @@ public final class RemoteClusterService extends AbstractComponent implements Clo numRemoteConnections = REMOTE_CONNECTIONS_PER_CLUSTER.get(settings); } + /** + * This method updates the list of remote clusters. it's intendet to be used as a update consumer on the settings infrastructure + * @param seedSettings the group settings returned from {@link #REMOTE_CLUSTERS_SEEDS} + * @param connectionListener a listener invoked once every configured cluster has been connected to + */ void updateRemoteClusters(Settings seedSettings, ActionListener connectionListener) { Map remoteClusters = new HashMap<>(); Map> seeds = buildRemoteClustersSeeds(seedSettings); @@ -139,6 +157,30 @@ public final class RemoteClusterService extends AbstractComponent implements Clo return remoteClusters.isEmpty() == false; } + public String[] filterIndices(Map> perClusterIndices, String... requestIndices) { + List localIndicesList = new ArrayList<>(); + for (String index : requestIndices) { + int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); + if (i >= 0) { + String remoteCluster = index.substring(0, i); + if (isRemoteClusterRegistered(remoteCluster)) { + String remoteIndex = index.substring(i + 1); + List indices = perClusterIndices.get(remoteCluster); + if (indices == null) { + indices = new ArrayList<>(); + perClusterIndices.put(remoteCluster, indices); + } + indices.add(remoteIndex); + } else { + localIndicesList.add(index); + } + } else { + localIndicesList.add(index); + } + } + return localIndicesList.toArray(new String[localIndicesList.size()]); +} + /** * Returns true iff the given cluster is configured as a remote cluster. Otherwise false */ @@ -146,8 +188,8 @@ public final class RemoteClusterService extends AbstractComponent implements Clo return remoteClusters.containsKey(clusterName); } - void sendSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, - ActionListener> listener) { + void collectSearchShards(SearchRequest searchRequest, Map> remoteIndicesByCluster, + ActionListener> listener) { final CountDown responsesCountDown = new CountDown(remoteIndicesByCluster.size()); final Map searchShardsResponses = new ConcurrentHashMap<>(); final AtomicReference transportException = new AtomicReference<>(); @@ -191,6 +233,47 @@ public final class RemoteClusterService extends AbstractComponent implements Clo } } + + Function processRemoteShards(Map searchShardsResponses, + List remoteShardIterators, + Map aliasFilterMap) { + Map> nodeToCluster = new HashMap<>(); + for (Map.Entry entry : searchShardsResponses.entrySet()) { + String clusterName = entry.getKey(); + ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); + for (DiscoveryNode remoteNode : searchShardsResponse.getNodes()) { + nodeToCluster.put(remoteNode.getId(), () -> getConnection(remoteNode, clusterName)); + } + Map indicesAndFilters = searchShardsResponse.getIndicesAndFilters(); + for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { + //add the cluster name to the remote index names for indices disambiguation + //this ends up in the hits returned with the search response + ShardId shardId = clusterSearchShardsGroup.getShardId(); + Index remoteIndex = shardId.getIndex(); + Index index = new Index(clusterName + REMOTE_CLUSTER_INDEX_SEPARATOR + remoteIndex.getName(), remoteIndex.getUUID()); + ShardIterator shardIterator = new PlainShardIterator(new ShardId(index, shardId.getId()), + Arrays.asList(clusterSearchShardsGroup.getShards())); + remoteShardIterators.add(shardIterator); + AliasFilter aliasFilter; + if (indicesAndFilters == null) { + aliasFilter = new AliasFilter(null, Strings.EMPTY_ARRAY); + } else { + aliasFilter = indicesAndFilters.get(shardId.getIndexName()); + assert aliasFilter != null; + } + // here we have to map the filters to the UUID since from now on we use the uuid for the lookup + aliasFilterMap.put(remoteIndex.getUUID(), aliasFilter); + } + } + return (nodeId) -> { + Supplier supplier = nodeToCluster.get(nodeId); + if (supplier == null) { + throw new IllegalArgumentException("unknown remote node: " + nodeId); + } + return supplier.get(); + }; + } + /** * Returns a connection to the given node on the given remote cluster * @throws IllegalArgumentException if the remote cluster is unknown diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 189b54a039d..b5e5c7ed9a7 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -19,10 +19,7 @@ package org.elasticsearch.action.search; -import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.ClusterState; @@ -31,17 +28,13 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.cluster.routing.PlainShardIterator; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; @@ -51,10 +44,8 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; 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; @@ -70,8 +61,6 @@ public class TransportSearchAction extends HandledTransportAction SHARD_COUNT_LIMIT_SETTING = Setting.longSetting( "action.search.shard_count.limit", 1000L, 1L, Property.Dynamic, Property.NodeScope); - private static final char REMOTE_CLUSTER_INDEX_SEPARATOR = '|'; - private final ClusterService clusterService; private final SearchTransportService searchTransportService; private final RemoteClusterService remoteClusterService; @@ -124,7 +113,6 @@ public class TransportSearchAction extends HandledTransportAction listener) { // pure paranoia if time goes backwards we are at least positive final long startTimeInMillis = Math.max(0, System.currentTimeMillis()); - final String[] localIndices; - final Map> remoteIndicesByCluster = new HashMap<>(); + final Map> remoteIndicesByCluster; if (remoteClusterService.isCrossClusterSearchEnabled()) { - List localIndicesList = new ArrayList<>(); - for (String index : searchRequest.indices()) { - int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); - if (i >= 0) { - String remoteCluster = index.substring(0, i); - if (remoteClusterService.isRemoteClusterRegistered(remoteCluster)) { - String remoteIndex = index.substring(i + 1); - List indices = remoteIndicesByCluster.get(remoteCluster); - if (indices == null) { - indices = new ArrayList<>(); - remoteIndicesByCluster.put(remoteCluster, indices); - } - indices.add(remoteIndex); - } else { - localIndicesList.add(index); - } - } else { - localIndicesList.add(index); - } - } - localIndices = localIndicesList.toArray(new String[localIndicesList.size()]); + remoteIndicesByCluster = new HashMap<>(); + localIndices = remoteClusterService.filterIndices(remoteIndicesByCluster); } else { + remoteIndicesByCluster = Collections.emptyMap(); localIndices = searchRequest.indices(); } @@ -165,61 +134,18 @@ public class TransportSearchAction extends HandledTransportAction null, Collections.emptyMap(), listener); } else { - remoteClusterService.sendSearchShards(searchRequest, remoteIndicesByCluster, + remoteClusterService.collectSearchShards(searchRequest, remoteIndicesByCluster, ActionListener.wrap((searchShardsResponses) -> { List remoteShardIterators = new ArrayList<>(); Map remoteAliasFilters = new HashMap<>(); - Function connectionFunction = processRemoteShards(searchShardsResponses, - remoteShardIterators, remoteAliasFilters); + Function connectionFunction = remoteClusterService.processRemoteShards( + searchShardsResponses, remoteShardIterators, remoteAliasFilters); executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, remoteShardIterators, connectionFunction, remoteAliasFilters, listener); }, listener::onFailure)); } } - private Function processRemoteShards(Map searchShardsResponses, - List remoteShardIterators, - Map aliasFilterMap) { - Map> nodeToCluster = new HashMap<>(); - for (Map.Entry entry : searchShardsResponses.entrySet()) { - String clusterName = entry.getKey(); - ClusterSearchShardsResponse searchShardsResponse = entry.getValue(); - for (DiscoveryNode remoteNode : searchShardsResponse.getNodes()) { - nodeToCluster.put(remoteNode.getId(), () -> remoteClusterService.getConnection(remoteNode, clusterName)); - } - Map indicesAndFilters = searchShardsResponse.getIndicesAndFilters(); - for (ClusterSearchShardsGroup clusterSearchShardsGroup : searchShardsResponse.getGroups()) { - //add the cluster name to the remote index names for indices disambiguation - //this ends up in the hits returned with the search response - ShardId shardId = clusterSearchShardsGroup.getShardId(); - Index index = new Index(clusterName + REMOTE_CLUSTER_INDEX_SEPARATOR + shardId.getIndex().getName(), - shardId.getIndex().getUUID()); - ShardIterator shardIterator = new PlainShardIterator(new ShardId(index, shardId.getId()), - Arrays.asList(clusterSearchShardsGroup.getShards())); - remoteShardIterators.add(shardIterator); - AliasFilter aliasFilter; - if (indicesAndFilters == null) { - //TODO this section is returned only by 5.1+ nodes. With 5.0.x nodes we should rather retrieve the alias filters - //using another api. What we do now causes the remote alias filters to be ignored whenever the node that we - //called search shards against was on 5.0.x. - aliasFilter = new AliasFilter(null, Strings.EMPTY_ARRAY); - } else { - aliasFilter = indicesAndFilters.get(shardId.getIndexName()); - assert aliasFilter != null; - } - // here we have to map the filters to the UUID since from now on we use the uuid for the lookup - aliasFilterMap.put(shardId.getIndex().getUUID(), aliasFilter); - } - } - return (nodeId) -> { - Supplier supplier = nodeToCluster.get(nodeId); - if (supplier == null) { - throw new IllegalArgumentException("unknown remote node: " + nodeId); - } - return supplier.get(); - }; - } - private void executeSearch(SearchTask task, long startTimeInMillis, SearchRequest searchRequest, String[] localIndices, List remoteShardIterators, Function remoteConnections, Map remoteAliasMap, ActionListener listener) { diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index a8b4fa54459..a35adefc0a0 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -256,6 +256,8 @@ public final class ClusterSettings extends AbstractScopedSettings { TransportSearchAction.SHARD_COUNT_LIMIT_SETTING, RemoteClusterService.REMOTE_CLUSTERS_SEEDS, RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER, + RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, + RemoteClusterServic.REMOTE_NODE_ATTRIBUTE TransportService.TRACE_LOG_EXCLUDE_SETTING, TransportService.TRACE_LOG_INCLUDE_SETTING, TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING, diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index 5db18833af8..4e2c8e9ca7c 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -137,10 +137,13 @@ public class SearchAsyncActionTests extends ESTestCase { latch.await(); assertNotNull(response.get()); assertFalse(nodeToContextMap.isEmpty()); - assertTrue(nodeToContextMap.containsKey(primaryNode)); + assertTrue(nodeToContextMap.toString(), nodeToContextMap.containsKey(primaryNode) || nodeToContextMap.containsKey(replicaNode)); assertEquals(shardsIter.size(), numFreedContext.get()); - assertTrue(nodeToContextMap.get(primaryNode).toString(), nodeToContextMap.get(primaryNode).isEmpty()); - + if (nodeToContextMap.containsKey(primaryNode)) { + assertTrue(nodeToContextMap.get(primaryNode).toString(), nodeToContextMap.get(primaryNode).isEmpty()); + } else { + assertTrue(nodeToContextMap.get(replicaNode).toString(), nodeToContextMap.get(replicaNode).isEmpty()); + } } private GroupShardsIterator getShardsIter(String index, int numShards, boolean doReplicas, DiscoveryNode primaryNode, diff --git a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java index 6cc07a92dca..099cce038c1 100644 --- a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -20,38 +20,19 @@ package org.elasticsearch.search.simple; import org.apache.lucene.util.Constants; -import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.client.Client; -import org.elasticsearch.cluster.ClusterName; -import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.xcontent.XContentFactory; -import org.elasticsearch.discovery.zen.PublishClusterStateActionTests; -import org.elasticsearch.env.Environment; -import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.node.MockNode; -import org.elasticsearch.node.Node; -import org.elasticsearch.node.NodeValidationException; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.rescore.QueryRescorerBuilder; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.discovery.TestZenDiscovery; -import org.elasticsearch.transport.MockTcpTransportPlugin; -import java.nio.file.Path; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import java.util.concurrent.ExecutionException; @@ -72,11 +53,6 @@ import static org.hamcrest.Matchers.containsString; public class SimpleSearchIT extends ESIntegTestCase { - @Override - protected boolean useSearchProxyNode() { - return true; - } - public void testSearchNullIndex() { expectThrows(NullPointerException.class, () -> client().prepareSearch((String) null).setQuery(QueryBuilders.termQuery("_id", "XXX1")).get()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index c433c6d2abc..038477cc3aa 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -30,7 +30,6 @@ import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.ShardOperationFailedException; @@ -55,24 +54,19 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.ClearScrollResponse; -import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.AdminClient; import org.elasticsearch.client.Client; -import org.elasticsearch.client.FilterClient; import org.elasticsearch.client.Requests; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; import org.elasticsearch.cluster.ClusterModule; -import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -83,10 +77,8 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.network.NetworkModule; @@ -98,7 +90,6 @@ import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; @@ -126,10 +117,7 @@ import org.elasticsearch.indices.IndicesQueryCache; import org.elasticsearch.indices.IndicesRequestCache; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.store.IndicesStore; -import org.elasticsearch.node.MockNode; -import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeMocksPlugin; -import org.elasticsearch.node.NodeValidationException; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.script.ScriptService; @@ -150,7 +138,6 @@ import org.junit.BeforeClass; import java.io.IOException; import java.io.InputStream; -import java.io.UncheckedIOException; import java.lang.annotation.Annotation; import java.lang.annotation.ElementType; import java.lang.annotation.Inherited; @@ -173,7 +160,6 @@ import java.util.IdentityHashMap; import java.util.List; import java.util.Locale; import java.util.Map; -import java.util.Objects; import java.util.Random; import java.util.Set; import java.util.concurrent.Callable; @@ -350,7 +336,6 @@ public abstract class ESIntegTestCase extends ESTestCase { private static ESIntegTestCase INSTANCE = null; // see @SuiteScope private static Long SUITE_SEED = null; - private static Node searchProxyNode; @BeforeClass public static void beforeClass() throws Exception { @@ -381,9 +366,6 @@ public abstract class ESIntegTestCase extends ESTestCase { cluster().beforeTest(random(), getPerTestTransportClientRatio()); cluster().wipe(excludeTemplates()); randomIndexTemplate(); - if (useSearchProxyNode()) { - searchProxyNode = startSearchProxyNode(); - } } private void printTestMessage(String message) { @@ -556,7 +538,6 @@ public abstract class ESIntegTestCase extends ESTestCase { protected final void afterInternal(boolean afterClass) throws Exception { boolean success = false; try { - final Scope currentClusterScope = getCurrentClusterScope(); clearDisruptionScheme(); try { @@ -600,8 +581,6 @@ public abstract class ESIntegTestCase extends ESTestCase { if (currentClusterScope == Scope.TEST) { clearClusters(); // it is ok to leave persistent / transient cluster state behind if scope is TEST } - IOUtils.close(searchProxyNode); - searchProxyNode = null; } success = true; } finally { @@ -655,53 +634,6 @@ public abstract class ESIntegTestCase extends ESTestCase { if (frequently()) { client = new RandomizingClient(client, random()); } - if (searchProxyNode != null && randomBoolean()) { - client = new FilterClient(client) { - @Override - public SearchRequestBuilder prepareSearch(String... indices) { - return searchProxyNode.client().prepareSearch(convertToRemoteIndices(indices)); - } - - private String[] convertToRemoteIndices(String[] indices) { - if (Objects.requireNonNull(indices).length == 0) { - return new String[] {"test_remote_cluster|_all"}; - } - String [] remoteIndices = new String[indices.length]; - for (int i = 0; i < indices.length; i++) { - remoteIndices[i] = "test_remote_cluster|"+ Objects.requireNonNull(indices[i]); - } - return remoteIndices; - } - - @Override - public ActionFuture search(SearchRequest request) { - // we copy the request to ensure we never modify the original request - try (BytesStreamOutput out = new BytesStreamOutput()) { - request.writeTo(out); - SearchRequest copy = new SearchRequest(); - copy.readFrom(out.bytes().streamInput()); - copy.indices(convertToRemoteIndices(request.indices())); - return searchProxyNode.client().search(copy); - } catch (IOException ex) { - throw new UncheckedIOException(ex); - } - } - - @Override - public void search(SearchRequest request, ActionListener listener) { - // we copy the request to ensure we never modify the original request - try (BytesStreamOutput out = new BytesStreamOutput()) { - request.writeTo(out); - SearchRequest copy = new SearchRequest(); - copy.readFrom(out.bytes().streamInput()); - copy.indices(convertToRemoteIndices(request.indices())); - searchProxyNode.client().search(copy, listener); - } catch (IOException ex) { - throw new UncheckedIOException(ex); - } - } - }; - } return client; } @@ -2271,47 +2203,4 @@ public abstract class ESIntegTestCase extends ESTestCase { String uuid = getIndexResponse.getSettings().get(index).get(IndexMetaData.SETTING_INDEX_UUID); return new Index(index, uuid); } - - protected boolean useSearchProxyNode() { - return false; // nocommit - lets enable this globally - } - - private synchronized Node startSearchProxyNode() { - if (isInternalCluster()) { - final DiscoveryNode seedNode = internalCluster().getInstance(ClusterService.class).localNode(); - final Path tempDir = createTempDir(); - Settings settings = Settings.builder() - .put(ClusterName.CLUSTER_NAME_SETTING.getKey(), "search_proxy_" + internalCluster().getClusterName()) - .put(Environment.PATH_HOME_SETTING.getKey(), tempDir) - .put(Environment.PATH_REPO_SETTING.getKey(), tempDir.resolve("repo")) - .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), createTempDir().getParent()) - .put("search.remote.seeds.test_remote_cluster", seedNode.getAddress().toString()) - .put("node.name", "node_prx_0") - .put(EsExecutors.PROCESSORS_SETTING.getKey(), 1) // limit the number of threads created - .put(NetworkModule.HTTP_ENABLED.getKey(), false) - .put("transport.type", MockTcpTransportPlugin.MOCK_TCP_TRANSPORT_NAME) - .put(Node.NODE_DATA_SETTING.getKey(), false) - .put(Node.NODE_MASTER_SETTING.getKey(), true) - .put(Node.NODE_INGEST_SETTING.getKey(), false) - .build(); - Collection> plugins = nodePlugins(); - if (plugins.contains(MockTcpTransportPlugin.class) == false) { - plugins = new ArrayList<>(plugins); - plugins.add(MockTcpTransportPlugin.class); - } - if (plugins.contains(TestZenDiscovery.TestPlugin.class) == false) { - plugins = new ArrayList<>(plugins); - plugins.add(TestZenDiscovery.TestPlugin.class); - } - Node build = new MockNode(settings, plugins); - try { - build.start(); - } catch (NodeValidationException e) { - throw new RuntimeException(e); - } - return build; - } else { - return null; - } - } } From dca54734acabde54bc333c64f318c69d96758aed Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 5 Jan 2017 16:10:34 +0100 Subject: [PATCH 29/59] add basic docs --- .../action/search/RemoteClusterService.java | 4 +- .../action/search/TransportSearchAction.java | 2 +- .../common/settings/ClusterSettings.java | 2 +- docs/build.gradle | 1 + docs/reference/modules.asciidoc | 5 + .../modules/cross-cluster-search.asciidoc | 112 ++++++++++++++++++ 6 files changed, 122 insertions(+), 4 deletions(-) create mode 100644 docs/reference/modules/cross-cluster-search.asciidoc diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 7af6905b6f6..53c043f2332 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -86,7 +86,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo /** * The name of a node attribute to filter out nodes that should not be connected to in the remote cluster. - * For instance a node can be configured with node.node_attr.gateway: true in order to be eligable as a gateway node between + * For instance a node can be configured with node.node_attr.gateway: true in order to be eligible as a gateway node between * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster */ public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", @@ -157,7 +157,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo return remoteClusters.isEmpty() == false; } - public String[] filterIndices(Map> perClusterIndices, String... requestIndices) { + public String[] filterIndices(Map> perClusterIndices, String[] requestIndices) { List localIndicesList = new ArrayList<>(); for (String index : requestIndices) { int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index b5e5c7ed9a7..17101b3f552 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -124,7 +124,7 @@ public class TransportSearchAction extends HandledTransportAction> remoteIndicesByCluster; if (remoteClusterService.isCrossClusterSearchEnabled()) { remoteIndicesByCluster = new HashMap<>(); - localIndices = remoteClusterService.filterIndices(remoteIndicesByCluster); + localIndices = remoteClusterService.filterIndices(remoteIndicesByCluster, searchRequest.indices()); } else { remoteIndicesByCluster = Collections.emptyMap(); localIndices = searchRequest.indices(); diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index a35adefc0a0..17ad63ed1d9 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -257,7 +257,7 @@ public final class ClusterSettings extends AbstractScopedSettings { RemoteClusterService.REMOTE_CLUSTERS_SEEDS, RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER, RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING, - RemoteClusterServic.REMOTE_NODE_ATTRIBUTE + RemoteClusterService.REMOTE_NODE_ATTRIBUTE, TransportService.TRACE_LOG_EXCLUDE_SETTING, TransportService.TRACE_LOG_INCLUDE_SETTING, TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING, diff --git a/docs/build.gradle b/docs/build.gradle index 5347e529fa3..798bce6a0a6 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -145,6 +145,7 @@ buildRestTests.expectedUnconvertedCandidates = [ 'reference/modules/scripting/security.asciidoc', 'reference/modules/scripting/using.asciidoc', 'reference/modules/transport.asciidoc', + 'reference/modules/cross-cluster-search.asciidoc', // this is hart to test since we need 2 clusters -- maybe we can trick it into referencing itself... 'reference/query-dsl/exists-query.asciidoc', 'reference/query-dsl/function-score-query.asciidoc', 'reference/query-dsl/geo-shape-query.asciidoc', diff --git a/docs/reference/modules.asciidoc b/docs/reference/modules.asciidoc index 2ab1232d3b5..0e55b858359 100644 --- a/docs/reference/modules.asciidoc +++ b/docs/reference/modules.asciidoc @@ -77,6 +77,11 @@ The modules in this section are: A tribe node joins one or more clusters and acts as a federated client across them. + +<>:: + + Cross cluster search allows to execute search requests across more than one cluster without joining them and acts + as a federated client across them. -- diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc new file mode 100644 index 00000000000..128f2cf180b --- /dev/null +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -0,0 +1,112 @@ +[[modules-cross-cluster-search]] +== Cross cluster search + +The _cross cluster search_ feature allows any node to act as a federated client across +multiple clusters. In contrast to the _tribe_ feature, a _cross cluster search_ node won't +join the remote cluster, instead it connects to a remote cluster in a light fashion in order to executed +federated search requests. + +The _cross cluster search_ feature works by configuring a remote cluster in the cluster state and connects only to a +limited number of nodes in the remote cluster. Each remote cluster is referenced by a name and a list of seed nodes. +Those seed nodes are used to discover other nodes eligible as so-called _gateway nodes_. Each node in a cluster that +has remote cluster configured connects to one or more _gateway nodes_ and uses them to federate search requests to +the remote cluster. + +Remote clusters can either be configured as part of the `elasticsearch.yml` file or be dynamically updated via +the <>. If a remote cluster is configured via `elasticsearch.yml` only +the nodes with the configuration set will be connecting to the remote cluster. Remote clusters set via the +<> will be available on every node in the cluster. + +The `elasticsearch.yml` config file for a _cross cluster search_ node just needs to list the +remote clusters that should be connected to, for instance: + +[source,yaml] +-------------------------------- +search: + remote: + seeds: + cluster_one: 127.0.0.1:9300 <1> + cluster_two: 127.0.0.1:9301 <1> + +-------------------------------- +<1> `cluster_one` and `cluster_two` are arbitrary names representing the connection to each cluster. These names are subsequently used to distinguish between local and remote indices. + +[float] +=== Using cross cluster search + +To search the `twitter` index on remote cluster `cluster_1` the index name must be prefixed with the cluster name +separated by a `|` character: + +[source,js] +-------------------------------------------------- +POST /cluster_1|twitter/tweet/_search +{ + "match_all": {} +} +-------------------------------------------------- + +In contrast to the `tribe` feature cross cluster search can also search indices with the same name on different clusters: + + +[source,js] +-------------------------------------------------- +POST /cluster_1|twitter,twitter/tweet/_search +{ + "match_all": {} +} +-------------------------------------------------- + +Search results are disambiguated the same way as the indices are disambiguated in the request. Even if index names are +identical these indices will be treated as different indices when results are merged. All results retrieved from a remote index +will be prefixed with it's remote clusters name: + +[source,js] +-------------------------------------------------- + { + "took" : 89, + "timed_out" : false, + "_shards" : { + "total" : 10, + "successful" : 10, + "failed" : 0 + }, + "hits" : { + "total" : 2, + "max_score" : 1.0, + "hits" : [ + { + "_index" : "cluster_1|twitter", + "_type" : "tweet", + "_id" : "1", + "_score" : 1.0, + "_source" : { + "user" : "kimchy", + "postDate" : "2009-11-15T14:12:12", + "message" : "trying out Elasticsearch" + } + }, + { + "_index" : "cluster_1|twitter", + "_type" : "tweet", + "_id" : "1", + "_score" : 1.0, + "_source" : { + "user" : "kimchy", + "postDate" : "2009-11-15T14:12:12", + "message" : "trying out Elasticsearch" + } + } + ] + } +} +-------------------------------------------------- + +[float] +=== Cross cluster search settings + +* `search.remote.connections_per_cluster` - the number of nodes to connect to per remote cluster. The default is `3` +* `search.remote.initial_connect_timeout` - the time to wait for remote connections to be established when the node starts. The default is `30s`. +* `search.remote.node_attribute` - a node attribute to filter out nodes that are eligible as a gateway node in the remote cluster. +For instance a node can have a node attribute `node.attr.gateway: true` such that only nodes with this attribute +will be connected to if `search.remote.node_attribute` is set to `gateway` + From 349ea0f9b6aeba9abd71cee4bb0d29326f59810b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 5 Jan 2017 17:03:12 +0100 Subject: [PATCH 30/59] cut over to use : instead of | for cross cluster search --- .../action/search/RemoteClusterService.java | 2 +- .../search/RemoteClusterConnectionTests.java | 6 +- .../search/RemoteClusterServiceTests.java | 61 +++++++++++++++++++ .../modules/cross-cluster-search.asciidoc | 10 +-- .../test/multi_cluster/10_basic.yaml | 18 +++--- 5 files changed, 81 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 53c043f2332..651f9731e31 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -92,7 +92,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", Setting.Property.NodeScope); - private static final char REMOTE_CLUSTER_INDEX_SEPARATOR = '|'; + private static final char REMOTE_CLUSTER_INDEX_SEPARATOR = ':'; private final TransportService transportService; private final int numRemoteConnections; diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index 674cf706268..2a8198e46a9 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -75,7 +75,11 @@ public class RemoteClusterConnectionTests extends ESTestCase { ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); } - public MockTransportService startTransport(String id, List knownNodes, Version version) { + private MockTransportService startTransport(String id, List knownNodes, Version version) { + return startTransport(id, knownNodes, version, threadPool); + } + + public static MockTransportService startTransport(String id, List knownNodes, Version version, ThreadPool threadPool) { boolean success = false; MockTransportService newService = MockTransportService.createNewService(Settings.EMPTY, version, threadPool, null); try { diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index 6e29c720893..fb98d902164 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -19,18 +19,48 @@ package org.elasticsearch.action.search; import org.elasticsearch.Version; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; +import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; +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.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; 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.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeUnit; public class RemoteClusterServiceTests extends ESTestCase { + 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 RemoteClusterConnectionTests.startTransport(id, knownNodes, version, threadPool); + } + public void testRemoteClusterSeedSetting() { // simple validation RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() @@ -64,4 +94,35 @@ public class RemoteClusterServiceTests extends ESTestCase { assertEquals(bar.getId(), "bar#[::1]:9090"); assertEquals(bar.getVersion(), Version.CURRENT.minimumCompatibilityVersion()); } + + + public void testFilterIndices() { + List knownNodes = new CopyOnWriteArrayList<>(); + try (MockTransportService seedTransport = startTransport("cluster_1_node", knownNodes, Version.CURRENT); + MockTransportService otherSeedTransport = startTransport("cluster_2_node", knownNodes, Version.CURRENT)) { + DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); + DiscoveryNode otherSeedNode = otherSeedTransport.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(otherSeedTransport.getLocalDiscoNode()); + Collections.shuffle(knownNodes, random()); + + try (MockTransportService transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, + null)) { + transportService.start(); + transportService.acceptIncomingRequests(); + Settings.Builder builder = Settings.builder(); + builder.putArray("search.remote.seeds.cluster_1", seedNode.getAddress().toString()); + builder.putArray("search.remote.seeds.cluster_2", otherSeedNode.getAddress().toString()); + RemoteClusterService service = new RemoteClusterService(builder.build(), transportService); + service.initializeRemoteClusters(); + Map> perClusterIndices = new HashMap<>(); + String[] localIndices = service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", + "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}); + assertArrayEquals(new String[]{"foo:bar", "foo"}, localIndices); + assertEquals(2, perClusterIndices.size()); + assertEquals(Arrays.asList("bar", "test"), perClusterIndices.get("cluster_1")); + assertEquals(Arrays.asList("foo:bar", "foo*"), perClusterIndices.get("cluster_2")); + } + } + } } diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index 128f2cf180b..af88ac123e5 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -35,11 +35,11 @@ search: === Using cross cluster search To search the `twitter` index on remote cluster `cluster_1` the index name must be prefixed with the cluster name -separated by a `|` character: +separated by a `:` character: [source,js] -------------------------------------------------- -POST /cluster_1|twitter/tweet/_search +POST /cluster_one:twitter/tweet/_search { "match_all": {} } @@ -50,7 +50,7 @@ In contrast to the `tribe` feature cross cluster search can also search indices [source,js] -------------------------------------------------- -POST /cluster_1|twitter,twitter/tweet/_search +POST /cluster_one:twitter,twitter/tweet/_search { "match_all": {} } @@ -75,7 +75,7 @@ will be prefixed with it's remote clusters name: "max_score" : 1.0, "hits" : [ { - "_index" : "cluster_1|twitter", + "_index" : "cluster_one:twitter", "_type" : "tweet", "_id" : "1", "_score" : 1.0, @@ -86,7 +86,7 @@ will be prefixed with it's remote clusters name: } }, { - "_index" : "cluster_1|twitter", + "_index" : "twitter", "_type" : "tweet", "_id" : "1", "_score" : 1.0, diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index 586b3764ea8..4ec7747bfee 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -27,7 +27,7 @@ - do: search: - index: test_index,my_remote_cluster|test_index + index: test_index,my_remote_cluster:test_index body: aggs: cluster: @@ -44,7 +44,7 @@ - do: search: - index: test_index,my_remote_cluster|test_index + index: test_index,my_remote_cluster:test_index body: query: term: @@ -56,14 +56,14 @@ - match: { _shards.total: 5 } - match: { hits.total: 6} - - match: { hits.hits.0._index: "my_remote_cluster|test_index"} + - match: { hits.hits.0._index: "my_remote_cluster:test_index"} - length: { aggregations.cluster.buckets: 1 } - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } - match: { aggregations.cluster.buckets.0.doc_count: 6 } - do: search: - index: my_remote_cluster|test_index + index: my_remote_cluster:test_index body: aggs: cluster: @@ -72,7 +72,7 @@ - match: { _shards.total: 3 } - match: { hits.total: 6} - - match: { hits.hits.0._index: "my_remote_cluster|test_index"} + - match: { hits.hits.0._index: "my_remote_cluster:test_index"} - length: { aggregations.cluster.buckets: 1 } - match: { aggregations.cluster.buckets.0.key: "remote_cluster" } - match: { aggregations.cluster.buckets.0.doc_count: 6 } @@ -112,20 +112,20 @@ - do: search: - index: test_remote_cluster|test_index + index: test_remote_cluster:test_index - match: { _shards.total: 3 } - match: { hits.total: 6 } - - match: { hits.hits.0._index: "test_remote_cluster|test_index" } + - match: { hits.hits.0._index: "test_remote_cluster:test_index" } --- "Search an filtered alias on the remote cluster": - do: search: - index: my_remote_cluster|aliased_test_index + index: my_remote_cluster:aliased_test_index - match: { _shards.total: 3 } - match: { hits.total: 2 } - match: { hits.hits.0._source.filter_field: 1 } - - match: { hits.hits.0._index: "my_remote_cluster|test_index" } + - match: { hits.hits.0._index: "my_remote_cluster:test_index" } From 56082d10283e16909d82e57088b09112ef1c7e9f Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 6 Jan 2017 12:35:18 +0100 Subject: [PATCH 31/59] add more tests for RemoteClusterService --- .../shards/ClusterSearchShardsGroup.java | 2 +- .../action/search/RemoteClusterService.java | 11 +- .../search/RemoteClusterServiceTests.java | 108 ++++++++++++++++-- 3 files changed, 109 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsGroup.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsGroup.java index 473d31754eb..79a014ebda7 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsGroup.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsGroup.java @@ -38,7 +38,7 @@ public class ClusterSearchShardsGroup implements Streamable, ToXContent { } - ClusterSearchShardsGroup(ShardId shardId, ShardRouting[] shards) { + public ClusterSearchShardsGroup(ShardId shardId, ShardRouting[] shards) { this.shardId = shardId; this.shards = shards; } diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 651f9731e31..1e01a6be19b 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -60,7 +60,9 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Predicate; -//nocommit this class needs more javadocs and must be unittested +/** + * Basic service for accessing remote clusters via gateway nodes + */ public final class RemoteClusterService extends AbstractComponent implements Closeable { /** @@ -157,6 +159,13 @@ public final class RemoteClusterService extends AbstractComponent implements Clo return remoteClusters.isEmpty() == false; } + /** + * Filters out indices that refer to a remote cluster and adds them to the given per cluster indices map. + * + * @param perClusterIndices a map to fill with remote cluster indices from the given request indices + * @param requestIndices the indices in the search request to filter + * @return all indices in the requestIndices array that are not remote cluster indices + */ public String[] filterIndices(Map> perClusterIndices, String[] requestIndices) { List localIndicesList = new ArrayList<>(); for (String index : requestIndices) { diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index fb98d902164..68136bfe376 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -30,15 +30,29 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.ShardIterator; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingHelper; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.TermsQueryBuilder; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.Transport; +import java.io.IOException; import java.net.InetAddress; import java.net.InetSocketAddress; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -46,6 +60,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; +import java.util.function.Function; public class RemoteClusterServiceTests extends ESTestCase { @@ -61,6 +76,13 @@ public class RemoteClusterServiceTests extends ESTestCase { return RemoteClusterConnectionTests.startTransport(id, knownNodes, version, threadPool); } + public void testSettingsAreRegistered() { + assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_CLUSTERS_SEEDS)); + assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_CONNECTIONS_PER_CLUSTER)); + assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING)); + assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_NODE_ATTRIBUTE)); + } + public void testRemoteClusterSeedSetting() { // simple validation RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() @@ -96,7 +118,7 @@ public class RemoteClusterServiceTests extends ESTestCase { } - public void testFilterIndices() { + public void testFilterIndices() throws IOException { List knownNodes = new CopyOnWriteArrayList<>(); try (MockTransportService seedTransport = startTransport("cluster_1_node", knownNodes, Version.CURRENT); MockTransportService otherSeedTransport = startTransport("cluster_2_node", knownNodes, Version.CURRENT)) { @@ -113,16 +135,82 @@ public class RemoteClusterServiceTests extends ESTestCase { Settings.Builder builder = Settings.builder(); builder.putArray("search.remote.seeds.cluster_1", seedNode.getAddress().toString()); builder.putArray("search.remote.seeds.cluster_2", otherSeedNode.getAddress().toString()); - RemoteClusterService service = new RemoteClusterService(builder.build(), transportService); - service.initializeRemoteClusters(); - Map> perClusterIndices = new HashMap<>(); - String[] localIndices = service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", - "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}); - assertArrayEquals(new String[]{"foo:bar", "foo"}, localIndices); - assertEquals(2, perClusterIndices.size()); - assertEquals(Arrays.asList("bar", "test"), perClusterIndices.get("cluster_1")); - assertEquals(Arrays.asList("foo:bar", "foo*"), perClusterIndices.get("cluster_2")); + try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { + assertFalse(service.isCrossClusterSearchEnabled()); + service.initializeRemoteClusters(); + assertTrue(service.isCrossClusterSearchEnabled()); + assertTrue(service.isRemoteClusterRegistered("cluster_1")); + assertTrue(service.isRemoteClusterRegistered("cluster_2")); + assertFalse(service.isRemoteClusterRegistered("foo")); + Map> perClusterIndices = new HashMap<>(); + String[] localIndices = service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", + "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}); + assertArrayEquals(new String[]{"foo:bar", "foo"}, localIndices); + assertEquals(2, perClusterIndices.size()); + assertEquals(Arrays.asList("bar", "test"), perClusterIndices.get("cluster_1")); + assertEquals(Arrays.asList("foo:bar", "foo*"), perClusterIndices.get("cluster_2")); + } } } } + + public void testProcessRemoteShards() throws IOException { + try (RemoteClusterService service = new RemoteClusterService(Settings.EMPTY, null)) { + assertFalse(service.isCrossClusterSearchEnabled()); + List iteratorList = new ArrayList<>(); + Map searchShardsResponseMap = new HashMap<>(); + DiscoveryNode[] nodes = new DiscoveryNode[] { + new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT), + new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT) + }; + Map indicesAndAliases = new HashMap<>(); + indicesAndAliases.put("foo", new AliasFilter(new TermsQueryBuilder("foo", "bar"), Strings.EMPTY_ARRAY)); + indicesAndAliases.put("bar", new AliasFilter(new MatchAllQueryBuilder(), Strings.EMPTY_ARRAY)); + ClusterSearchShardsGroup[] groups = new ClusterSearchShardsGroup[] { + new ClusterSearchShardsGroup(new ShardId("foo", "foo_id", 0), + new ShardRouting[] {TestShardRouting.newShardRouting("foo", 0, "node1", true, ShardRoutingState.STARTED), + TestShardRouting.newShardRouting("foo", 0, "node2", false, ShardRoutingState.STARTED)}), + new ClusterSearchShardsGroup(new ShardId("foo", "foo_id", 1), + new ShardRouting[] {TestShardRouting.newShardRouting("foo", 0, "node1", true, ShardRoutingState.STARTED), + TestShardRouting.newShardRouting("foo", 1, "node2", false, ShardRoutingState.STARTED)}), + new ClusterSearchShardsGroup(new ShardId("bar", "bar_id", 0), + new ShardRouting[] {TestShardRouting.newShardRouting("bar", 0, "node2", true, ShardRoutingState.STARTED), + TestShardRouting.newShardRouting("bar", 0, "node1", false, ShardRoutingState.STARTED)}) + }; + searchShardsResponseMap.put("test_cluster_1", new ClusterSearchShardsResponse(groups, nodes, indicesAndAliases)); + Map remoteAliases = new HashMap<>(); + service.processRemoteShards(searchShardsResponseMap, iteratorList, remoteAliases); + assertEquals(3, iteratorList.size()); + for (ShardIterator iterator : iteratorList) { + if (iterator.shardId().getIndexName().endsWith("foo")) { + assertTrue(iterator.shardId().getId() == 0 || iterator.shardId().getId() == 1); + assertEquals("test_cluster_1:foo", iterator.shardId().getIndexName()); + ShardRouting shardRouting = iterator.nextOrNull(); + assertNotNull(shardRouting); + assertEquals(shardRouting.getIndexName(), "foo"); + shardRouting = iterator.nextOrNull(); + assertNotNull(shardRouting); + assertEquals(shardRouting.getIndexName(), "foo"); + + assertNull(iterator.nextOrNull()); + } else { + assertEquals(0, iterator.shardId().getId()); + assertEquals("test_cluster_1:bar", iterator.shardId().getIndexName()); + ShardRouting shardRouting = iterator.nextOrNull(); + assertNotNull(shardRouting); + assertEquals(shardRouting.getIndexName(), "bar"); + shardRouting = iterator.nextOrNull(); + assertNotNull(shardRouting); + assertEquals(shardRouting.getIndexName(), "bar"); + + assertNull(iterator.nextOrNull()); + } + } + assertEquals(2, remoteAliases.size()); + assertTrue(remoteAliases.toString(), remoteAliases.containsKey("foo_id")); + assertTrue(remoteAliases.toString(), remoteAliases.containsKey("bar_id")); + assertEquals(new TermsQueryBuilder("foo", "bar"), remoteAliases.get("foo_id").getQueryBuilder()); + assertEquals(new MatchAllQueryBuilder(), remoteAliases.get("bar_id").getQueryBuilder()); + } + } } From bb5b1a022e48964bfff18d341833b8ed714a037d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 9 Jan 2017 17:01:57 +0100 Subject: [PATCH 32/59] Fix line length --- .../admin/cluster/shards/ClusterSearchShardsResponse.java | 3 ++- .../elasticsearch/action/search/RemoteClusterConnection.java | 4 ++-- .../elasticsearch/action/search/SearchTransportService.java | 3 ++- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java index 9f855cb690f..3ee5f56ebdc 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/shards/ClusterSearchShardsResponse.java @@ -42,7 +42,8 @@ public class ClusterSearchShardsResponse extends ActionResponse implements ToXCo } - public ClusterSearchShardsResponse(ClusterSearchShardsGroup[] groups, DiscoveryNode[] nodes, Map indicesAndFilters) { + public ClusterSearchShardsResponse(ClusterSearchShardsGroup[] groups, DiscoveryNode[] nodes, + Map indicesAndFilters) { this.groups = groups; this.nodes = nodes; this.indicesAndFilters = indicesAndFilters; diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index e4549f1b529..157c3a075fb 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -76,8 +76,8 @@ import java.util.function.Predicate; * in the remote cluster and connects to all eligible nodes, for details see {@link RemoteClusterService#REMOTE_NODE_ATTRIBUTE}. * * In the case of a disconnection, this class will issue a re-connect task to establish at most - * {@link RemoteClusterService#REMOTE_CONNECTIONS_PER_CLUSTER} until either all eligible nodes are exhausted or the maximum number of connections - * per cluster has been reached. + * {@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 extends AbstractComponent implements TransportConnectionListener, Closeable { diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 49cd1e2059d..77754c55bd4 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -299,7 +299,8 @@ public class SearchTransportService extends AbstractLifecycleComponent { channel.sendResponse(TransportResponse.Empty.INSTANCE); } }); - TransportActionProxy.registerProxyAction(transportService, CLEAR_SCROLL_CONTEXTS_ACTION_NAME, () -> TransportResponse.Empty.INSTANCE); + TransportActionProxy.registerProxyAction(transportService, CLEAR_SCROLL_CONTEXTS_ACTION_NAME, + () -> TransportResponse.Empty.INSTANCE); transportService.registerRequestHandler(DFS_ACTION_NAME, ShardSearchTransportRequest::new, ThreadPool.Names.SEARCH, new TaskAwareTransportRequestHandler() { From b0a5212b1e114c889aa8c14c574404d2f3107e02 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 9 Jan 2017 17:34:18 +0100 Subject: [PATCH 33/59] add missing license, forbidden API suppression and line length --- .../transport/TransportActionProxy.java | 18 ++++++++++++++++++ .../search/RemoteClusterConnectionTests.java | 2 ++ .../action/search/SearchAsyncActionTests.java | 9 ++++++--- 3 files changed, 26 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index 7e4927294d2..f74696a9b40 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -1,3 +1,21 @@ +/* + * 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.util.Supplier; diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index 2a8198e46a9..e681a457b01 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.CancellableThreads; @@ -276,6 +277,7 @@ public class RemoteClusterConnectionTests extends ESTestCase { } } + @SuppressForbidden(reason = "calls getLocalHost here but it's fine in this case") public void testSlowNodeCanBeCanceled() throws IOException, InterruptedException { try (ServerSocket socket = new MockServerSocket()) { socket.bind(new InetSocketAddress(InetAddress.getLocalHost(), 0), 1); diff --git a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index 4e2c8e9ca7c..11428b51709 100644 --- a/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -99,9 +99,11 @@ public class SearchAsyncActionTests extends ESTestCase { TestSearchResponse response = new TestSearchResponse(); @Override - protected void sendExecuteFirstPhase(Transport.Connection connection, ShardSearchTransportRequest request, ActionListener listener) { + protected void sendExecuteFirstPhase(Transport.Connection connection, ShardSearchTransportRequest request, + ActionListener listener) { assertTrue("shard: " + request.shardId() + " has been queried twice", response.queried.add(request.shardId())); - TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(contextIdGenerator.incrementAndGet(), connection.getNode()); + TestSearchPhaseResult testSearchPhaseResult = new TestSearchPhaseResult(contextIdGenerator.incrementAndGet(), + connection.getNode()); Set ids = nodeToContextMap.computeIfAbsent(connection.getNode(), (n) -> new HashSet<>()); ids.add(testSearchPhaseResult.id); if (randomBoolean()) { @@ -236,7 +238,8 @@ public class SearchAsyncActionTests extends ESTestCase { } @Override - public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) throws IOException, TransportException { + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws IOException, TransportException { throw new UnsupportedOperationException(); } From 4d47fafd164f2102a6661a9e623e4080d442b760 Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 9 Jan 2017 17:26:07 +0100 Subject: [PATCH 34/59] Adjust RemoteClusterService javadocs --- .../elasticsearch/action/search/RemoteClusterService.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 1e01a6be19b..c25eeaa7340 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -66,7 +66,7 @@ import java.util.function.Predicate; public final class RemoteClusterService extends AbstractComponent implements Closeable { /** - * A list of initial seed nodes to discover eligibale nodes from the remote cluster + * A list of initial seed nodes to discover eligible nodes from the remote cluster */ //TODO this should be an affix settings? public static final Setting REMOTE_CLUSTERS_SEEDS = Setting.groupSetting("search.remote.seeds.", @@ -87,9 +87,10 @@ public final class RemoteClusterService extends AbstractComponent implements Clo Setting.positiveTimeSetting("search.remote.initial_connect_timeout", TimeValue.timeValueSeconds(30), Setting.Property.NodeScope); /** - * The name of a node attribute to filter out nodes that should not be connected to in the remote cluster. + * The name of a node attribute to select nodes that should be connected to in the remote cluster. * For instance a node can be configured with node.node_attr.gateway: true in order to be eligible as a gateway node between - * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster + * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster. + * The value of the setting is expected to be a boolean, for nodes that can become gateways, false otherwise. */ public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", Setting.Property.NodeScope); @@ -326,7 +327,6 @@ public final class RemoteClusterService extends AbstractComponent implements Clo } static void validateRemoteClustersSeeds(Settings settings) { - //TODO do we need a static whitelist like in reindex from remote? for (String clusterName : settings.names()) { String[] remoteHosts = settings.getAsArray(clusterName); if (remoteHosts.length == 0) { From 4d51be125776f8c39bcf88322f7f44e9d6cb85f6 Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 9 Jan 2017 17:35:48 +0100 Subject: [PATCH 35/59] Adjust RemoteClusterConnection javadocs --- .../action/search/RemoteClusterConnection.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 157c3a075fb..b7b6c57661f 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -32,8 +32,6 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.component.AbstractComponent; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -68,7 +66,7 @@ import java.util.function.Predicate; /** * Represents a connection to a single remote cluster. In contrast to a local cluster a remote cluster is not joined such that the - * current node is part of the cluster and it won't receive cluster state updated from the remote cluster. Remote clusters are also not + * current node is part of the cluster and it won't receive cluster state updates from the remote cluster. Remote clusters are also not * fully connected with the current node. From a connection perspective a local cluster forms a bi-directional star network while in the * remote case we only connect to a subset of the nodes in the cluster in an uni-directional fashion. * @@ -157,7 +155,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo public void fetchSearchShards(SearchRequest searchRequest, final List indices, ActionListener listener) { if (connectedNodes.isEmpty()) { - // just in case if we are not connected for some reason we try to connect and if we fail we to and notify the listener + // just in case if we are not connected for some reason we try to connect and if we fail we have to notify the listener // this will cause some back pressure on the search end and eventually will cause rejections but that's fine // we can't proceed with a search on a cluster level. // in the future we might want to just skip the remote nodes in such a case but that can already be implemented on the caller @@ -199,7 +197,6 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo }); } - /** * Returns a connection to the remote cluster. This connection might be a proxy connection that redirects internally to the * given node. @@ -244,7 +241,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo * 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 once the current running job returns. * - * The handler has ab built-in queue that can hold up to 100 connect attempts and will reject request once the queue is full. + * 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 up immediate request but if we can't connect quick enough * we will just reject the connect trigger which will lead to failing searches. */ @@ -419,8 +416,8 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private final Iterator seedNodes; private final CancellableThreads cancellableThreads; - public StateResponseHandler(TransportService transportService, Transport.Connection connection, ActionListener listener - , Iterator seedNodes, CancellableThreads cancellableThreads) { + StateResponseHandler(TransportService transportService, Transport.Connection connection, ActionListener listener, + Iterator seedNodes, CancellableThreads cancellableThreads) { this.transportService = transportService; this.connection = connection; this.listener = listener; @@ -492,5 +489,4 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo assert connectHandler.running.availablePermits() == 1; return true; } - } From 77141716ba737af3d6b3672d442384fcb6b3f11e Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 9 Jan 2017 18:14:26 +0100 Subject: [PATCH 36/59] adjust some javadocs and methods visibility --- .../action/search/RemoteClusterService.java | 4 ++-- .../action/search/SearchTransportService.java | 2 -- .../action/search/TransportSearchAction.java | 2 -- .../elasticsearch/transport/TransportActionProxy.java | 10 +++++----- .../org/elasticsearch/transport/TransportService.java | 2 +- 5 files changed, 8 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index c25eeaa7340..a93ba345a5f 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -108,7 +108,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo } /** - * This method updates the list of remote clusters. it's intendet to be used as a update consumer on the settings infrastructure + * This method updates the list of remote clusters. It's intended to be used as an update consumer on the settings infrastructure * @param seedSettings the group settings returned from {@link #REMOTE_CLUSTERS_SEEDS} * @param connectionListener a listener invoked once every configured cluster has been connected to */ @@ -288,7 +288,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo * Returns a connection to the given node on the given remote cluster * @throws IllegalArgumentException if the remote cluster is unknown */ - Transport.Connection getConnection(DiscoveryNode node, String cluster) { + private Transport.Connection getConnection(DiscoveryNode node, String cluster) { RemoteClusterConnection connection = remoteClusters.get(cluster); if (connection == null) { throw new IllegalArgumentException("no such remote cluster: " + cluster); diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 77754c55bd4..f4ccd837ac6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -411,6 +411,4 @@ public class SearchTransportService extends AbstractLifecycleComponent { protected void doClose() throws IOException { remoteClusterService.close(); } - - } diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 17101b3f552..1b080a66211 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -227,8 +227,6 @@ public class TransportSearchAction extends HandledTransportAction listener) { throw new UnsupportedOperationException("the task parameter is required"); diff --git a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index f74696a9b40..b10c29c008a 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -28,7 +28,7 @@ import java.io.IOException; import java.io.UncheckedIOException; /** - * TransportActionProxy allows an arbitrary action to be executed on a defined target node while the initial request is send to a second + * TransportActionProxy allows an arbitrary action to be executed on a defined target node while the initial request is sent to a second * node that acts as a request proxy to the target node. This is useful if a node is not directly connected to a target node but is * connected to an intermediate node that establishes a transitive connection. */ @@ -42,7 +42,7 @@ public final class TransportActionProxy { private final String action; private final Supplier responseFactory; - public ProxyRequestHandler(TransportService service, String action, Supplier responseFactory) { + ProxyRequestHandler(TransportService service, String action, Supplier responseFactory) { this.service = service; this.action = action; this.responseFactory = responseFactory; @@ -61,7 +61,7 @@ public final class TransportActionProxy { private final Supplier responseFactory; private final TransportChannel channel; - public ProxyResponseHandler(TransportChannel channel, Supplier responseFactory) { + ProxyResponseHandler(TransportChannel channel, Supplier responseFactory) { this.responseFactory = responseFactory; this.channel = channel; @@ -100,11 +100,11 @@ public final class TransportActionProxy { Supplier supplier; DiscoveryNode targetNode; - public ProxyRequest(Supplier supplier) { + ProxyRequest(Supplier supplier) { this.supplier = supplier; } - public ProxyRequest(T wrapped, DiscoveryNode targetNode) { + ProxyRequest(T wrapped, DiscoveryNode targetNode) { this.wrapped = wrapped; this.targetNode = targetNode; } diff --git a/core/src/main/java/org/elasticsearch/transport/TransportService.java b/core/src/main/java/org/elasticsearch/transport/TransportService.java index 28527d938fb..debdea19e01 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportService.java @@ -351,7 +351,7 @@ public class TransportService extends AbstractLifecycleComponent { * Executes a high-level handshake using the given connection * and returns the discovery node of the node the connection * was established with. The handshake will fail if the cluster - * name on the target node mismatches the local cluster name. + * name on the target node doesn't match the local cluster name. * * @param connection the connection to a specific node * @param handshakeTimeout handshake timeout From 56c0bd659daf2aa938ef0aadd9dc0d1415ae5e3f Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 9 Jan 2017 18:17:43 +0100 Subject: [PATCH 37/59] fix couple of typos in the docs --- docs/reference/modules/cross-cluster-search.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index af88ac123e5..5a2a146a4b4 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -3,13 +3,13 @@ The _cross cluster search_ feature allows any node to act as a federated client across multiple clusters. In contrast to the _tribe_ feature, a _cross cluster search_ node won't -join the remote cluster, instead it connects to a remote cluster in a light fashion in order to executed +join the remote cluster, instead it connects to a remote cluster in a light fashion in order to execute federated search requests. The _cross cluster search_ feature works by configuring a remote cluster in the cluster state and connects only to a limited number of nodes in the remote cluster. Each remote cluster is referenced by a name and a list of seed nodes. Those seed nodes are used to discover other nodes eligible as so-called _gateway nodes_. Each node in a cluster that -has remote cluster configured connects to one or more _gateway nodes_ and uses them to federate search requests to +has remote clusters configured connects to one or more _gateway nodes_ and uses them to federate search requests to the remote cluster. Remote clusters can either be configured as part of the `elasticsearch.yml` file or be dynamically updated via From d3d81fcab58a5934dbbb38c7d0976f03002a7f0a Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 9 Jan 2017 18:22:11 +0100 Subject: [PATCH 38/59] fix another typo in the docs --- docs/reference/modules/cross-cluster-search.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index 5a2a146a4b4..ce2bac98dd5 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -58,7 +58,7 @@ POST /cluster_one:twitter,twitter/tweet/_search Search results are disambiguated the same way as the indices are disambiguated in the request. Even if index names are identical these indices will be treated as different indices when results are merged. All results retrieved from a remote index -will be prefixed with it's remote clusters name: +will be prefixed with their remote cluster name: [source,js] -------------------------------------------------- From 5ff029857625a033369330eddd6c678f8d294834 Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 9 Jan 2017 18:30:19 +0100 Subject: [PATCH 39/59] fix typo in RemoteClusterConnection javadocs --- .../elasticsearch/action/search/RemoteClusterConnection.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index b7b6c57661f..f9aeab7cf35 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -239,10 +239,10 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo /** * 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 once the current running job returns. + * 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 up immediate request but if we can't connect quick enough + * 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 { From 22438855d371f2c8ab358b9b311198858ed2a88a Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 9 Jan 2017 21:17:50 +0100 Subject: [PATCH 40/59] fix typo --- .../org/elasticsearch/action/search/RemoteClusterService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index a93ba345a5f..dcc568b58d6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -90,7 +90,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo * The name of a node attribute to select nodes that should be connected to in the remote cluster. * For instance a node can be configured with node.node_attr.gateway: true in order to be eligible as a gateway node between * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster. - * The value of the setting is expected to be a boolean, for nodes that can become gateways, false otherwise. + * The value of the setting is expected to be a boolean, true for nodes that can become gateways, false otherwise. */ public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", Setting.Property.NodeScope); From 7f6c89f9a8fa86ac1ab2727360d20278a5606c60 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 9 Jan 2017 21:27:09 +0100 Subject: [PATCH 41/59] first review round --- .../search/RemoteClusterConnection.java | 35 ++++++++++--------- .../action/search/RemoteClusterService.java | 11 +++--- .../action/search/TransportSearchAction.java | 2 +- 3 files changed, 25 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index f9aeab7cf35..38abcfd79a6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -83,7 +83,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private final ConnectionProfile remoteProfile; private final CopyOnWriteArrayList connectedNodes = new CopyOnWriteArrayList(); private final Supplier nodeSupplier; - private final String clusterName; + private final String clusterAlias; private final int maxNumRemoteConnections; private final Predicate nodePredicate; private volatile List seedNodes; @@ -92,19 +92,19 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo /** * Creates a new {@link RemoteClusterConnection} * @param settings the nodes settings object - * @param clusterName the configured name of the cluster to connect to + * @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 */ - RemoteClusterConnection(Settings settings, String clusterName, List seedNodes, - TransportService transportService, int maxNumRemoteConnections, Predicate nodePredicate) { + RemoteClusterConnection(Settings settings, String clusterAlias, List seedNodes, + TransportService transportService, int maxNumRemoteConnections, Predicate nodePredicate) { super(settings); this.transportService = transportService; this.maxNumRemoteConnections = maxNumRemoteConnections; this.nodePredicate = nodePredicate; - this.clusterName = clusterName; + this.clusterAlias = clusterAlias; ConnectionProfile.Builder builder = new ConnectionProfile.Builder(); builder.setConnectTimeout(TcpTransport.TCP_CONNECT_TIMEOUT.get(settings)); builder.setHandshakeTimeout(TcpTransport.TCP_CONNECT_TIMEOUT.get(settings)); @@ -121,7 +121,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo if (current == null || current.hasNext() == false) { current = connectedNodes.iterator(); if (current.hasNext() == false) { - throw new IllegalStateException("No node available for cluster: " + clusterName + " nodes: " + connectedNodes); + throw new IllegalStateException("No node available for cluster: " + clusterAlias + " nodes: " + connectedNodes); } } return current.next(); @@ -225,10 +225,10 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo } /** - * Returns the name of the remote cluster + * Returns the alias / name of the remote cluster */ - public String getClusterName() { - return clusterName; + public String getClusterAlias() { + return clusterAlias; } @Override @@ -370,7 +370,8 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo request.nodes(true); transportService.sendRequest(connection, ClusterStateAction.NAME, request, TransportRequestOptions.EMPTY, - new StateResponseHandler(transportService, connection, listener, seedNodes, cancellableThreads)); + new SniffClusterStateResponseHandler(transportService, connection, listener, seedNodes, + cancellableThreads)); success = true; } finally { if (success == false) { @@ -387,7 +388,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo // ISE if we fail the handshake with an version incompatible node if (seedNodes.hasNext()) { logger.debug((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), ex); + clusterAlias), ex); collectRemoteNodes(seedNodes, transportService, listener); } else { listener.onFailure(ex); @@ -408,7 +409,8 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo } } - private class StateResponseHandler implements TransportResponseHandler { + /* This class handles the _state response from the remote cluster when sniffing nodes to connect to */ + private class SniffClusterStateResponseHandler implements TransportResponseHandler { private final TransportService transportService; private final Transport.Connection connection; @@ -416,8 +418,9 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private final Iterator seedNodes; private final CancellableThreads cancellableThreads; - StateResponseHandler(TransportService transportService, Transport.Connection connection, ActionListener listener, - Iterator seedNodes, CancellableThreads cancellableThreads) { + SniffClusterStateResponseHandler(TransportService transportService, Transport.Connection connection, + ActionListener listener, Iterator seedNodes, + CancellableThreads cancellableThreads) { this.transportService = transportService; this.connection = connection; this.listener = listener; @@ -457,7 +460,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo } catch (Exception ex) { logger.warn((Supplier) () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", - clusterName), ex); + clusterAlias), ex); collectRemoteNodes(seedNodes, transportService, listener); } finally { // just to make sure we don't leak anything we close the connection here again even if we managed to do so before @@ -468,7 +471,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public void handleException(TransportException exp) { logger.warn((Supplier) - () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterName), + () -> new ParameterizedMessage("fetching nodes from external cluster {} failed", clusterAlias), exp); try { IOUtils.closeWhileHandlingException(connection); diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index dcc568b58d6..b25a5f52277 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -134,16 +134,16 @@ public final class RemoteClusterService extends AbstractComponent implements Clo remoteClusters.put(entry.getKey(), remote); } remote.updateSeedNodes(entry.getValue(), ActionListener.wrap( - x -> { + response -> { if (countDown.countDown()) { - connectionListener.onResponse(x); + connectionListener.onResponse(response); } }, - e -> { + exception -> { if (countDown.fastForward()) { - connectionListener.onFailure(e); + connectionListener.onFailure(exception); } - logger.error("failed to update seed list for cluster: " + entry.getKey(), e); + logger.error("failed to update seed list for cluster: " + entry.getKey(), exception); })); } } @@ -314,7 +314,6 @@ public final class RemoteClusterService extends AbstractComponent implements Clo DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, new TransportAddress(new InetSocketAddress(hostAddress, port)), Version.CURRENT.minimumCompatibilityVersion()); - //don't connect yet as that would require the remote node to be up and would fail the local node startup otherwise List nodes = remoteClustersNodes.get(clusterName); if (nodes == null) { nodes = new ArrayList<>(); diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 1b080a66211..ffe5f07c543 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -204,7 +204,7 @@ public class TransportSearchAction extends HandledTransportAction Date: Tue, 10 Jan 2017 13:20:20 +0100 Subject: [PATCH 42/59] fix generics warning --- .../elasticsearch/action/search/RemoteClusterConnection.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 38abcfd79a6..1ebfcc0bfd1 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -81,7 +81,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private final TransportService transportService; private final ConnectionProfile remoteProfile; - private final CopyOnWriteArrayList connectedNodes = new CopyOnWriteArrayList(); + private final CopyOnWriteArrayList connectedNodes = new CopyOnWriteArrayList<>(); private final Supplier nodeSupplier; private final String clusterAlias; private final int maxNumRemoteConnections; From 4c61f1d75da426b9df1f3bca494d6b53ad271914 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Jan 2017 12:36:00 +0100 Subject: [PATCH 43/59] Cut over to use affix setting for remote cluster configuration Instead of `search.remote.seeds.${clustername}` we now specify the seeds as: `search.remote.${clustername}.seeds` which is a real list setting compared to an unvalidated group setting before. --- .../search/RemoteClusterConnection.java | 11 ++ .../action/search/RemoteClusterService.java | 141 +++++++++--------- .../action/search/SearchTransportService.java | 6 +- .../common/settings/Setting.java | 31 +++- .../elasticsearch/transport/TcpTransport.java | 12 +- .../search/RemoteClusterServiceTests.java | 55 +++++-- .../common/settings/SettingTests.java | 29 ++++ .../modules/cross-cluster-search.asciidoc | 7 +- qa/multi-cluster-search/build.gradle | 2 +- .../test/multi_cluster/10_basic.yaml | 6 +- 10 files changed, 207 insertions(+), 93 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 1ebfcc0bfd1..0dda9ed2ef6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -236,6 +236,10 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo connectHandler.close(); } + public boolean isClosed() { + return connectHandler.isClosed(); + } + /** * 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 @@ -350,6 +354,9 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo void collectRemoteNodes(Iterator seedNodes, final TransportService transportService, ActionListener listener) { + if (Thread.currentThread().isInterrupted()) { + listener.onFailure(new InterruptedException("remote connect thread got interrupted")); + } try { if (seedNodes.hasNext()) { cancellableThreads.executeIO(() -> { @@ -409,6 +416,10 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo } } + 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 { diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index b25a5f52277..26412903981 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -59,6 +59,8 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; /** * Basic service for accessing remote clusters via gateway nodes @@ -68,11 +70,9 @@ public final class RemoteClusterService extends AbstractComponent implements Clo /** * A list of initial seed nodes to discover eligible nodes from the remote cluster */ - //TODO this should be an affix settings? - public static final Setting REMOTE_CLUSTERS_SEEDS = Setting.groupSetting("search.remote.seeds.", - RemoteClusterService::validateRemoteClustersSeeds, - Setting.Property.NodeScope, - Setting.Property.Dynamic); + public static final Setting.AffixSetting> REMOTE_CLUSTERS_SEEDS = Setting.affixKeySetting("search.remote.", + "seeds", (key) -> Setting.listSetting(key, Collections.emptyList(), RemoteClusterService::parseSeedAddress, + Setting.Property.NodeScope, Setting.Property.Dynamic)); /** * The maximum number of connections that will be established to a remote cluster. For instance if there is only a single * seed node, other nodes will be discovered up to the given number of nodes in this setting. The default is 3. @@ -109,12 +109,11 @@ public final class RemoteClusterService extends AbstractComponent implements Clo /** * This method updates the list of remote clusters. It's intended to be used as an update consumer on the settings infrastructure - * @param seedSettings the group settings returned from {@link #REMOTE_CLUSTERS_SEEDS} + * @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 */ - void updateRemoteClusters(Settings seedSettings, ActionListener connectionListener) { + private synchronized void updateRemoteClusters(Map> seeds, ActionListener connectionListener) { Map remoteClusters = new HashMap<>(); - Map> seeds = buildRemoteClustersSeeds(seedSettings); if (seeds.isEmpty()) { connectionListener.onResponse(null); } else { @@ -126,13 +125,27 @@ public final class RemoteClusterService extends AbstractComponent implements Clo String attribute = REMOTE_NODE_ATTRIBUTE.get(settings); nodePredicate = nodePredicate.and((node) -> Boolean.getBoolean(node.getAttributes().getOrDefault(attribute, "false"))); } + remoteClusters.putAll(this.remoteClusters); for (Map.Entry> entry : seeds.entrySet()) { RemoteClusterConnection remote = this.remoteClusters.get(entry.getKey()); - if (remote == null) { + if (entry.getValue().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: " + entry.getKey(), e); + } + remoteClusters.remove(entry.getKey()); + continue; + } + + if (remote == null) { // this is a new cluster we have to add a new representation remote = new RemoteClusterConnection(settings, entry.getKey(), entry.getValue(), transportService, numRemoteConnections, nodePredicate); remoteClusters.put(entry.getKey(), remote); } + + // now update the seed nodes no matter if it's new or already existing + RemoteClusterConnection finalRemote = remote; remote.updateSeedNodes(entry.getValue(), ActionListener.wrap( response -> { if (countDown.countDown()) { @@ -143,14 +156,13 @@ public final class RemoteClusterService extends AbstractComponent implements Clo if (countDown.fastForward()) { connectionListener.onFailure(exception); } - logger.error("failed to update seed list for cluster: " + entry.getKey(), exception); + if (finalRemote.isClosed() == false) { + logger.warn("failed to update seed list for cluster: " + entry.getKey(), exception); + } })); } } - if (remoteClusters.isEmpty() == false) { - remoteClusters.putAll(this.remoteClusters); - this.remoteClusters = Collections.unmodifiableMap(remoteClusters); - } + this.remoteClusters = Collections.unmodifiableMap(remoteClusters); } /** @@ -296,65 +308,55 @@ public final class RemoteClusterService extends AbstractComponent implements Clo return connection.getConnection(node); } - - static Map> buildRemoteClustersSeeds(Settings settings) { - Map> remoteClustersNodes = new HashMap<>(); - for (String clusterName : settings.names()) { - String[] remoteHosts = settings.getAsArray(clusterName); - for (String remoteHost : remoteHosts) { - int portSeparator = remoteHost.lastIndexOf(':'); // in case we have a IPv6 address ie. [::1]:9300 - String host = remoteHost.substring(0, portSeparator); - InetAddress hostAddress; - try { - hostAddress = InetAddress.getByName(host); - } catch (UnknownHostException e) { - throw new IllegalArgumentException("unknown host [" + host + "]", e); - } - int port = Integer.valueOf(remoteHost.substring(portSeparator + 1)); - DiscoveryNode node = new DiscoveryNode(clusterName + "#" + remoteHost, - new TransportAddress(new InetSocketAddress(hostAddress, port)), + public void updateRemoteCluster(String clusterAlias, List addresses) { + updateRemoteClusters(Collections.singletonMap(clusterAlias, addresses.stream().map(address -> { + TransportAddress transportAddress = new TransportAddress(address); + return new DiscoveryNode(clusterAlias + "#" + transportAddress.toString(), + transportAddress, Version.CURRENT.minimumCompatibilityVersion()); - List nodes = remoteClustersNodes.get(clusterName); - if (nodes == null) { - nodes = new ArrayList<>(); - remoteClustersNodes.put(clusterName, nodes); - } - nodes.add(node); - } - } - return remoteClustersNodes; + }).collect(Collectors.toList())), + ActionListener.wrap((x) -> {}, (x) -> {}) ); } - static void validateRemoteClustersSeeds(Settings settings) { - for (String clusterName : settings.names()) { - String[] remoteHosts = settings.getAsArray(clusterName); - if (remoteHosts.length == 0) { - throw new IllegalArgumentException("no hosts set for remote cluster [" + clusterName + "], at least one host is required"); - } - for (String remoteHost : remoteHosts) { - 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 for remote cluster [" + clusterName + "]"); - } - String host = remoteHost.substring(0, portSeparator); - try { - InetAddress.getByName(host); - } catch (UnknownHostException e) { - throw new IllegalArgumentException("unknown host [" + host + "]", e); - } - String port = remoteHost.substring(portSeparator + 1); - try { - Integer portValue = Integer.valueOf(port); - if (portValue <= 0) { - throw new IllegalArgumentException("port number must be > 0 but was: [" + portValue + "]"); - } - } catch (NumberFormatException e) { - throw new IllegalArgumentException("port must be a number, found [" + port + "] instead for remote cluster [" + - clusterName + "]"); - } + static Map> buildRemoteClustersSeeds(Settings settings) { + Stream>> allConcreteSettings = REMOTE_CLUSTERS_SEEDS.getAllConcreteSettings(settings); + return allConcreteSettings.collect( + Collectors.toMap(REMOTE_CLUSTERS_SEEDS::getNamespace, concreteSetting -> { + String clusterName = REMOTE_CLUSTERS_SEEDS.getNamespace(concreteSetting); + List nodes = new ArrayList<>(); + for (InetSocketAddress address : concreteSetting.get(settings)) { + TransportAddress transportAddress = new TransportAddress(address); + DiscoveryNode node = new DiscoveryNode(clusterName + "#" + transportAddress.toString(), + transportAddress, + Version.CURRENT.minimumCompatibilityVersion()); + nodes.add(node); } + return nodes; + })); + } + + static final InetSocketAddress parseSeedAddress(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"); } + String host = remoteHost.substring(0, portSeparator); + InetAddress hostAddress; + try { + hostAddress = InetAddress.getByName(host); + } catch (UnknownHostException e) { + throw new IllegalArgumentException("unknown host [" + host + "]", e); + } + try { + int port = Integer.valueOf(remoteHost.substring(portSeparator + 1)); + if (port <= 0) { + throw new IllegalArgumentException("port number must be > 0 but was: [" + port + "]"); + } + return new InetSocketAddress(hostAddress, port); + } catch (NumberFormatException e) { + throw new IllegalArgumentException("port must be a number"); + } + } /** @@ -364,7 +366,8 @@ public final class RemoteClusterService extends AbstractComponent implements Clo void initializeRemoteClusters() { final TimeValue timeValue = REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING.get(settings); final PlainActionFuture future = new PlainActionFuture<>(); - updateRemoteClusters(REMOTE_CLUSTERS_SEEDS.get(settings), future); + Map> seeds = buildRemoteClustersSeeds(settings); + updateRemoteClusters(seeds, future); try { future.get(timeValue.millis(), TimeUnit.MILLISECONDS); } catch (InterruptedException e) { diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index f4ccd837ac6..2eb6633b1f7 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -83,10 +83,8 @@ public class SearchTransportService extends AbstractLifecycleComponent { super(settings); this.transportService = transportService; this.remoteClusterService = new RemoteClusterService(settings, transportService); - final Consumer clusterUpdateConsumer = (s) -> remoteClusterService.updateRemoteClusters(s, - ActionListener.wrap((x) -> {}, (x) -> {})); - clusterSettings.addSettingsUpdateConsumer(RemoteClusterService.REMOTE_CLUSTERS_SEEDS, clusterUpdateConsumer, - RemoteClusterService::validateRemoteClustersSeeds); + clusterSettings.addAffixUpdateConsumer(RemoteClusterService.REMOTE_CLUSTERS_SEEDS, remoteClusterService::updateRemoteCluster, + (namespace, value) -> {}); } public void sendFreeContext(Transport.Connection connection, final long contextId, SearchRequest request) { diff --git a/core/src/main/java/org/elasticsearch/common/settings/Setting.java b/core/src/main/java/org/elasticsearch/common/settings/Setting.java index d3bc4ebaf0b..5d592b8c452 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/Setting.java +++ b/core/src/main/java/org/elasticsearch/common/settings/Setting.java @@ -42,6 +42,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; +import java.util.HashMap; import java.util.IdentityHashMap; import java.util.List; import java.util.Map; @@ -480,7 +481,7 @@ public class Setting extends ToXContentToBytes { public Map, T> getValue(Settings current, Settings previous) { // we collect all concrete keys and then delegate to the actual setting for validation and settings extraction final Map, T> result = new IdentityHashMap<>(); - Stream.concat(matchStream(current), matchStream(previous)).forEach(aKey -> { + Stream.concat(matchStream(current), matchStream(previous)).distinct().forEach(aKey -> { String namespace = key.getNamespace(aKey); AbstractScopedSettings.SettingUpdater updater = getConcreteSetting(aKey).newUpdater((v) -> consumer.accept(namespace, v), logger, @@ -504,6 +505,18 @@ public class Setting extends ToXContentToBytes { }; } + @Override + public T get(Settings settings) { + throw new UnsupportedOperationException("affix settings can't return values" + + " use #getConcreteSetting to obtain a concrete setting"); + } + + @Override + public String getRaw(Settings settings) { + throw new UnsupportedOperationException("affix settings can't return values" + + " use #getConcreteSetting to obtain a concrete setting"); + } + @Override public Setting getConcreteSetting(String key) { if (match(key)) { @@ -517,6 +530,22 @@ public class Setting extends ToXContentToBytes { public void diff(Settings.Builder builder, Settings source, Settings defaultSettings) { matchStream(defaultSettings).forEach((key) -> getConcreteSetting(key).diff(builder, source, defaultSettings)); } + + /** + * Returns the namespace for a concrete settting. Ie. an affix setting with prefix: search. and suffix: username + * will return remote as a namespace for the setting search.remote.username + */ + public String getNamespace(Setting concreteSetting) { + return key.getNamespace(concreteSetting.getKey()); + } + + /** + * Returns a stream of all concrete setting instances for the given settings. AffixSetting is only a specification, concrete + * settings depend on an actual set of setting keys. + */ + public Stream> getAllConcreteSettings(Settings settings) { + return matchStream(settings).distinct().map(this::getConcreteSetting); + } } diff --git a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java index c2f0832b75e..8be107b1919 100644 --- a/core/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/core/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -477,8 +477,10 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i @Override public final NodeChannels openConnection(DiscoveryNode node, ConnectionProfile connectionProfile) throws IOException { + boolean success = false; + NodeChannels nodeChannels = null; try { - NodeChannels nodeChannels = connectToChannels(node, connectionProfile); + nodeChannels = connectToChannels(node, connectionProfile); final Channel channel = nodeChannels.getChannels().get(0); // one channel is guaranteed by the connection profile final TimeValue connectTimeout = connectionProfile.getConnectTimeout() == null ? defaultConnectionProfile.getConnectTimeout() : @@ -487,13 +489,19 @@ public abstract class TcpTransport extends AbstractLifecycleComponent i connectTimeout : connectionProfile.getHandshakeTimeout(); final Version version = executeHandshake(node, channel, handshakeTimeout); transportServiceAdapter.onConnectionOpened(node); - return new NodeChannels(nodeChannels, version); // clone the channels - we now have the correct version + nodeChannels = new NodeChannels(nodeChannels, version);// clone the channels - we now have the correct version + success = true; + return nodeChannels; } catch (ConnectTransportException e) { throw e; } catch (Exception e) { // ConnectTransportExceptions are handled specifically on the caller end - we wrap the actual exception to ensure // only relevant exceptions are logged on the caller end.. this is the same as in connectToNode throw new ConnectTransportException(node, "general node connection failure", e); + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(nodeChannels); + } } } diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index 68136bfe376..9650b0821dc 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -85,20 +85,20 @@ public class RemoteClusterServiceTests extends ESTestCase { public void testRemoteClusterSeedSetting() { // simple validation - RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() - .put("search.remote.seeds.foo", "192.168.0.1:8080") - .put("search.remote.seeds.bar", "[::1]:9090").build()); + Settings settings = Settings.builder() + .put("search.remote.foo.seeds", "192.168.0.1:8080") + .put("search.remote.bar.seed", "[::1]:9090").build(); + RemoteClusterService.REMOTE_CLUSTERS_SEEDS.getAllConcreteSettings(settings).forEach(setting -> setting.get(settings)); + Settings brokenSettings = Settings.builder() + .put("search.remote.foo.seeds", "192.168.0.1").build(); expectThrows(IllegalArgumentException.class, () -> - RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() - .put("search.remote.seeds.foo", "192.168.0.1").build())); + RemoteClusterService.REMOTE_CLUSTERS_SEEDS.getAllConcreteSettings(brokenSettings).forEach(setting -> setting.get(brokenSettings))); } public void testBuiltRemoteClustersSeeds() throws Exception { Map> map = RemoteClusterService.buildRemoteClustersSeeds( - RemoteClusterService.REMOTE_CLUSTERS_SEEDS.get(Settings.builder() - .put("search.remote.seeds.foo", "192.168.0.1:8080") - .put("search.remote.seeds.bar", "[::1]:9090").build())); + Settings.builder().put("search.remote.foo.seeds", "192.168.0.1:8080").put("search.remote.bar.seeds", "[::1]:9090").build()); assertEquals(2, map.size()); assertTrue(map.containsKey("foo")); assertTrue(map.containsKey("bar")); @@ -133,8 +133,8 @@ public class RemoteClusterServiceTests extends ESTestCase { transportService.start(); transportService.acceptIncomingRequests(); Settings.Builder builder = Settings.builder(); - builder.putArray("search.remote.seeds.cluster_1", seedNode.getAddress().toString()); - builder.putArray("search.remote.seeds.cluster_2", otherSeedNode.getAddress().toString()); + builder.putArray("search.remote.cluster_1.seeds", seedNode.getAddress().toString()); + builder.putArray("search.remote.cluster_2.seeds", otherSeedNode.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); service.initializeRemoteClusters(); @@ -154,6 +154,41 @@ public class RemoteClusterServiceTests extends ESTestCase { } } + public void testIncrementallyAddClusters() throws IOException { + List knownNodes = new CopyOnWriteArrayList<>(); + try (MockTransportService seedTransport = startTransport("cluster_1_node", knownNodes, Version.CURRENT); + MockTransportService otherSeedTransport = startTransport("cluster_2_node", knownNodes, Version.CURRENT)) { + DiscoveryNode seedNode = seedTransport.getLocalDiscoNode(); + DiscoveryNode otherSeedNode = otherSeedTransport.getLocalDiscoNode(); + knownNodes.add(seedTransport.getLocalDiscoNode()); + knownNodes.add(otherSeedTransport.getLocalDiscoNode()); + Collections.shuffle(knownNodes, random()); + + try (MockTransportService transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, + null)) { + transportService.start(); + transportService.acceptIncomingRequests(); + Settings.Builder builder = Settings.builder(); + builder.putArray("search.remote.cluster_1.seeds", seedNode.getAddress().toString()); + builder.putArray("search.remote.cluster_2.seeds", otherSeedNode.getAddress().toString()); + try (RemoteClusterService service = new RemoteClusterService(Settings.EMPTY, transportService)) { + assertFalse(service.isCrossClusterSearchEnabled()); + service.initializeRemoteClusters(); + assertFalse(service.isCrossClusterSearchEnabled()); + service.updateRemoteCluster("cluster_1", Collections.singletonList(seedNode.getAddress().address())); + assertTrue(service.isCrossClusterSearchEnabled()); + assertTrue(service.isRemoteClusterRegistered("cluster_1")); + service.updateRemoteCluster("cluster_2", Collections.singletonList(otherSeedNode.getAddress().address())); + assertTrue(service.isCrossClusterSearchEnabled()); + assertTrue(service.isRemoteClusterRegistered("cluster_1")); + assertTrue(service.isRemoteClusterRegistered("cluster_2")); + service.updateRemoteCluster("cluster_2", Collections.emptyList()); + assertFalse(service.isRemoteClusterRegistered("cluster_2")); + } + } + } + } + public void testProcessRemoteShards() throws IOException { try (RemoteClusterService service = new RemoteClusterService(Settings.EMPTY, null)) { assertFalse(service.isCrossClusterSearchEnabled()); diff --git a/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java b/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java index 4ce23ebcaf0..3789ea40459 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/SettingTests.java @@ -31,6 +31,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -475,6 +477,33 @@ public class SettingTests extends ESTestCase { assertFalse(listAffixSetting.match("foo")); } + public void testGetAllConcreteSettings() { + Setting.AffixSetting> listAffixSetting = Setting.affixKeySetting("foo.", "bar", + (key) -> Setting.listSetting(key, Collections.emptyList(), Function.identity(), Property.NodeScope)); + + Settings settings = Settings.builder() + .putArray("foo.1.bar", "1", "2") + .putArray("foo.2.bar", "3", "4", "5") + .putArray("foo.bar", "6") + .putArray("some.other", "6") + .putArray("foo.3.bar", "6") + .build(); + Stream>> allConcreteSettings = listAffixSetting.getAllConcreteSettings(settings); + Map> collect = allConcreteSettings.collect(Collectors.toMap(Setting::getKey, (s) -> s.get(settings))); + assertEquals(3, collect.size()); + assertEquals(Arrays.asList("1", "2"), collect.get("foo.1.bar")); + assertEquals(Arrays.asList("3", "4", "5"), collect.get("foo.2.bar")); + assertEquals(Arrays.asList("6"), collect.get("foo.3.bar")); + } + + public void testAffixSettingsFailOnGet() { + Setting.AffixSetting> listAffixSetting = Setting.affixKeySetting("foo.", "bar", + (key) -> Setting.listSetting(key, Collections.singletonList("testelement"), Function.identity(), Property.NodeScope)); + expectThrows(UnsupportedOperationException.class, () -> listAffixSetting.get(Settings.EMPTY)); + expectThrows(UnsupportedOperationException.class, () -> listAffixSetting.getRaw(Settings.EMPTY)); + assertEquals(Collections.singletonList("testelement"), listAffixSetting.getDefault(Settings.EMPTY)); + assertEquals("[\"testelement\"]", listAffixSetting.getDefaultRaw(Settings.EMPTY)); + } public void testMinMaxInt() { Setting integerSetting = Setting.intSetting("foo.bar", 1, 0, 10, Property.NodeScope); diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index ce2bac98dd5..18bf597f596 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -24,9 +24,10 @@ remote clusters that should be connected to, for instance: -------------------------------- search: remote: - seeds: - cluster_one: 127.0.0.1:9300 <1> - cluster_two: 127.0.0.1:9301 <1> + cluster_one: <1> + seeds: 127.0.0.1:9300 + cluster_two: <1> + seeds: 127.0.0.1:9301 -------------------------------- <1> `cluster_one` and `cluster_two` are arbitrary names representing the connection to each cluster. These names are subsequently used to distinguish between local and remote indices. diff --git a/qa/multi-cluster-search/build.gradle b/qa/multi-cluster-search/build.gradle index 1abe6e984c3..b48cd9a3dd1 100644 --- a/qa/multi-cluster-search/build.gradle +++ b/qa/multi-cluster-search/build.gradle @@ -35,7 +35,7 @@ task mixedClusterTest(type: RestIntegTestTask) { dependsOn(remoteClusterTest) cluster { distribution = 'zip' - setting 'search.remote.seeds.my_remote_cluster', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" + setting 'search.remote.my_remote_cluster.seeds', "\"${-> remoteClusterTest.nodes.get(0).transportUri()}\"" setting 'search.remote.connections_per_cluster', 1 } diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml index 4ec7747bfee..31b8fbd251e 100644 --- a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/10_basic.yaml @@ -99,16 +99,16 @@ cluster.get_settings: include_defaults: true - - set: { defaults.search.remote.seeds.my_remote_cluster: remote_ip } + - set: { defaults.search.remote.my_remote_cluster.seeds.0: remote_ip } - do: cluster.put_settings: flat_settings: true body: transient: - search.remote.seeds.test_remote_cluster: $remote_ip + search.remote.test_remote_cluster.seeds: $remote_ip - - match: {transient: {search.remote.seeds.test_remote_cluster: $remote_ip}} + - match: {transient: {search.remote.test_remote_cluster.seeds: $remote_ip}} - do: search: From a79896674ad76cbc17ae2a8f6cca56bd152f5bd7 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Jan 2017 14:47:45 +0100 Subject: [PATCH 44/59] Simplify ActionListener helpers and add dedicated unittests --- .../elasticsearch/action/ActionListener.java | 27 ++-- .../action/ActionListenerTests.java | 151 ++++++++++++++++++ 2 files changed, 160 insertions(+), 18 deletions(-) create mode 100644 core/src/test/java/org/elasticsearch/action/ActionListenerTests.java diff --git a/core/src/main/java/org/elasticsearch/action/ActionListener.java b/core/src/main/java/org/elasticsearch/action/ActionListener.java index 19d33707ed7..f9fafa9f95a 100644 --- a/core/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/core/src/main/java/org/elasticsearch/action/ActionListener.java @@ -19,8 +19,11 @@ package org.elasticsearch.action; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.CheckedConsumer; +import java.util.ArrayList; +import java.util.List; import java.util.function.Consumer; /** @@ -72,7 +75,7 @@ public interface ActionListener { * listeners will be processed and the caught exception will be re-thrown. */ static void onResponse(Iterable> listeners, Response response) { - RuntimeException exception = null; + List exceptionList = new ArrayList<>(); for (ActionListener listener : listeners) { try { listener.onResponse(response); @@ -80,17 +83,11 @@ public interface ActionListener { try { listener.onFailure(ex); } catch (Exception ex1) { - if (exception != null) { - exception = new RuntimeException(ex1); - } else { - exception.addSuppressed(ex1); - } + exceptionList.add(ex1); } } } - if (exception != null) { - throw exception; - } + ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptionList); } /** @@ -98,20 +95,14 @@ public interface ActionListener { * all remaining listeners will be processed and the caught exception will be re-thrown. */ static void onFailure(Iterable> listeners, Exception failure) { - RuntimeException exception = null; + List exceptionList = new ArrayList<>(); for (ActionListener listener : listeners) { try { listener.onFailure(failure); } catch (Exception ex) { - if (exception != null) { - exception = new RuntimeException(ex); - } else { - exception.addSuppressed(ex); - } + exceptionList.add(ex); } } - if (exception != null) { - throw exception; - } + ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptionList); } } diff --git a/core/src/test/java/org/elasticsearch/action/ActionListenerTests.java b/core/src/test/java/org/elasticsearch/action/ActionListenerTests.java new file mode 100644 index 00000000000..6414c81058b --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/ActionListenerTests.java @@ -0,0 +1,151 @@ +/* + * 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.action; + +import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.test.ESTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class ActionListenerTests extends ESTestCase { + + public void testWrap() { + AtomicReference reference = new AtomicReference<>(); + AtomicReference exReference = new AtomicReference<>(); + + CheckedConsumer handler = (o) -> { + if (Boolean.FALSE.equals(o)) { + throw new IllegalArgumentException("must not be false"); + } + reference.set(o); + }; + ActionListener wrap = ActionListener.wrap(handler, exReference::set); + wrap.onResponse(Boolean.FALSE); + assertNull(reference.get()); + assertNotNull(exReference.get()); + assertEquals("must not be false", exReference.get().getMessage()); + exReference.set(null); + + wrap.onResponse(Boolean.TRUE); + assertTrue(reference.get()); + assertNull(exReference.get()); + } + + public void testOnResponse() { + final int numListeners = randomIntBetween(1, 20); + List> refList = new ArrayList<>(); + List> excList = new ArrayList<>(); + List> listeners = new ArrayList<>(); + List failOnTrue = new ArrayList<>(); + AtomicInteger exceptionCounter = new AtomicInteger(0); + for (int i = 0; i < numListeners; i++) { + boolean doFailOnTrue = rarely(); + failOnTrue.add(doFailOnTrue); + AtomicReference reference = new AtomicReference<>(); + AtomicReference exReference = new AtomicReference<>(); + refList.add(reference); + excList.add(exReference); + CheckedConsumer handler = (o) -> { + if (Boolean.FALSE.equals(o)) { + throw new IllegalArgumentException("must not be false " + exceptionCounter.getAndIncrement()); + } + if (doFailOnTrue) { + throw new IllegalStateException("must not be true"); + } + reference.set(o); + }; + listeners.add(ActionListener.wrap(handler, exReference::set)); + } + + ActionListener.onResponse(listeners, Boolean.TRUE); + for (int i = 0; i < numListeners; i++) { + if (failOnTrue.get(i) == false) { + assertTrue("listener index " + i, refList.get(i).get()); + refList.get(i).set(null); + } else { + assertNull("listener index " + i, refList.get(i).get()); + } + + } + + for (int i = 0; i < numListeners; i++) { + if (failOnTrue.get(i) == false) { + assertNull("listener index " + i, excList.get(i).get()); + } else { + assertEquals("listener index " + i, "must not be true", excList.get(i).get().getMessage()); + } + } + + ActionListener.onResponse(listeners, Boolean.FALSE); + for (int i = 0; i < numListeners; i++) { + assertNull("listener index " + i, refList.get(i).get()); + } + + assertEquals(numListeners, exceptionCounter.get()); + for (int i = 0; i < numListeners; i++) { + assertNotNull(excList.get(i).get()); + assertEquals("listener index " + i, "must not be false " + i, excList.get(i).get().getMessage()); + } + } + + public void testOnFailure() { + final int numListeners = randomIntBetween(1, 20); + List> refList = new ArrayList<>(); + List> excList = new ArrayList<>(); + List> listeners = new ArrayList<>(); + + final int listenerToFail = randomBoolean() ? -1 : randomIntBetween(0, numListeners-1); + for (int i = 0; i < numListeners; i++) { + AtomicReference reference = new AtomicReference<>(); + AtomicReference exReference = new AtomicReference<>(); + refList.add(reference); + excList.add(exReference); + boolean fail = i == listenerToFail; + CheckedConsumer handler = (o) -> { + reference.set(o); + }; + listeners.add(ActionListener.wrap(handler, (e) -> { + exReference.set(e); + if (fail) { + throw new RuntimeException("double boom"); + } + })); + } + + try { + ActionListener.onFailure(listeners, new Exception("booom")); + assertTrue("unexpected succces listener to fail: " + listenerToFail, listenerToFail == -1); + } catch (RuntimeException ex) { + assertTrue("listener to fail: " + listenerToFail, listenerToFail >= 0); + assertNotNull(ex.getCause()); + assertEquals("double boom", ex.getCause().getMessage()); + } + + for (int i = 0; i < numListeners; i++) { + assertNull("listener index " + i, refList.get(i).get()); + } + + for (int i = 0; i < numListeners; i++) { + assertEquals("listener index " + i, "booom", excList.get(i).get().getMessage()); + } + } +} From e23de3229f89459d0cae290ff9f0e79183333e64 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Jan 2017 14:49:55 +0100 Subject: [PATCH 45/59] add additional explaination to exception handling in AbstractSearchAsyncAction --- .../elasticsearch/action/search/AbstractSearchAsyncAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 2eeef2a02ce..527f400a682 100644 --- a/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -146,7 +146,8 @@ abstract class AbstractSearchAsyncAction }); } catch (ConnectTransportException | IllegalArgumentException ex) { // we are getting the connection early here so we might run into nodes that are not connected. in that case we move on to - // the next shard. + // the next shard. previously when using discovery nodes here we had a special case for null when a node was not connected + // at all which is not not needed anymore. onFirstPhaseResult(shardIndex, shard, shard.currentNodeId(), shardIt, ex); } } From ea0e72f3d9be82a9839a56cd699e182976fa7fd6 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Jan 2017 14:52:41 +0100 Subject: [PATCH 46/59] improve docs based on review commments: --- .../modules/cross-cluster-search.asciidoc | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index 18bf597f596..f34aac193e0 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -14,7 +14,8 @@ the remote cluster. Remote clusters can either be configured as part of the `elasticsearch.yml` file or be dynamically updated via the <>. If a remote cluster is configured via `elasticsearch.yml` only -the nodes with the configuration set will be connecting to the remote cluster. Remote clusters set via the +the nodes with the configuration set will be connecting to the remote cluster in which case federated search requests +will have to be sent specifically to those nodes". Remote clusters set via the <> will be available on every node in the cluster. The `elasticsearch.yml` config file for a _cross cluster search_ node just needs to list the @@ -30,12 +31,13 @@ search: seeds: 127.0.0.1:9301 -------------------------------- -<1> `cluster_one` and `cluster_two` are arbitrary names representing the connection to each cluster. These names are subsequently used to distinguish between local and remote indices. +<1> `cluster_one` and `cluster_two` are arbitrary cluster aliases representing the connection to each cluster. +These names are subsequently used to distinguish between local and remote indices. [float] === Using cross cluster search -To search the `twitter` index on remote cluster `cluster_1` the index name must be prefixed with the cluster name +To search the `twitter` index on remote cluster `cluster_1` the index name must be prefixed with the cluster alias separated by a `:` character: [source,js] @@ -46,8 +48,8 @@ POST /cluster_one:twitter/tweet/_search } -------------------------------------------------- -In contrast to the `tribe` feature cross cluster search can also search indices with the same name on different clusters: - +In contrast to the `tribe` feature cross cluster search can also search indices with the same name on different +clusters: [source,js] -------------------------------------------------- @@ -58,7 +60,8 @@ POST /cluster_one:twitter,twitter/tweet/_search -------------------------------------------------- Search results are disambiguated the same way as the indices are disambiguated in the request. Even if index names are -identical these indices will be treated as different indices when results are merged. All results retrieved from a remote index +identical these indices will be treated as different indices when results are merged. All results retrieved from a +remote index will be prefixed with their remote cluster name: [source,js] @@ -106,8 +109,9 @@ will be prefixed with their remote cluster name: === Cross cluster search settings * `search.remote.connections_per_cluster` - the number of nodes to connect to per remote cluster. The default is `3` -* `search.remote.initial_connect_timeout` - the time to wait for remote connections to be established when the node starts. The default is `30s`. -* `search.remote.node_attribute` - a node attribute to filter out nodes that are eligible as a gateway node in the remote cluster. -For instance a node can have a node attribute `node.attr.gateway: true` such that only nodes with this attribute -will be connected to if `search.remote.node_attribute` is set to `gateway` +* `search.remote.initial_connect_timeout` - the time to wait for remote connections to be established when the node +starts. The default is `30s`. +* `search.remote.node_attribute` - a node attribute to filter out nodes that are eligible as a gateway node in the +remote cluster. For instance a node can have a node attribute `node.attr.gateway: true` such that only nodes with this + attribute will be connected to if `search.remote.node_attribute` is set to `gateway` From d36fc66af14134a2fcf06ac4da55b7b20a166016 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Jan 2017 15:09:29 +0100 Subject: [PATCH 47/59] fix redundant modifier --- .../org/elasticsearch/action/search/RemoteClusterService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 26412903981..107f910597c 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -335,7 +335,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo })); } - static final InetSocketAddress parseSeedAddress(String remoteHost) { + static InetSocketAddress parseSeedAddress(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"); From 46713fb9aaa6caf889f9264e74bdebf244858eba Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Jan 2017 15:10:36 +0100 Subject: [PATCH 48/59] remove leftover --- docs/reference/modules/cross-cluster-search.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index f34aac193e0..dad971c5e67 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -15,7 +15,7 @@ the remote cluster. Remote clusters can either be configured as part of the `elasticsearch.yml` file or be dynamically updated via the <>. If a remote cluster is configured via `elasticsearch.yml` only the nodes with the configuration set will be connecting to the remote cluster in which case federated search requests -will have to be sent specifically to those nodes". Remote clusters set via the +will have to be sent specifically to those nodes. Remote clusters set via the <> will be available on every node in the cluster. The `elasticsearch.yml` config file for a _cross cluster search_ node just needs to list the From bf15decf2056c2b460ee893597fd550bdc96fa43 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 12 Jan 2017 00:21:46 +0100 Subject: [PATCH 49/59] flush pending listeners if remote cluster connection is closed --- .../search/RemoteClusterConnection.java | 1 + .../search/RemoteClusterConnectionTests.java | 32 ++++++++++++------- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 0dda9ed2ef6..79381cc87e4 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -410,6 +410,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo cancellableThreads.cancel("connect handler is closed"); running.acquire(); // acquire the semaphore to ensure all connections are closed and all thread joined running.release(); + maybeConnect(); // now go an notify pending listeners } } catch (InterruptedException e) { Thread.currentThread().interrupt(); diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index e681a457b01..1e13a6c2585 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -464,17 +464,27 @@ public class RemoteClusterConnectionTests extends ESTestCase { 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 || x instanceof AlreadyClosedException) { - // that's fine - } else { - throw new AssertionError(x); - } + AtomicReference executed = new AtomicReference<>(); + ActionListener listener = ActionListener.wrap( + x -> { + if (executed.compareAndSet(null, new RuntimeException())) { + latch.countDown(); + } else { + throw new AssertionError("shit's been called twice", executed.get()); + } + }, + x -> { + if (executed.compareAndSet(null, new RuntimeException())) { + latch.countDown(); + } else { + throw new AssertionError("shit's been called twice", executed.get()); + } + if (x instanceof RejectedExecutionException || x instanceof AlreadyClosedException + || x instanceof CancellableThreads.ExecutionCancelledException) { + // that's fine + } else { + throw new AssertionError(x); + } }); connection.updateSeedNodes(seedNodes, listener); } From a8bd57b93cf6c3ca4ba21cc2f5c4c6e9a1678fd9 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 13 Jan 2017 12:22:52 +0100 Subject: [PATCH 50/59] Fail request if there is a local index that matches the both a remote and local index --- .../action/search/RemoteClusterService.java | 13 +++++++++++-- .../action/search/TransportSearchAction.java | 12 +++++++----- .../action/search/RemoteClusterServiceTests.java | 6 +++++- 3 files changed, 23 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 107f910597c..3a91430b6d9 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -88,7 +88,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo /** * The name of a node attribute to select nodes that should be connected to in the remote cluster. - * For instance a node can be configured with node.node_attr.gateway: true in order to be eligible as a gateway node between + * For instance a node can be configured with node.attr.gateway: true in order to be eligible as a gateway node between * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster. * The value of the setting is expected to be a boolean, true for nodes that can become gateways, false otherwise. */ @@ -177,15 +177,24 @@ public final class RemoteClusterService extends AbstractComponent implements Clo * * @param perClusterIndices a map to fill with remote cluster indices from the given request indices * @param requestIndices the indices in the search request to filter + * @param indexExists a predicate that can test if a certain index or alias exists + * * @return all indices in the requestIndices array that are not remote cluster indices */ - public String[] filterIndices(Map> perClusterIndices, String[] requestIndices) { + public String[] filterIndices(Map> perClusterIndices, String[] requestIndices, Predicate indexExists) { List localIndicesList = new ArrayList<>(); for (String index : requestIndices) { int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); if (i >= 0) { String remoteCluster = index.substring(0, i); if (isRemoteClusterRegistered(remoteCluster)) { + if (indexExists.test(index)) { + // we use : as a separator for remote clusters. might conflict if there is an index that is actually named + // remote_cluster_alias:index_name - for this case we fail the request. the user can easily change the cluster alias + // if that happens + throw new IllegalArgumentException("Index " + index + " exists but there is also a remote cluster named: " + + remoteCluster + " can't filter indices"); + } String remoteIndex = index.substring(i + 1); List indices = perClusterIndices.get(remoteCluster); if (indices == null) { diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index ffe5f07c543..d7441484644 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -122,9 +122,11 @@ public class TransportSearchAction extends HandledTransportAction> remoteIndicesByCluster; + final ClusterState clusterState = clusterService.state(); if (remoteClusterService.isCrossClusterSearchEnabled()) { remoteIndicesByCluster = new HashMap<>(); - localIndices = remoteClusterService.filterIndices(remoteIndicesByCluster, searchRequest.indices()); + localIndices = remoteClusterService.filterIndices(remoteIndicesByCluster, searchRequest.indices(), + idx -> indexNameExpressionResolver.hasIndexOrAlias(idx, clusterState)); } else { remoteIndicesByCluster = Collections.emptyMap(); localIndices = searchRequest.indices(); @@ -132,7 +134,7 @@ public class TransportSearchAction extends HandledTransportAction null, Collections.emptyMap(), listener); + (nodeId) -> null, clusterState, Collections.emptyMap(), listener); } else { remoteClusterService.collectSearchShards(searchRequest, remoteIndicesByCluster, ActionListener.wrap((searchShardsResponses) -> { @@ -141,16 +143,16 @@ public class TransportSearchAction extends HandledTransportAction connectionFunction = remoteClusterService.processRemoteShards( searchShardsResponses, remoteShardIterators, remoteAliasFilters); executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, remoteShardIterators, - connectionFunction, remoteAliasFilters, listener); + connectionFunction, clusterState, remoteAliasFilters, listener); }, listener::onFailure)); } } private void executeSearch(SearchTask task, long startTimeInMillis, SearchRequest searchRequest, String[] localIndices, List remoteShardIterators, Function remoteConnections, - Map remoteAliasMap, ActionListener listener) { + ClusterState clusterState, Map remoteAliasMap, + ActionListener listener) { - ClusterState clusterState = clusterService.state(); clusterState.blocks().globalBlockedRaiseException(ClusterBlockLevel.READ); // TODO: I think startTime() should become part of ActionRequest and that should be used both for index name // date math expressions and $now in scripts. This way all apis will deal with now in the same way instead diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index 9650b0821dc..6c3752d7104 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -144,11 +144,15 @@ public class RemoteClusterServiceTests extends ESTestCase { assertFalse(service.isRemoteClusterRegistered("foo")); Map> perClusterIndices = new HashMap<>(); String[] localIndices = service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", - "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}); + "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> false); assertArrayEquals(new String[]{"foo:bar", "foo"}, localIndices); assertEquals(2, perClusterIndices.size()); assertEquals(Arrays.asList("bar", "test"), perClusterIndices.get("cluster_1")); assertEquals(Arrays.asList("foo:bar", "foo*"), perClusterIndices.get("cluster_2")); + + expectThrows(IllegalArgumentException.class, () -> + service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", + "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> "cluster_1:bar".equals(i))); } } } From 9c167cc92d92306bfe07da5e969a27ef1a2abd84 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 13 Jan 2017 12:25:18 +0100 Subject: [PATCH 51/59] preserve original excetption --- .../org/elasticsearch/action/search/RemoteClusterService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 3a91430b6d9..0600d0999e6 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -363,7 +363,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo } return new InetSocketAddress(hostAddress, port); } catch (NumberFormatException e) { - throw new IllegalArgumentException("port must be a number"); + throw new IllegalArgumentException("port must be a number", e); } } From e2ebabcb3ca550165ec6cfa6c4d4f02181f6b4fc Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 13 Jan 2017 12:25:35 +0100 Subject: [PATCH 52/59] Use a set rather than a list for connected nodes --- .../action/search/RemoteClusterConnection.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 79381cc87e4..63775c17f0f 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -55,8 +55,10 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.ExecutorService; import java.util.concurrent.RejectedExecutionException; @@ -81,7 +83,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo private final TransportService transportService; private final ConnectionProfile remoteProfile; - private final CopyOnWriteArrayList connectedNodes = new CopyOnWriteArrayList<>(); + private final Set connectedNodes = Collections.newSetFromMap(new ConcurrentHashMap<>()); private final Supplier nodeSupplier; private final String clusterAlias; private final int maxNumRemoteConnections; @@ -410,7 +412,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo cancellableThreads.cancel("connect handler is closed"); running.acquire(); // acquire the semaphore to ensure all connections are closed and all thread joined running.release(); - maybeConnect(); // now go an notify pending listeners + maybeConnect(); // now go and notify pending listeners } } catch (InterruptedException e) { Thread.currentThread().interrupt(); From fde11649fbcf8fbc0673a74a83e5b2a7b07f3364 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 13 Jan 2017 23:59:59 +0100 Subject: [PATCH 53/59] harden tests --- .../search/RemoteClusterConnectionTests.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java index 1e13a6c2585..f08f810f33f 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterConnectionTests.java @@ -99,7 +99,6 @@ public class RemoteClusterConnectionTests extends ESTestCase { channel.sendResponse(new ClusterStateResponse(ClusterName.DEFAULT, build)); }); newService.start(); - newService.setLocalNode(new DiscoveryNode(id, newService.boundAddress().publishAddress(), version)); newService.acceptIncomingRequests(); success = true; return newService; @@ -305,11 +304,16 @@ public class RemoteClusterConnectionTests extends ESTestCase { try (MockTransportService service = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool, null)) { service.start(); service.acceptIncomingRequests(); - AtomicReference exceptionAtomicReference = new AtomicReference<>(); + CountDownLatch listenerCalled = new CountDownLatch(1); + AtomicReference exceptionReference = new AtomicReference<>(); try (RemoteClusterConnection connection = new RemoteClusterConnection(Settings.EMPTY, "test-cluster", Arrays.asList(seedNode), service, Integer.MAX_VALUE, n -> true)) { - ActionListener listener = ActionListener.wrap(x -> {}, x -> { - exceptionAtomicReference.set(x); + ActionListener listener = ActionListener.wrap(x -> { + listenerCalled.countDown(); + fail("expected exception"); + }, x -> { + exceptionReference.set(x); + listenerCalled.countDown(); }); connection.updateSeedNodes(Arrays.asList(seedNode), listener); acceptedLatch.await(); @@ -317,7 +321,9 @@ public class RemoteClusterConnectionTests extends ESTestCase { assertTrue(connection.assertNoRunningConnections()); } closeRemote.countDown(); - expectThrows(CancellableThreads.ExecutionCancelledException.class, () -> {throw exceptionAtomicReference.get();}); + listenerCalled.await(); + assertNotNull(exceptionReference.get()); + expectThrows(CancellableThreads.ExecutionCancelledException.class, () -> {throw exceptionReference.get();}); } } From 1c5cc58373e6c28a4688cd9f88db0af04e00f9b9 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Jan 2017 11:46:55 +0100 Subject: [PATCH 54/59] apply review comments --- .../action/search/RemoteClusterConnection.java | 9 +-------- .../action/search/RemoteClusterService.java | 10 +++++----- .../action/search/TransportSearchAction.java | 2 +- .../elasticsearch/transport/TransportActionProxy.java | 2 +- 4 files changed, 8 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java index 63775c17f0f..cce03cc4ed4 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterConnection.java @@ -145,7 +145,7 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo @Override public void onNodeDisconnected(DiscoveryNode node) { boolean remove = connectedNodes.remove(node); - if (remove == true && connectedNodes.size() < maxNumRemoteConnections) { + if (remove && connectedNodes.size() < maxNumRemoteConnections) { // try to reconnect and fill up the slot of the disconnected node connectHandler.forceConnect(); } @@ -226,13 +226,6 @@ final class RemoteClusterConnection extends AbstractComponent implements Transpo }; } - /** - * Returns the alias / name of the remote cluster - */ - public String getClusterAlias() { - return clusterAlias; - } - @Override public void close() throws IOException { connectHandler.close(); diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 0600d0999e6..2286e92c215 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -89,10 +89,10 @@ public final class RemoteClusterService extends AbstractComponent implements Clo /** * The name of a node attribute to select nodes that should be connected to in the remote cluster. * For instance a node can be configured with node.attr.gateway: true in order to be eligible as a gateway node between - * clusters. In that case search.remote.node_attribute: gateway can be used to filter out other nodes in the remote cluster. + * clusters. In that case search.remote.node.attr: gateway can be used to filter out other nodes in the remote cluster. * The value of the setting is expected to be a boolean, true for nodes that can become gateways, false otherwise. */ - public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node_attribute", + public static final Setting REMOTE_NODE_ATTRIBUTE = Setting.simpleString("search.remote.node.attr", Setting.Property.NodeScope); private static final char REMOTE_CLUSTER_INDEX_SEPARATOR = ':'; @@ -181,7 +181,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo * * @return all indices in the requestIndices array that are not remote cluster indices */ - public String[] filterIndices(Map> perClusterIndices, String[] requestIndices, Predicate indexExists) { + String[] filterIndices(Map> perClusterIndices, String[] requestIndices, Predicate indexExists) { List localIndicesList = new ArrayList<>(); for (String index : requestIndices) { int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); @@ -317,7 +317,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo return connection.getConnection(node); } - public void updateRemoteCluster(String clusterAlias, List addresses) { + void updateRemoteCluster(String clusterAlias, List addresses) { updateRemoteClusters(Collections.singletonMap(clusterAlias, addresses.stream().map(address -> { TransportAddress transportAddress = new TransportAddress(address); return new DiscoveryNode(clusterAlias + "#" + transportAddress.toString(), @@ -344,7 +344,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo })); } - static InetSocketAddress parseSeedAddress(String remoteHost) { + private static InetSocketAddress parseSeedAddress(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"); diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index d7441484644..9710b4931e9 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -159,7 +159,7 @@ public class TransportSearchAction extends HandledTransportAction 0) { - indices = new Index[0]; // don't search on _all if only remote indices were specified + indices = Index.EMPTY_ARRAY; // don't search on _all if only remote indices were specified } else { indices = indexNameExpressionResolver.concreteIndices(clusterState, searchRequest.indicesOptions(), startTimeInMillis, localIndices); diff --git a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java index b10c29c008a..30b7299f59d 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportActionProxy.java @@ -100,7 +100,7 @@ public final class TransportActionProxy { Supplier supplier; DiscoveryNode targetNode; - ProxyRequest(Supplier supplier) { + ProxyRequest(Supplier supplier) { this.supplier = supplier; } From d7eee637d9f69362f71b45ace744fefbf937d003 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Jan 2017 11:47:29 +0100 Subject: [PATCH 55/59] fix some docs issues --- docs/reference/modules.asciidoc | 2 +- docs/reference/modules/cross-cluster-search.asciidoc | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/reference/modules.asciidoc b/docs/reference/modules.asciidoc index 0e55b858359..2699f425c99 100644 --- a/docs/reference/modules.asciidoc +++ b/docs/reference/modules.asciidoc @@ -80,7 +80,7 @@ The modules in this section are: <>:: - Cross cluster search allows to execute search requests across more than one cluster without joining them and acts + Cross cluster search enables executing search requests across more than one cluster without joining them and acts as a federated client across them. -- diff --git a/docs/reference/modules/cross-cluster-search.asciidoc b/docs/reference/modules/cross-cluster-search.asciidoc index dad971c5e67..7bc3cbf388e 100644 --- a/docs/reference/modules/cross-cluster-search.asciidoc +++ b/docs/reference/modules/cross-cluster-search.asciidoc @@ -111,7 +111,7 @@ will be prefixed with their remote cluster name: * `search.remote.connections_per_cluster` - the number of nodes to connect to per remote cluster. The default is `3` * `search.remote.initial_connect_timeout` - the time to wait for remote connections to be established when the node starts. The default is `30s`. -* `search.remote.node_attribute` - a node attribute to filter out nodes that are eligible as a gateway node in the +* `search.remote.node.attr` - a node attribute to filter out nodes that are eligible as a gateway node in the remote cluster. For instance a node can have a node attribute `node.attr.gateway: true` such that only nodes with this - attribute will be connected to if `search.remote.node_attribute` is set to `gateway` + attribute will be connected to if `search.remote.node.attr` is set to `gateway` From 88f6ae55f5c0dface051e4c1e9c0c57ecac38d68 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Jan 2017 14:05:36 +0100 Subject: [PATCH 56/59] Improve remote / local indices filtering by not modifying external state --- .../action/search/RemoteClusterService.java | 40 +++++++++---------- .../action/search/TransportSearchAction.java | 14 ++++--- .../search/RemoteClusterServiceTests.java | 23 ++++------- 3 files changed, 36 insertions(+), 41 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 2286e92c215..ddb07bbcecb 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -67,6 +67,8 @@ import java.util.stream.Stream; */ public final class RemoteClusterService extends AbstractComponent implements Closeable { + public static final String LOCAL_CLUSTER_GROUP_KEY = ""; + /** * A list of initial seed nodes to discover eligible nodes from the remote cluster */ @@ -113,6 +115,9 @@ public final class RemoteClusterService extends AbstractComponent implements Clo * @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 have the empty string as it's key"); + } Map remoteClusters = new HashMap<>(); if (seeds.isEmpty()) { connectionListener.onResponse(null); @@ -173,43 +178,38 @@ public final class RemoteClusterService extends AbstractComponent implements Clo } /** - * Filters out indices that refer to a remote cluster and adds them to the given per cluster indices map. + * Groups indices per cluster by splitting remote cluster-alias, index-name pairs on {@link #REMOTE_CLUSTER_INDEX_SEPARATOR}. All + * indices per cluster are collected as a list in the returned map keyed by the cluster alias. Local indices are grouped under + * {@link #LOCAL_CLUSTER_GROUP_KEY}. The returned map is mutable. * - * @param perClusterIndices a map to fill with remote cluster indices from the given request indices * @param requestIndices the indices in the search request to filter * @param indexExists a predicate that can test if a certain index or alias exists * - * @return all indices in the requestIndices array that are not remote cluster indices + * @return a map of grouped remote and local indices */ - String[] filterIndices(Map> perClusterIndices, String[] requestIndices, Predicate indexExists) { - List localIndicesList = new ArrayList<>(); + Map> groupClusterIndices(String[] requestIndices, Predicate indexExists) { + Map> perClusterIndices = new HashMap<>(); for (String index : requestIndices) { int i = index.indexOf(REMOTE_CLUSTER_INDEX_SEPARATOR); + String indexName = index; + String clusterName = LOCAL_CLUSTER_GROUP_KEY; if (i >= 0) { - String remoteCluster = index.substring(0, i); - if (isRemoteClusterRegistered(remoteCluster)) { + String remoteClusterName = index.substring(0, i); + if (isRemoteClusterRegistered(remoteClusterName)) { if (indexExists.test(index)) { // we use : as a separator for remote clusters. might conflict if there is an index that is actually named // remote_cluster_alias:index_name - for this case we fail the request. the user can easily change the cluster alias // if that happens throw new IllegalArgumentException("Index " + index + " exists but there is also a remote cluster named: " - + remoteCluster + " can't filter indices"); + + clusterName + " can't filter indices"); } - String remoteIndex = index.substring(i + 1); - List indices = perClusterIndices.get(remoteCluster); - if (indices == null) { - indices = new ArrayList<>(); - perClusterIndices.put(remoteCluster, indices); - } - indices.add(remoteIndex); - } else { - localIndicesList.add(index); + indexName = index.substring(i + 1); + clusterName = remoteClusterName; } - } else { - localIndicesList.add(index); } + perClusterIndices.computeIfAbsent(clusterName, k -> new ArrayList()).add(indexName); } - return localIndicesList.toArray(new String[localIndicesList.size()]); + return perClusterIndices; } /** diff --git a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 9710b4931e9..f67fc543425 100644 --- a/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/core/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; @@ -121,22 +122,23 @@ public class TransportSearchAction extends HandledTransportAction> remoteIndicesByCluster; + final Map> remoteClusterIndices; final ClusterState clusterState = clusterService.state(); if (remoteClusterService.isCrossClusterSearchEnabled()) { - remoteIndicesByCluster = new HashMap<>(); - localIndices = remoteClusterService.filterIndices(remoteIndicesByCluster, searchRequest.indices(), + remoteClusterIndices = remoteClusterService.groupClusterIndices( searchRequest.indices(), // empty string is not allowed idx -> indexNameExpressionResolver.hasIndexOrAlias(idx, clusterState)); + List remove = remoteClusterIndices.remove(RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY); + localIndices = remove == null ? Strings.EMPTY_ARRAY : remove.toArray(new String[remove.size()]); } else { - remoteIndicesByCluster = Collections.emptyMap(); + remoteClusterIndices = Collections.emptyMap(); localIndices = searchRequest.indices(); } - if (remoteIndicesByCluster.isEmpty()) { + if (remoteClusterIndices.isEmpty()) { executeSearch((SearchTask)task, startTimeInMillis, searchRequest, localIndices, Collections.emptyList(), (nodeId) -> null, clusterState, Collections.emptyMap(), listener); } else { - remoteClusterService.collectSearchShards(searchRequest, remoteIndicesByCluster, + remoteClusterService.collectSearchShards(searchRequest, remoteClusterIndices, ActionListener.wrap((searchShardsResponses) -> { List remoteShardIterators = new ArrayList<>(); Map remoteAliasFilters = new HashMap<>(); diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index 6c3752d7104..bdbb350596f 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -19,20 +19,11 @@ package org.elasticsearch.action.search; import org.elasticsearch.Version; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; -import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; -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.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Strings; @@ -47,7 +38,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.Transport; import java.io.IOException; import java.net.InetAddress; @@ -60,7 +50,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; -import java.util.function.Function; public class RemoteClusterServiceTests extends ESTestCase { @@ -118,7 +107,7 @@ public class RemoteClusterServiceTests extends ESTestCase { } - public void testFilterIndices() throws IOException { + public void testGroupClusterIndices() throws IOException { List knownNodes = new CopyOnWriteArrayList<>(); try (MockTransportService seedTransport = startTransport("cluster_1_node", knownNodes, Version.CURRENT); MockTransportService otherSeedTransport = startTransport("cluster_2_node", knownNodes, Version.CURRENT)) { @@ -142,16 +131,18 @@ public class RemoteClusterServiceTests extends ESTestCase { assertTrue(service.isRemoteClusterRegistered("cluster_1")); assertTrue(service.isRemoteClusterRegistered("cluster_2")); assertFalse(service.isRemoteClusterRegistered("foo")); - Map> perClusterIndices = new HashMap<>(); - String[] localIndices = service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", + Map> perClusterIndices = service.groupClusterIndices(new String[]{"foo:bar", "cluster_1:bar", "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> false); + String[] localIndices = perClusterIndices.computeIfAbsent(RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY, + k -> Collections.emptyList()).toArray(new String[0]); + assertNotNull(perClusterIndices.remove(RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY)); assertArrayEquals(new String[]{"foo:bar", "foo"}, localIndices); assertEquals(2, perClusterIndices.size()); assertEquals(Arrays.asList("bar", "test"), perClusterIndices.get("cluster_1")); assertEquals(Arrays.asList("foo:bar", "foo*"), perClusterIndices.get("cluster_2")); expectThrows(IllegalArgumentException.class, () -> - service.filterIndices(perClusterIndices, new String[]{"foo:bar", "cluster_1:bar", + service.groupClusterIndices(new String[]{"foo:bar", "cluster_1:bar", "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> "cluster_1:bar".equals(i))); } } @@ -188,6 +179,8 @@ public class RemoteClusterServiceTests extends ESTestCase { assertTrue(service.isRemoteClusterRegistered("cluster_2")); service.updateRemoteCluster("cluster_2", Collections.emptyList()); assertFalse(service.isRemoteClusterRegistered("cluster_2")); + expectThrows(IllegalArgumentException.class, + () -> service.updateRemoteCluster(RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY, Collections.emptyList())); } } } From 197cd7d7a93788f7fee3ebb7c6af9159554c5230 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Jan 2017 14:13:09 +0100 Subject: [PATCH 57/59] Add test for the grouping error message if indices and cluster can't be disambiguated --- .../action/search/RemoteClusterService.java | 2 +- .../action/search/RemoteClusterServiceTests.java | 9 ++++++--- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index ddb07bbcecb..81c1982c9e3 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -201,7 +201,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo // remote_cluster_alias:index_name - for this case we fail the request. the user can easily change the cluster alias // if that happens throw new IllegalArgumentException("Index " + index + " exists but there is also a remote cluster named: " - + clusterName + " can't filter indices"); + + remoteClusterName + " can't filter indices"); } indexName = index.substring(i + 1); clusterName = remoteClusterName; diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index bdbb350596f..d65ad1143fb 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -141,9 +141,12 @@ public class RemoteClusterServiceTests extends ESTestCase { assertEquals(Arrays.asList("bar", "test"), perClusterIndices.get("cluster_1")); assertEquals(Arrays.asList("foo:bar", "foo*"), perClusterIndices.get("cluster_2")); - expectThrows(IllegalArgumentException.class, () -> - service.groupClusterIndices(new String[]{"foo:bar", "cluster_1:bar", - "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> "cluster_1:bar".equals(i))); + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> + service.groupClusterIndices(new String[]{"foo:bar", "cluster_1:bar", + "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> "cluster_1:bar".equals(i))); + + assertEquals("Index cluster_1:bar exists but there is also a remote cluster named: cluster_1 can't filter indices", + iae.getMessage()); } } } From 292e3a60d148bfad76e8a0f0886d4665add8260e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Jan 2017 17:20:52 +0100 Subject: [PATCH 58/59] apply review comments --- .../action/search/RemoteClusterService.java | 8 ++++---- .../action/search/RemoteClusterServiceTests.java | 9 ++++----- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 81c1982c9e3..55202556d28 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -67,7 +67,7 @@ import java.util.stream.Stream; */ public final class RemoteClusterService extends AbstractComponent implements Closeable { - public static final String LOCAL_CLUSTER_GROUP_KEY = ""; + static final String LOCAL_CLUSTER_GROUP_KEY = ""; /** * A list of initial seed nodes to discover eligible nodes from the remote cluster @@ -116,7 +116,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo */ private synchronized void updateRemoteClusters(Map> seeds, ActionListener connectionListener) { if (seeds.containsKey(LOCAL_CLUSTER_GROUP_KEY)) { - throw new IllegalArgumentException("remote clusters must have the empty string as it's key"); + throw new IllegalArgumentException("remote clusters must not have the empty string as its key"); } Map remoteClusters = new HashMap<>(); if (seeds.isEmpty()) { @@ -200,8 +200,8 @@ public final class RemoteClusterService extends AbstractComponent implements Clo // we use : as a separator for remote clusters. might conflict if there is an index that is actually named // remote_cluster_alias:index_name - for this case we fail the request. the user can easily change the cluster alias // if that happens - throw new IllegalArgumentException("Index " + index + " exists but there is also a remote cluster named: " - + remoteClusterName + " can't filter indices"); + throw new IllegalArgumentException("Can not filter indices; index " + index + + " exists but there is also a remote cluster named: " + remoteClusterName + " can't filter indices"); } indexName = index.substring(i + 1); clusterName = remoteClusterName; diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index d65ad1143fb..57b0a341554 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -145,8 +145,8 @@ public class RemoteClusterServiceTests extends ESTestCase { service.groupClusterIndices(new String[]{"foo:bar", "cluster_1:bar", "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> "cluster_1:bar".equals(i))); - assertEquals("Index cluster_1:bar exists but there is also a remote cluster named: cluster_1 can't filter indices", - iae.getMessage()); + assertEquals("Can not filter indices; index cluster_1:bar exists but there is also a remote cluster named:" + + " cluster_1 can't filter indices", iae.getMessage()); } } } @@ -182,8 +182,9 @@ public class RemoteClusterServiceTests extends ESTestCase { assertTrue(service.isRemoteClusterRegistered("cluster_2")); service.updateRemoteCluster("cluster_2", Collections.emptyList()); assertFalse(service.isRemoteClusterRegistered("cluster_2")); - expectThrows(IllegalArgumentException.class, + IllegalArgumentException iae = expectThrows(IllegalArgumentException.class, () -> service.updateRemoteCluster(RemoteClusterService.LOCAL_CLUSTER_GROUP_KEY, Collections.emptyList())); + assertEquals("remote clusters must not have the empty string as its key", iae.getMessage()); } } } @@ -226,7 +227,6 @@ public class RemoteClusterServiceTests extends ESTestCase { shardRouting = iterator.nextOrNull(); assertNotNull(shardRouting); assertEquals(shardRouting.getIndexName(), "foo"); - assertNull(iterator.nextOrNull()); } else { assertEquals(0, iterator.shardId().getId()); @@ -237,7 +237,6 @@ public class RemoteClusterServiceTests extends ESTestCase { shardRouting = iterator.nextOrNull(); assertNotNull(shardRouting); assertEquals(shardRouting.getIndexName(), "bar"); - assertNull(iterator.nextOrNull()); } } From 69f1ffb1f806f4172b678c304834ffce66fb6679 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Jan 2017 17:29:43 +0100 Subject: [PATCH 59/59] fix exception message --- .../org/elasticsearch/action/search/RemoteClusterService.java | 2 +- .../elasticsearch/action/search/RemoteClusterServiceTests.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java index 55202556d28..92b4ca5f0b3 100644 --- a/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java +++ b/core/src/main/java/org/elasticsearch/action/search/RemoteClusterService.java @@ -201,7 +201,7 @@ public final class RemoteClusterService extends AbstractComponent implements Clo // remote_cluster_alias:index_name - for this case we fail the request. the user can easily change the cluster alias // if that happens throw new IllegalArgumentException("Can not filter indices; index " + index + - " exists but there is also a remote cluster named: " + remoteClusterName + " can't filter indices"); + " exists but there is also a remote cluster named: " + remoteClusterName); } indexName = index.substring(i + 1); clusterName = remoteClusterName; diff --git a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java index 57b0a341554..1531d66e5da 100644 --- a/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/RemoteClusterServiceTests.java @@ -146,7 +146,7 @@ public class RemoteClusterServiceTests extends ESTestCase { "cluster_2:foo:bar", "cluster_1:test", "cluster_2:foo*", "foo"}, i -> "cluster_1:bar".equals(i))); assertEquals("Can not filter indices; index cluster_1:bar exists but there is also a remote cluster named:" + - " cluster_1 can't filter indices", iae.getMessage()); + " cluster_1", iae.getMessage()); } } }