Skip to content

Commit

Permalink
Move states of search to coordinating node (#52741)
Browse files Browse the repository at this point in the history
This commit moves the states of search to the coordinating node instead 
of keeping them in the data node. 

Relates #46523
  • Loading branch information
dnhatn authored Mar 3, 2020
1 parent f921838 commit 206381e
Show file tree
Hide file tree
Showing 28 changed files with 505 additions and 171 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,13 +36,17 @@
import org.elasticsearch.index.query.MultiMatchQueryBuilder;
import org.elasticsearch.index.query.Operator;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder;
import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;

import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.common.xcontent.XContentFactory.smileBuilder;
Expand Down Expand Up @@ -71,6 +75,14 @@

public class PercolatorQuerySearchIT extends ESIntegTestCase {

@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
List<Class<? extends Plugin>> plugins = new ArrayList<>();
plugins.addAll(super.nodePlugins());
plugins.add(PercolatorPlugin.class);
return plugins;
}

public void testPercolatorQuery() throws Exception {
assertAcked(client().admin().indices().prepareCreate("test")
.setMapping("id", "type=keyword", "field1", "type=keyword", "field2", "type=keyword", "query", "type=percolator")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
package org.elasticsearch.action.search;

import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.builder.SearchSourceBuilder;
Expand All @@ -43,21 +42,24 @@
final class DfsQueryPhase extends SearchPhase {
private final ArraySearchPhaseResults<SearchPhaseResult> queryResult;
private final SearchPhaseController searchPhaseController;
private final AtomicArray<DfsSearchResult> dfsSearchResults;
private final List<DfsSearchResult> searchResults;
private final AggregatedDfs dfs;
private final Function<ArraySearchPhaseResults<SearchPhaseResult>, SearchPhase> nextPhaseFactory;
private final SearchPhaseContext context;
private final SearchTransportService searchTransportService;
private final SearchProgressListener progressListener;

DfsQueryPhase(AtomicArray<DfsSearchResult> dfsSearchResults,
DfsQueryPhase(List<DfsSearchResult> searchResults,
AggregatedDfs dfs,
SearchPhaseController searchPhaseController,
Function<ArraySearchPhaseResults<SearchPhaseResult>, SearchPhase> nextPhaseFactory,
SearchPhaseContext context) {
super("dfs_query");
this.progressListener = context.getTask().getProgressListener();
this.queryResult = searchPhaseController.newSearchPhaseResults(progressListener, context.getRequest(), context.getNumShards());
this.searchPhaseController = searchPhaseController;
this.dfsSearchResults = dfsSearchResults;
this.searchResults = searchResults;
this.dfs = dfs;
this.nextPhaseFactory = nextPhaseFactory;
this.context = context;
this.searchTransportService = context.getSearchTransport();
Expand All @@ -67,18 +69,16 @@ final class DfsQueryPhase extends SearchPhase {
public void run() throws IOException {
// TODO we can potentially also consume the actual per shard results from the initial phase here in the aggregateDfs
// to free up memory early
final List<DfsSearchResult> resultList = dfsSearchResults.asList();
final AggregatedDfs dfs = searchPhaseController.aggregateDfs(resultList);
final CountedCollector<SearchPhaseResult> counter = new CountedCollector<>(queryResult::consumeResult,
resultList.size(),
searchResults.size(),
() -> context.executeNextPhase(this, nextPhaseFactory.apply(queryResult)), context);
final SearchSourceBuilder sourceBuilder = context.getRequest().source();
progressListener.notifyListShards(progressListener.searchShards(resultList), sourceBuilder == null || sourceBuilder.size() != 0);
for (final DfsSearchResult dfsResult : resultList) {
progressListener.notifyListShards(progressListener.searchShards(searchResults), sourceBuilder == null || sourceBuilder.size() != 0);
for (final DfsSearchResult dfsResult : searchResults) {
final SearchShardTarget searchShardTarget = dfsResult.getSearchShardTarget();
Transport.Connection connection = context.getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId());
QuerySearchRequest querySearchRequest = new QuerySearchRequest(searchShardTarget.getOriginalIndices(),
dfsResult.getRequestId(), dfs);
dfsResult.getRequestId(), dfsResult.getShardSearchRequest(), dfs);
final int shardIndex = dfsResult.getShardIndex();
searchTransportService.sendExecuteQuery(connection, querySearchRequest, context.getTask(),
new SearchActionListener<QuerySearchResult>(searchShardTarget, shardIndex) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,14 @@
import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.search.RescoreDocIds;
import org.elasticsearch.search.SearchPhaseResult;
import org.elasticsearch.search.SearchShardTarget;
import org.elasticsearch.search.dfs.AggregatedDfs;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.ShardFetchSearchRequest;
import org.elasticsearch.search.internal.InternalSearchResponse;
import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.transport.Transport;

Expand All @@ -50,16 +53,19 @@ final class FetchSearchPhase extends SearchPhase {
private final Logger logger;
private final SearchPhaseResults<SearchPhaseResult> resultConsumer;
private final SearchProgressListener progressListener;
private final AggregatedDfs aggregatedDfs;

FetchSearchPhase(SearchPhaseResults<SearchPhaseResult> resultConsumer,
SearchPhaseController searchPhaseController,
AggregatedDfs aggregatedDfs,
SearchPhaseContext context) {
this(resultConsumer, searchPhaseController, context,
this(resultConsumer, searchPhaseController, aggregatedDfs, context,
(response, scrollId) -> new ExpandSearchPhase(context, response, scrollId));
}

FetchSearchPhase(SearchPhaseResults<SearchPhaseResult> resultConsumer,
SearchPhaseController searchPhaseController,
AggregatedDfs aggregatedDfs,
SearchPhaseContext context, BiFunction<InternalSearchResponse, String, SearchPhase> nextPhaseFactory) {
super("fetch");
if (context.getNumShards() != resultConsumer.getNumShards()) {
Expand All @@ -69,6 +75,7 @@ final class FetchSearchPhase extends SearchPhase {
this.fetchResults = new AtomicArray<>(resultConsumer.getNumShards());
this.searchPhaseController = searchPhaseController;
this.queryResults = resultConsumer.getAtomicArray();
this.aggregatedDfs = aggregatedDfs;
this.nextPhaseFactory = nextPhaseFactory;
this.context = context;
this.logger = context.getLogger();
Expand Down Expand Up @@ -144,7 +151,8 @@ private void innerRun() throws IOException {
Transport.Connection connection = context.getConnection(searchShardTarget.getClusterAlias(),
searchShardTarget.getNodeId());
ShardFetchSearchRequest fetchSearchRequest = createFetchRequest(queryResult.queryResult().getRequestId(), i, entry,
lastEmittedDocPerShard, searchShardTarget.getOriginalIndices());
lastEmittedDocPerShard, searchShardTarget.getOriginalIndices(), queryResult.getShardSearchRequest(),
queryResult.getRescoreDocIds());
executeFetch(i, searchShardTarget, counter, fetchSearchRequest, queryResult.queryResult(),
connection);
}
Expand All @@ -153,10 +161,12 @@ private void innerRun() throws IOException {
}
}

protected ShardFetchSearchRequest createFetchRequest(long queryId, int index, IntArrayList entry,
ScoreDoc[] lastEmittedDocPerShard, OriginalIndices originalIndices) {
protected ShardFetchSearchRequest createFetchRequest(long queryId, int index, IntArrayList entry, ScoreDoc[] lastEmittedDocPerShard,
OriginalIndices originalIndices, ShardSearchRequest shardSearchRequest,
RescoreDocIds rescoreDocIds) {
final ScoreDoc lastEmittedDoc = (lastEmittedDocPerShard != null) ? lastEmittedDocPerShard[index] : null;
return new ShardFetchSearchRequest(originalIndices, queryId, entry, lastEmittedDoc);
return new ShardFetchSearchRequest(originalIndices, queryId, shardSearchRequest, entry, lastEmittedDoc,
rescoreDocIds, aggregatedDfs);
}

private void executeFetch(final int shardIndex, final SearchShardTarget shardTarget,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,10 +23,12 @@
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.search.dfs.AggregatedDfs;
import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.internal.AliasFilter;
import org.elasticsearch.transport.Transport;

import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Executor;
Expand Down Expand Up @@ -59,7 +61,10 @@ protected void executePhaseOnShard(final SearchShardIterator shardIt, final Shar

@Override
protected SearchPhase getNextPhase(final SearchPhaseResults<DfsSearchResult> results, final SearchPhaseContext context) {
return new DfsQueryPhase(results.getAtomicArray(), searchPhaseController, (queryResults) ->
new FetchSearchPhase(queryResults, searchPhaseController, context), context);
final List<DfsSearchResult> dfsSearchResults = results.getAtomicArray().asList();
final AggregatedDfs aggregatedDfs = searchPhaseController.aggregateDfs(dfsSearchResults);

return new DfsQueryPhase(dfsSearchResults, aggregatedDfs, searchPhaseController, (queryResults) ->
new FetchSearchPhase(queryResults, searchPhaseController, aggregatedDfs, context), context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,6 @@ protected void onShardGroupFailure(int shardIndex, Exception exc) {

@Override
protected SearchPhase getNextPhase(final SearchPhaseResults<SearchPhaseResult> results, final SearchPhaseContext context) {
return new FetchSearchPhase(results, searchPhaseController, context);
return new FetchSearchPhase(results, searchPhaseController, null, context);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@

package org.elasticsearch.action.search;

import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.IndicesRequest;
Expand Down Expand Up @@ -282,6 +283,10 @@ public void writeTo(StreamOutput out) throws IOException {
}
}

static boolean keepStatesInContext(Version version) {
return version.before(Version.V_8_0_0);
}

public static void registerRequestHandler(TransportService transportService, SearchService searchService) {
transportService.registerRequestHandler(FREE_CONTEXT_SCROLL_ACTION_NAME, ThreadPool.Names.SAME, ScrollFreeContextRequest::new,
(request, channel, task) -> {
Expand All @@ -306,15 +311,15 @@ public static void registerRequestHandler(TransportService transportService, Sea

transportService.registerRequestHandler(DFS_ACTION_NAME, ThreadPool.Names.SAME, ShardSearchRequest::new,
(request, channel, task) ->
searchService.executeDfsPhase(request, (SearchShardTask) task,
searchService.executeDfsPhase(request, keepStatesInContext(channel.getVersion()), (SearchShardTask) task,
new ChannelActionListener<>(channel, DFS_ACTION_NAME, request))
);

TransportActionProxy.registerProxyAction(transportService, DFS_ACTION_NAME, DfsSearchResult::new);

transportService.registerRequestHandler(QUERY_ACTION_NAME, ThreadPool.Names.SAME, ShardSearchRequest::new,
(request, channel, task) -> {
searchService.executeQueryPhase(request, (SearchShardTask) task,
searchService.executeQueryPhase(request, keepStatesInContext(channel.getVersion()), (SearchShardTask) task,
new ChannelActionListener<>(channel, QUERY_ACTION_NAME, request));
});
TransportActionProxy.registerProxyActionWithDynamicResponseType(transportService, QUERY_ACTION_NAME,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -138,6 +138,7 @@ final class DefaultSearchContext extends SearchContext {
private SearchContextAggregations aggregations;
private SearchContextHighlight highlight;
private SuggestionSearchContext suggest;
private List<RescoreContext> rescore;
private Profilers profilers;

private final Map<String, SearchExtBuilder> searchExtBuilders = new HashMap<>();
Expand All @@ -155,16 +156,15 @@ final class DefaultSearchContext extends SearchContext {
this.shardTarget = shardTarget;
// SearchContexts use a BigArrays that can circuit break
this.bigArrays = bigArrays.withCircuitBreaking();
this.dfsResult = new DfsSearchResult(readerContext.id(), shardTarget);
this.queryResult = new QuerySearchResult(readerContext.id(), shardTarget);
this.dfsResult = new DfsSearchResult(readerContext.id(), shardTarget, request);
this.queryResult = new QuerySearchResult(readerContext.id(), shardTarget, request);
this.fetchResult = new FetchSearchResult(readerContext.id(), shardTarget);
this.indexShard = indexShard;
this.indexService = indexService;
this.clusterService = clusterService;
final Engine.Searcher engineSearcher = readerContext.engineSearcher();
this.searcher = new ContextIndexSearcher(engineSearcher.getIndexReader(), engineSearcher.getSimilarity(),
engineSearcher.getQueryCache(), engineSearcher.getQueryCachingPolicy());
searcher.setAggregatedDfs(readerContext.aggregatedDfs());
this.relativeTimeSupplier = relativeTimeSupplier;
this.timeout = timeout;
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), this.searcher,
Expand Down Expand Up @@ -203,7 +203,7 @@ public void preProcess(boolean rewrite) {
+ "]. Scroll batch sizes cost as much memory as result windows so they are controlled by the ["
+ IndexSettings.MAX_RESULT_WINDOW_SETTING.getKey() + "] index level setting.");
}
if (rescore().isEmpty() == false) {
if (rescore != null) {
if (sort != null) {
throw new IllegalArgumentException("Cannot use [sort] option in conjunction with [rescore].");
}
Expand Down Expand Up @@ -327,7 +327,6 @@ public ScrollContext scrollContext() {
return readerContext.scrollContext();
}


@Override
public SearchContextAggregations aggregations() {
return aggregations;
Expand Down Expand Up @@ -373,7 +372,18 @@ public void suggest(SuggestionSearchContext suggest) {

@Override
public List<RescoreContext> rescore() {
return readerContext.rescore();
if (rescore == null) {
return List.of();
}
return rescore;
}

@Override
public void addRescore(RescoreContext rescore) {
if (this.rescore == null) {
this.rescore = new ArrayList<>();
}
this.rescore.add(rescore);
}

@Override
Expand Down
51 changes: 51 additions & 0 deletions server/src/main/java/org/elasticsearch/search/RescoreDocIds.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.search;

import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;

import java.io.IOException;
import java.util.Map;
import java.util.Set;

public final class RescoreDocIds implements Writeable {
public static final RescoreDocIds EMPTY = new RescoreDocIds(Map.of());

private final Map<Integer, Set<Integer>> docIds;

public RescoreDocIds(Map<Integer, Set<Integer>> docIds) {
this.docIds = docIds;
}

public RescoreDocIds(StreamInput in) throws IOException {
docIds = in.readMap(StreamInput::readVInt, i -> i.readSet(StreamInput::readVInt));
}

public Set<Integer> getId(int index) {
return docIds.get(index);
}

@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeMap(docIds, StreamOutput::writeVInt, (o, v) -> o.writeCollection(v, StreamOutput::writeVInt));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,11 @@

package org.elasticsearch.search;

import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.internal.ShardSearchRequest;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.transport.TransportResponse;

Expand All @@ -40,6 +42,8 @@ public abstract class SearchPhaseResult extends TransportResponse {
private SearchShardTarget searchShardTarget;
private int shardIndex = -1;
protected long requestId;
private ShardSearchRequest shardSearchRequest;
private RescoreDocIds rescoreDocIds = RescoreDocIds.EMPTY;

protected SearchPhaseResult() {

Expand Down Expand Up @@ -90,6 +94,23 @@ public QuerySearchResult queryResult() {
*/
public FetchSearchResult fetchResult() { return null; }

@Nullable
public ShardSearchRequest getShardSearchRequest() {
return shardSearchRequest;
}

public void setShardSearchRequest(ShardSearchRequest shardSearchRequest) {
this.shardSearchRequest = shardSearchRequest;
}

public RescoreDocIds getRescoreDocIds() {
return rescoreDocIds;
}

public void setRescoreDocIds(RescoreDocIds rescoreDocIds) {
this.rescoreDocIds = rescoreDocIds;
}

@Override
public void writeTo(StreamOutput out) throws IOException {
// TODO: this seems wrong, SearchPhaseResult should have a writeTo?
Expand Down
Loading

0 comments on commit 206381e

Please sign in to comment.