Skip to content

Commit ff94792

Browse files
authored
Shortcut query phase using the results of other shards (#51852)
This commit, built on top of #51708, allows to modify shard search requests based on informations collected on other shards. It is intended to speed up sorted queries on time-based indices. For queries that are only interested in the top documents. This change will rewrite the shard queries to match none if the bottom sort value computed in prior shards is better than all values in the shard. For queries that mix top documents and aggregations this change will reset the size of the top documents to 0 instead of rewriting to match none. This means that we don't need to keep a search context open for this shard since we know in advance that it doesn't contain any competitive hit.
1 parent f65e4d6 commit ff94792

18 files changed

+1036
-30
lines changed

modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollParallelizationHelper.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ static SearchRequest[] sliceIntoSubRequests(SearchRequest request, String field,
203203
if (request.source().slice() != null) {
204204
throw new IllegalStateException("Can't slice a request that already has a slice configuration");
205205
}
206-
slicedSource = request.source().copyWithNewSlice(sliceBuilder);
206+
slicedSource = request.source().shallowCopy().slice(sliceBuilder);
207207
}
208208
SearchRequest searchRequest = new SearchRequest(request);
209209
searchRequest.source(slicedSource);

rest-api-spec/src/main/resources/rest-api-spec/test/search/140_pre_filter_search_shards.yml

+46-6
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ setup:
1919
mappings:
2020
properties:
2121
created_at:
22-
type: date
22+
type: date_nanos
2323
format: "yyyy-MM-dd"
2424
- do:
2525
indices.create:
@@ -154,10 +154,9 @@ setup:
154154
- match: { hits.total: 2 }
155155
- length: { aggregations.idx_terms.buckets: 2 }
156156

157-
# check that empty responses are correctly handled when rewriting to match_no_docs
157+
# check that empty responses are correctly handled when rewriting to match_no_docs
158158
- do:
159159
search:
160-
rest_total_hits_as_int: true
161160
# ensure that one shard can return empty response
162161
max_concurrent_shard_requests: 1
163162
body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } }
@@ -166,12 +165,11 @@ setup:
166165
- match: { _shards.successful: 3 }
167166
- match: { _shards.skipped : 0 }
168167
- match: { _shards.failed: 0 }
169-
- match: { hits.total: 2 }
168+
- match: { hits.total.value: 2 }
170169
- length: { aggregations.idx_terms.buckets: 2 }
171170

172171
- do:
173172
search:
174-
rest_total_hits_as_int: true
175173
# ensure that one shard can return empty response
176174
max_concurrent_shard_requests: 2
177175
body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2019-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } }
@@ -180,5 +178,47 @@ setup:
180178
- match: { _shards.successful: 3 }
181179
- match: { _shards.skipped : 0 }
182180
- match: { _shards.failed: 0 }
183-
- match: { hits.total: 0 }
181+
- match: { hits.total.value: 0 }
184182
- length: { aggregations.idx_terms.buckets: 0 }
183+
184+
# check field sort is correct when skipping query phase
185+
- do:
186+
search:
187+
# ensure that one shard can return empty response
188+
max_concurrent_shard_requests: 1
189+
pre_filter_shard_size: 1
190+
body:
191+
"size": 1
192+
"track_total_hits": 1
193+
"sort": [{ "created_at": { "order": "desc", "numeric_type": "date" } }]
194+
195+
- match: { _shards.total: 3 }
196+
- match: { _shards.successful: 3 }
197+
- match: { _shards.skipped: 0 }
198+
- match: { _shards.failed: 0 }
199+
- match: { hits.total.value: 1 }
200+
- match: { hits.total.relation: "gte" }
201+
- length: { hits.hits: 1 }
202+
- match: { hits.hits.0._id: "3" }
203+
204+
# same with aggs
205+
- do:
206+
search:
207+
# ensure that one shard can return empty response
208+
max_concurrent_shard_requests: 1
209+
pre_filter_shard_size: 1
210+
body:
211+
"size": 1
212+
"track_total_hits": 1
213+
"sort": [{ "created_at": { "order": "desc", "numeric_type": "date" } }]
214+
"aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } }
215+
216+
- match: { _shards.total: 3 }
217+
- match: { _shards.successful: 3 }
218+
- match: { _shards.skipped: 0 }
219+
- match: { _shards.failed: 0 }
220+
- match: { hits.total.value: 1 }
221+
- match: { hits.total.relation: "gte" }
222+
- length: { hits.hits: 1 }
223+
- match: {hits.hits.0._id: "3" }
224+
- length: { aggregations.idx_terms.buckets: 3 }

server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
7777
**/
7878
private final BiFunction<String, String, Transport.Connection> nodeIdToConnection;
7979
private final SearchTask task;
80-
private final SearchPhaseResults<Result> results;
80+
final SearchPhaseResults<Result> results;
8181
private final ClusterState clusterState;
8282
private final Map<String, AliasFilter> aliasFilter;
8383
private final Map<String, Float> concreteIndexBoosts;
@@ -467,7 +467,7 @@ public final void onShardFailure(final int shardIndex, @Nullable SearchShardTarg
467467
* @param result the result returned form the shard
468468
* @param shardIt the shard iterator
469469
*/
470-
private void onShardResult(Result result, SearchShardIterator shardIt) {
470+
protected void onShardResult(Result result, SearchShardIterator shardIt) {
471471
assert result.getShardIndex() != -1 : "shard index is not set";
472472
assert result.getSearchShardTarget() != null : "search shard target must not be null";
473473
successfulOps.incrementAndGet();
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,111 @@
1+
/*
2+
* Licensed to Elasticsearch under one or more contributor
3+
* license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright
5+
* ownership. Elasticsearch licenses this file to you under
6+
* the Apache License, Version 2.0 (the "License"); you may
7+
* not use this file except in compliance with the License.
8+
* You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.elasticsearch.action.search;
21+
22+
import org.apache.lucene.search.FieldComparator;
23+
import org.apache.lucene.search.FieldDoc;
24+
import org.apache.lucene.search.SortField;
25+
import org.apache.lucene.search.TopFieldDocs;
26+
import org.elasticsearch.search.DocValueFormat;
27+
import org.elasticsearch.search.SearchSortValuesAndFormats;
28+
29+
/**
30+
* Utility class to keep track of the bottom doc's sort values in a distributed search.
31+
*/
32+
class BottomSortValuesCollector {
33+
private final int topNSize;
34+
private final SortField[] sortFields;
35+
private final FieldComparator[] comparators;
36+
private final int[] reverseMuls;
37+
38+
private volatile long totalHits;
39+
private volatile SearchSortValuesAndFormats bottomSortValues;
40+
41+
BottomSortValuesCollector(int topNSize, SortField[] sortFields) {
42+
this.topNSize = topNSize;
43+
this.comparators = new FieldComparator[sortFields.length];
44+
this.reverseMuls = new int[sortFields.length];
45+
this.sortFields = sortFields;
46+
for (int i = 0; i < sortFields.length; i++) {
47+
comparators[i] = sortFields[i].getComparator(1, i);
48+
reverseMuls[i] = sortFields[i].getReverse() ? -1 : 1;
49+
}
50+
}
51+
52+
long getTotalHits() {
53+
return totalHits;
54+
}
55+
56+
/**
57+
* @return The best bottom sort values consumed so far.
58+
*/
59+
SearchSortValuesAndFormats getBottomSortValues() {
60+
return bottomSortValues;
61+
}
62+
63+
synchronized void consumeTopDocs(TopFieldDocs topDocs, DocValueFormat[] sortValuesFormat) {
64+
totalHits += topDocs.totalHits.value;
65+
if (validateShardSortFields(topDocs.fields) == false) {
66+
return;
67+
}
68+
69+
FieldDoc shardBottomDoc = extractBottom(topDocs);
70+
if (shardBottomDoc == null) {
71+
return;
72+
}
73+
if (bottomSortValues == null
74+
|| compareValues(shardBottomDoc.fields, bottomSortValues.getRawSortValues()) < 0) {
75+
bottomSortValues = new SearchSortValuesAndFormats(shardBottomDoc.fields, sortValuesFormat);
76+
}
77+
}
78+
79+
/**
80+
* @return <code>false</code> if the provided {@link SortField} array differs
81+
* from the initial {@link BottomSortValuesCollector#sortFields}.
82+
*/
83+
private boolean validateShardSortFields(SortField[] shardSortFields) {
84+
for (int i = 0; i < shardSortFields.length; i++) {
85+
if (shardSortFields[i].equals(sortFields[i]) == false) {
86+
// ignore shards response that would make the sort incompatible
87+
// (e.g.: mixing keyword/numeric or long/double).
88+
// TODO: we should fail the entire request because the topdocs
89+
// merge will likely fail later but this is not possible with
90+
// the current async logic that only allows shard failures here.
91+
return false;
92+
}
93+
}
94+
return true;
95+
}
96+
97+
private FieldDoc extractBottom(TopFieldDocs topDocs) {
98+
return topNSize > 0 && topDocs.scoreDocs.length == topNSize ?
99+
(FieldDoc) topDocs.scoreDocs[topNSize-1] : null;
100+
}
101+
102+
private int compareValues(Object[] v1, Object[] v2) {
103+
for (int i = 0; i < v1.length; i++) {
104+
int cmp = reverseMuls[i] * comparators[i].compareValues(v1[i], v2[i]);
105+
if (cmp != 0) {
106+
return cmp;
107+
}
108+
}
109+
return 0;
110+
}
111+
}

server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java

+3-11
Original file line numberDiff line numberDiff line change
@@ -478,7 +478,8 @@ private ReducedQueryPhase reducedQueryPhase(Collection<? extends SearchPhaseResu
478478
for (SearchPhaseResult entry : queryResults) {
479479
QuerySearchResult result = entry.queryResult();
480480
from = result.from();
481-
size = result.size();
481+
// sorted queries can set the size to 0 if they have enough competitive hits.
482+
size = Math.max(result.size(), size);
482483
if (hasSuggest) {
483484
assert result.suggest() != null;
484485
for (Suggestion<? extends Suggestion.Entry<? extends Suggestion.Entry.Option>> suggestion : result.suggest()) {
@@ -724,15 +725,6 @@ int getNumBuffered() {
724725
int getNumReducePhases() { return numReducePhases; }
725726
}
726727

727-
private int resolveTrackTotalHits(SearchRequest request) {
728-
if (request.scroll() != null) {
729-
// no matter what the value of track_total_hits is
730-
return SearchContext.TRACK_TOTAL_HITS_ACCURATE;
731-
}
732-
return request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : request.source().trackTotalHitsUpTo() == null ?
733-
SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : request.source().trackTotalHitsUpTo();
734-
}
735-
736728
/**
737729
* Returns a new ArraySearchPhaseResults instance. This might return an instance that reduces search responses incrementally.
738730
*/
@@ -743,7 +735,7 @@ ArraySearchPhaseResults<SearchPhaseResult> newSearchPhaseResults(SearchProgressL
743735
boolean isScrollRequest = request.scroll() != null;
744736
final boolean hasAggs = source != null && source.aggregations() != null;
745737
final boolean hasTopDocs = source == null || source.size() != 0;
746-
final int trackTotalHitsUpTo = resolveTrackTotalHits(request);
738+
final int trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo();
747739
InternalAggregation.ReduceContextBuilder aggReduceContextBuilder = requestToAggReduceContextBuilder.apply(request);
748740
if (isScrollRequest == false && (hasAggs || hasTopDocs)) {
749741
// no incremental reduce if scroll is used - we only hit a single shard or sometimes more...

server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java

+51-3
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
package org.elasticsearch.action.search;
2121

2222
import org.apache.logging.log4j.Logger;
23+
import org.apache.lucene.search.TopFieldDocs;
2324
import org.elasticsearch.action.ActionListener;
2425
import org.elasticsearch.cluster.ClusterState;
2526
import org.elasticsearch.cluster.routing.GroupShardsIterator;
@@ -28,18 +29,28 @@
2829
import org.elasticsearch.search.SearchShardTarget;
2930
import org.elasticsearch.search.builder.SearchSourceBuilder;
3031
import org.elasticsearch.search.internal.AliasFilter;
32+
import org.elasticsearch.search.internal.SearchContext;
33+
import org.elasticsearch.search.internal.ShardSearchRequest;
34+
import org.elasticsearch.search.query.QuerySearchResult;
3135
import org.elasticsearch.transport.Transport;
3236

3337
import java.util.Map;
3438
import java.util.Set;
3539
import java.util.concurrent.Executor;
3640
import java.util.function.BiFunction;
3741

38-
final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<SearchPhaseResult> {
42+
import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize;
43+
44+
class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<SearchPhaseResult> {
3945

4046
private final SearchPhaseController searchPhaseController;
4147
private final SearchProgressListener progressListener;
4248

49+
// informations to track the best bottom top doc globally.
50+
private final int topDocsSize;
51+
private final int trackTotalHitsUpTo;
52+
private volatile BottomSortValuesCollector bottomSortCollector;
53+
4354
SearchQueryThenFetchAsyncAction(final Logger logger, final SearchTransportService searchTransportService,
4455
final BiFunction<String, String, Transport.Connection> nodeIdToConnection,
4556
final Map<String, AliasFilter> aliasFilter,
@@ -53,27 +64,64 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<Se
5364
executor, request, listener, shardsIts, timeProvider, clusterState, task,
5465
searchPhaseController.newSearchPhaseResults(task.getProgressListener(), request, shardsIts.size()),
5566
request.getMaxConcurrentShardRequests(), clusters);
67+
this.topDocsSize = getTopDocsSize(request);
68+
this.trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo();
5669
this.searchPhaseController = searchPhaseController;
5770
this.progressListener = task.getProgressListener();
58-
final SearchProgressListener progressListener = task.getProgressListener();
5971
final SearchSourceBuilder sourceBuilder = request.source();
6072
progressListener.notifyListShards(SearchProgressListener.buildSearchShards(this.shardsIts),
6173
SearchProgressListener.buildSearchShards(toSkipShardsIts), clusters, sourceBuilder == null || sourceBuilder.size() != 0);
6274
}
6375

6476
protected void executePhaseOnShard(final SearchShardIterator shardIt, final ShardRouting shard,
6577
final SearchActionListener<SearchPhaseResult> listener) {
78+
ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt));
6679
getSearchTransport().sendExecuteQuery(getConnection(shardIt.getClusterAlias(), shard.currentNodeId()),
67-
buildShardSearchRequest(shardIt), getTask(), listener);
80+
request, getTask(), listener);
6881
}
6982

7083
@Override
7184
protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
7285
progressListener.notifyQueryFailure(shardIndex, shardTarget, exc);
7386
}
7487

88+
@Override
89+
protected void onShardResult(SearchPhaseResult result, SearchShardIterator shardIt) {
90+
QuerySearchResult queryResult = result.queryResult();
91+
if (queryResult.isNull() == false && queryResult.topDocs().topDocs instanceof TopFieldDocs) {
92+
TopFieldDocs topDocs = (TopFieldDocs) queryResult.topDocs().topDocs;
93+
if (bottomSortCollector == null) {
94+
synchronized (this) {
95+
if (bottomSortCollector == null) {
96+
bottomSortCollector = new BottomSortValuesCollector(topDocsSize, topDocs.fields);
97+
}
98+
}
99+
}
100+
bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats());
101+
}
102+
super.onShardResult(result, shardIt);
103+
}
104+
75105
@Override
76106
protected SearchPhase getNextPhase(final SearchPhaseResults<SearchPhaseResult> results, final SearchPhaseContext context) {
77107
return new FetchSearchPhase(results, searchPhaseController, context, clusterState());
78108
}
109+
110+
private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) {
111+
if (bottomSortCollector == null) {
112+
return request;
113+
}
114+
115+
// disable tracking total hits if we already reached the required estimation.
116+
if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_ACCURATE
117+
&& bottomSortCollector.getTotalHits() > trackTotalHitsUpTo) {
118+
request.source(request.source().shallowCopy().trackTotalHits(false));
119+
}
120+
121+
// set the current best bottom field doc
122+
if (bottomSortCollector.getBottomSortValues() != null) {
123+
request.setBottomSortValues(bottomSortCollector.getBottomSortValues());
124+
}
125+
return request;
126+
}
79127
}

server/src/main/java/org/elasticsearch/action/search/SearchRequest.java

+13
Original file line numberDiff line numberDiff line change
@@ -558,6 +558,19 @@ public boolean isSuggestOnly() {
558558
return source != null && source.isSuggestOnly();
559559
}
560560

561+
public int resolveTrackTotalHitsUpTo() {
562+
return resolveTrackTotalHitsUpTo(scroll, source);
563+
}
564+
565+
public static int resolveTrackTotalHitsUpTo(Scroll scroll, SearchSourceBuilder source) {
566+
if (scroll != null) {
567+
// no matter what the value of track_total_hits is
568+
return SearchContext.TRACK_TOTAL_HITS_ACCURATE;
569+
}
570+
return source == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo() == null ?
571+
SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO : source.trackTotalHitsUpTo();
572+
}
573+
561574
@Override
562575
public SearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
563576
// generating description in a lazy way since source can be quite big

server/src/main/java/org/elasticsearch/search/DocValueFormat.java

+4
Original file line numberDiff line numberDiff line change
@@ -230,6 +230,10 @@ public void writeTo(StreamOutput out) throws IOException {
230230
}
231231
}
232232

233+
public DateMathParser getDateMathParser() {
234+
return parser;
235+
}
236+
233237
@Override
234238
public String format(long value) {
235239
return formatter.format(resolution.toInstant(value).atZone(timeZone));

0 commit comments

Comments
 (0)