|
@@ -48,9 +48,13 @@ import org.elasticsearch.index.shard.ShardId;
|
|
|
import org.elasticsearch.search.SearchPhaseResult;
|
|
|
import org.elasticsearch.search.SearchService;
|
|
|
import org.elasticsearch.search.aggregations.InternalAggregation;
|
|
|
+import org.elasticsearch.search.aggregations.InternalAggregations;
|
|
|
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
|
|
import org.elasticsearch.search.internal.AliasFilter;
|
|
|
+import org.elasticsearch.search.internal.InternalSearchResponse;
|
|
|
import org.elasticsearch.search.internal.SearchContext;
|
|
|
+import org.elasticsearch.search.profile.ProfileShardResult;
|
|
|
+import org.elasticsearch.search.profile.SearchProfileShardResults;
|
|
|
import org.elasticsearch.tasks.Task;
|
|
|
import org.elasticsearch.threadpool.ThreadPool;
|
|
|
import org.elasticsearch.transport.RemoteClusterAware;
|
|
@@ -253,30 +257,66 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|
|
SearchTimeProvider timeProvider, Function<Boolean, InternalAggregation.ReduceContext> reduceContext,
|
|
|
RemoteClusterService remoteClusterService, ThreadPool threadPool, ActionListener<SearchResponse> listener,
|
|
|
BiConsumer<SearchRequest, ActionListener<SearchResponse>> localSearchConsumer) {
|
|
|
- SearchResponseMerger searchResponseMerger = createSearchResponseMerger(searchRequest.source(), timeProvider, reduceContext);
|
|
|
- AtomicInteger skippedClusters = new AtomicInteger(0);
|
|
|
- final AtomicReference<Exception> exceptions = new AtomicReference<>();
|
|
|
- int totalClusters = remoteIndices.size() + (localIndices == null ? 0 : 1);
|
|
|
- final CountDown countDown = new CountDown(totalClusters);
|
|
|
- for (Map.Entry<String, OriginalIndices> entry : remoteIndices.entrySet()) {
|
|
|
+
|
|
|
+ if (localIndices == null && remoteIndices.size() == 1) {
|
|
|
+ //if we are searching against a single remote cluster, we simply forward the original search request to such cluster
|
|
|
+ //and we directly perform final reduction in the remote cluster
|
|
|
+ Map.Entry<String, OriginalIndices> entry = remoteIndices.entrySet().iterator().next();
|
|
|
String clusterAlias = entry.getKey();
|
|
|
boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias);
|
|
|
OriginalIndices indices = entry.getValue();
|
|
|
SearchRequest ccsSearchRequest = SearchRequest.withLocalReduction(searchRequest, indices.indices(),
|
|
|
- clusterAlias, timeProvider.getAbsoluteStartMillis());
|
|
|
- ActionListener<SearchResponse> ccsListener = createCCSListener(clusterAlias, skipUnavailable, countDown,
|
|
|
- skippedClusters, exceptions, searchResponseMerger, totalClusters, listener);
|
|
|
+ clusterAlias, timeProvider.getAbsoluteStartMillis(), true);
|
|
|
Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias);
|
|
|
- remoteClusterClient.search(ccsSearchRequest, ccsListener);
|
|
|
- }
|
|
|
- if (localIndices != null) {
|
|
|
- ActionListener<SearchResponse> ccsListener = createCCSListener(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY,
|
|
|
- false, countDown, skippedClusters, exceptions, searchResponseMerger, totalClusters, listener);
|
|
|
- //here we provide the empty string a cluster alias, which means no prefix in index name,
|
|
|
- //but the coord node will perform non final reduce as it's not null.
|
|
|
- SearchRequest ccsLocalSearchRequest = SearchRequest.withLocalReduction(searchRequest, localIndices.indices(),
|
|
|
- RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis());
|
|
|
- localSearchConsumer.accept(ccsLocalSearchRequest, ccsListener);
|
|
|
+ remoteClusterClient.search(ccsSearchRequest, new ActionListener<SearchResponse>() {
|
|
|
+ @Override
|
|
|
+ public void onResponse(SearchResponse searchResponse) {
|
|
|
+ Map<String, ProfileShardResult> profileResults = searchResponse.getProfileResults();
|
|
|
+ SearchProfileShardResults profile = profileResults == null || profileResults.isEmpty()
|
|
|
+ ? null : new SearchProfileShardResults(profileResults);
|
|
|
+ InternalSearchResponse internalSearchResponse = new InternalSearchResponse(searchResponse.getHits(),
|
|
|
+ (InternalAggregations) searchResponse.getAggregations(), searchResponse.getSuggest(), profile,
|
|
|
+ searchResponse.isTimedOut(), searchResponse.isTerminatedEarly(), searchResponse.getNumReducePhases());
|
|
|
+ listener.onResponse(new SearchResponse(internalSearchResponse, searchResponse.getScrollId(),
|
|
|
+ searchResponse.getTotalShards(), searchResponse.getSuccessfulShards(), searchResponse.getSkippedShards(),
|
|
|
+ timeProvider.buildTookInMillis(), searchResponse.getShardFailures(), new SearchResponse.Clusters(1, 1, 0)));
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void onFailure(Exception e) {
|
|
|
+ if (skipUnavailable) {
|
|
|
+ listener.onResponse(SearchResponse.empty(timeProvider::buildTookInMillis, new SearchResponse.Clusters(1, 0, 1)));
|
|
|
+ } else {
|
|
|
+ listener.onFailure(wrapRemoteClusterFailure(clusterAlias, e));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
+ } else {
|
|
|
+ SearchResponseMerger searchResponseMerger = createSearchResponseMerger(searchRequest.source(), timeProvider, reduceContext);
|
|
|
+ AtomicInteger skippedClusters = new AtomicInteger(0);
|
|
|
+ final AtomicReference<Exception> exceptions = new AtomicReference<>();
|
|
|
+ int totalClusters = remoteIndices.size() + (localIndices == null ? 0 : 1);
|
|
|
+ final CountDown countDown = new CountDown(totalClusters);
|
|
|
+ for (Map.Entry<String, OriginalIndices> entry : remoteIndices.entrySet()) {
|
|
|
+ String clusterAlias = entry.getKey();
|
|
|
+ boolean skipUnavailable = remoteClusterService.isSkipUnavailable(clusterAlias);
|
|
|
+ OriginalIndices indices = entry.getValue();
|
|
|
+ SearchRequest ccsSearchRequest = SearchRequest.withLocalReduction(searchRequest, indices.indices(),
|
|
|
+ clusterAlias, timeProvider.getAbsoluteStartMillis(), false);
|
|
|
+ ActionListener<SearchResponse> ccsListener = createCCSListener(clusterAlias, skipUnavailable, countDown,
|
|
|
+ skippedClusters, exceptions, searchResponseMerger, totalClusters, listener);
|
|
|
+ Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias);
|
|
|
+ remoteClusterClient.search(ccsSearchRequest, ccsListener);
|
|
|
+ }
|
|
|
+ if (localIndices != null) {
|
|
|
+ ActionListener<SearchResponse> ccsListener = createCCSListener(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY,
|
|
|
+ false, countDown, skippedClusters, exceptions, searchResponseMerger, totalClusters, listener);
|
|
|
+ //here we provide the empty string a cluster alias, which means no prefix in index name,
|
|
|
+ //but the coord node will perform non final reduce as it's not null.
|
|
|
+ SearchRequest ccsLocalSearchRequest = SearchRequest.withLocalReduction(searchRequest, localIndices.indices(),
|
|
|
+ RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY, timeProvider.getAbsoluteStartMillis(), false);
|
|
|
+ localSearchConsumer.accept(ccsLocalSearchRequest, ccsListener);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -297,9 +337,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|
|
//here we modify the original source so we can re-use it by setting it to each outgoing search request
|
|
|
source.from(0);
|
|
|
source.size(from + size);
|
|
|
- //TODO when searching only against a remote cluster, we could ask directly for the final number of results and let
|
|
|
- //the remote cluster do a final reduction, yet that is not possible as we are providing a localClusterAlias which
|
|
|
- //will automatically make the reduction non final
|
|
|
}
|
|
|
return new SearchResponseMerger(from, size, trackTotalHitsUpTo, timeProvider, reduceContextFunction);
|
|
|
}
|
|
@@ -604,7 +641,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|
|
} else {
|
|
|
Exception exception = e;
|
|
|
if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias) == false) {
|
|
|
- exception = new RemoteTransportException("error while communicating with remote cluster [" + clusterAlias + "]", e);
|
|
|
+ exception = wrapRemoteClusterFailure(clusterAlias, e);
|
|
|
}
|
|
|
if (exceptions.compareAndSet(null, exception) == false) {
|
|
|
exceptions.accumulateAndGet(exception, (previous, current) -> {
|
|
@@ -636,4 +673,8 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|
|
|
|
|
abstract FinalResponse createFinalResponse();
|
|
|
}
|
|
|
+
|
|
|
+ private static RemoteTransportException wrapRemoteClusterFailure(String clusterAlias, Exception e) {
|
|
|
+ return new RemoteTransportException("error while communicating with remote cluster [" + clusterAlias + "]", e);
|
|
|
+ }
|
|
|
}
|