Skip to content

Commit 0f333c8

Browse files
authored
Always rewrite search shard request outside of the search thread pool (#51708) (#51979)
This change ensures that the rewrite of the shard request is executed in the network thread or in the refresh listener when waiting for an active shard. This allows queries that rewrite to match_no_docs to bypass the search thread pool entirely even if the can_match phase was skipped (pre_filter_shard_size > number of shards). Coordinating nodes don't have the ability to create empty responses so this change also ensures that at least one shard creates a full empty response while the other can return null ones. This is needed since creating true empty responses on shards require to create concrete aggregators which would be too costly to build on a network thread. We should move this functionality to aggregation builders in a follow up but that would be a much bigger change. This change is also important for #49601 since we want to add the ability to use the result of other shards to rewrite the request of subsequent ones. For instance if the first M shards have their top N computed, the top worst document in the global queue can be pass to subsequent shards that can then rewrite to match_no_docs if they can guarantee that they don't have any document better than the provided one.
1 parent fb710cc commit 0f333c8

File tree

13 files changed

+624
-170
lines changed

13 files changed

+624
-170
lines changed

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

+29
Original file line numberDiff line numberDiff line change
@@ -153,3 +153,32 @@ setup:
153153
- match: { _shards.failed: 0 }
154154
- match: { hits.total: 2 }
155155
- length: { aggregations.idx_terms.buckets: 2 }
156+
157+
# check that empty responses are correctly handled when rewriting to match_no_docs
158+
- do:
159+
search:
160+
rest_total_hits_as_int: true
161+
# ensure that one shard can return empty response
162+
max_concurrent_shard_requests: 1
163+
body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2016-02-01", "lt": "2018-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } }
164+
165+
- match: { _shards.total: 3 }
166+
- match: { _shards.successful: 3 }
167+
- match: { _shards.skipped : 0 }
168+
- match: { _shards.failed: 0 }
169+
- match: { hits.total: 2 }
170+
- length: { aggregations.idx_terms.buckets: 2 }
171+
172+
- do:
173+
search:
174+
rest_total_hits_as_int: true
175+
# ensure that one shard can return empty response
176+
max_concurrent_shard_requests: 2
177+
body: { "size" : 0, "query" : { "range" : { "created_at" : { "gte" : "2019-02-01"}}}, "aggs" : { "idx_terms" : { "terms" : { "field" : "_index" } } } }
178+
179+
- match: { _shards.total: 3 }
180+
- match: { _shards.successful: 3 }
181+
- match: { _shards.skipped : 0 }
182+
- match: { _shards.failed: 0 }
183+
- match: { hits.total: 0 }
184+
- length: { aggregations.idx_terms.buckets: 0 }

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

+9-1
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@
5151
import java.util.Set;
5252
import java.util.concurrent.ConcurrentHashMap;
5353
import java.util.concurrent.Executor;
54+
import java.util.concurrent.atomic.AtomicBoolean;
5455
import java.util.concurrent.atomic.AtomicInteger;
5556
import java.util.function.BiFunction;
5657
import java.util.stream.Collectors;
@@ -82,6 +83,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
8283
private final Map<String, Set<String>> indexRoutings;
8384
private final SetOnce<AtomicArray<ShardSearchFailure>> shardFailures = new SetOnce<>();
8485
private final Object shardFailuresMutex = new Object();
86+
private final AtomicBoolean hasShardResponse = new AtomicBoolean(false);
8587
private final AtomicInteger successfulOps = new AtomicInteger();
8688
private final AtomicInteger skippedOps = new AtomicInteger();
8789
private final SearchTimeProvider timeProvider;
@@ -467,6 +469,7 @@ private void onShardResult(Result result, SearchShardIterator shardIt) {
467469
assert result.getSearchShardTarget() != null : "search shard target must not be null";
468470
successfulOps.incrementAndGet();
469471
results.consumeResult(result);
472+
hasShardResponse.set(true);
470473
if (logger.isTraceEnabled()) {
471474
logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null);
472475
}
@@ -602,8 +605,13 @@ public final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shar
602605
String indexName = shardIt.shardId().getIndex().getName();
603606
final String[] routings = indexRoutings.getOrDefault(indexName, Collections.emptySet())
604607
.toArray(new String[0]);
605-
return new ShardSearchRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(),
608+
ShardSearchRequest shardRequest = new ShardSearchRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(),
606609
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), routings);
610+
// if we already received a search result we can inform the shard that it
611+
// can return a null response if the request rewrites to match none rather
612+
// than creating an empty response in the search thread pool.
613+
shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get());
614+
return shardRequest;
607615
}
608616

609617
/**

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

+57-29
Original file line numberDiff line numberDiff line change
@@ -39,6 +39,7 @@
3939
import org.elasticsearch.search.SearchHit;
4040
import org.elasticsearch.search.SearchHits;
4141
import org.elasticsearch.search.SearchPhaseResult;
42+
import org.elasticsearch.search.SearchService;
4243
import org.elasticsearch.search.SearchShardTarget;
4344
import org.elasticsearch.search.aggregations.InternalAggregation;
4445
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
@@ -65,6 +66,7 @@
6566
import java.util.Map;
6667
import java.util.function.Function;
6768
import java.util.function.IntFunction;
69+
import java.util.stream.Collectors;
6870

6971
public final class SearchPhaseController {
7072

@@ -427,6 +429,15 @@ private ReducedQueryPhase reducedQueryPhase(Collection<? extends SearchPhaseResu
427429
return new ReducedQueryPhase(totalHits, topDocsStats.fetchHits, topDocsStats.getMaxScore(),
428430
false, null, null, null, null, SortedTopDocs.EMPTY, null, numReducePhases, 0, 0, true);
429431
}
432+
int total = queryResults.size();
433+
queryResults = queryResults.stream()
434+
.filter(res -> res.queryResult().isNull() == false)
435+
.collect(Collectors.toList());
436+
String errorMsg = "must have at least one non-empty search result, got 0 out of " + total;
437+
assert queryResults.isEmpty() == false : errorMsg;
438+
if (queryResults.isEmpty()) {
439+
throw new IllegalStateException(errorMsg);
440+
}
430441
final QuerySearchResult firstResult = queryResults.stream().findFirst().get().queryResult();
431442
final boolean hasSuggest = firstResult.suggest() != null;
432443
final boolean hasProfileResults = firstResult.hasProfileResults();
@@ -497,6 +508,18 @@ private ReducedQueryPhase reducedQueryPhase(Collection<? extends SearchPhaseResu
497508
firstResult.sortValueFormats(), numReducePhases, size, from, false);
498509
}
499510

511+
/*
512+
* Returns the size of the requested top documents (from + size)
513+
*/
514+
static int getTopDocsSize(SearchRequest request) {
515+
if (request.source() == null) {
516+
return SearchService.DEFAULT_SIZE;
517+
}
518+
SearchSourceBuilder source = request.source();
519+
return (source.size() == -1 ? SearchService.DEFAULT_SIZE : source.size()) +
520+
(source.from() == -1 ? SearchService.DEFAULT_FROM : source.from());
521+
}
522+
500523
public static final class ReducedQueryPhase {
501524
// the sum of all hits across all reduces shards
502525
final TotalHits totalHits;
@@ -576,6 +599,7 @@ static final class QueryPhaseResultConsumer extends ArraySearchPhaseResults<Sear
576599
private final SearchProgressListener progressListener;
577600
private int numReducePhases = 0;
578601
private final TopDocsStats topDocsStats;
602+
private final int topNSize;
579603
private final boolean performFinalReduce;
580604

581605
/**
@@ -589,7 +613,7 @@ static final class QueryPhaseResultConsumer extends ArraySearchPhaseResults<Sear
589613
*/
590614
private QueryPhaseResultConsumer(SearchProgressListener progressListener, SearchPhaseController controller,
591615
int expectedResultSize, int bufferSize, boolean hasTopDocs, boolean hasAggs,
592-
int trackTotalHitsUpTo, boolean performFinalReduce) {
616+
int trackTotalHitsUpTo, int topNSize, boolean performFinalReduce) {
593617
super(expectedResultSize);
594618
if (expectedResultSize != 1 && bufferSize < 2) {
595619
throw new IllegalArgumentException("buffer size must be >= 2 if there is more than one expected result");
@@ -610,6 +634,7 @@ private QueryPhaseResultConsumer(SearchProgressListener progressListener, Search
610634
this.hasAggs = hasAggs;
611635
this.bufferSize = bufferSize;
612636
this.topDocsStats = new TopDocsStats(trackTotalHitsUpTo);
637+
this.topNSize = topNSize;
613638
this.performFinalReduce = performFinalReduce;
614639
}
615640

@@ -622,36 +647,38 @@ public void consumeResult(SearchPhaseResult result) {
622647
}
623648

624649
private synchronized void consumeInternal(QuerySearchResult querySearchResult) {
625-
if (index == bufferSize) {
650+
if (querySearchResult.isNull() == false) {
651+
if (index == bufferSize) {
652+
if (hasAggs) {
653+
ReduceContext reduceContext = controller.reduceContextFunction.apply(false);
654+
InternalAggregations reducedAggs = InternalAggregations.topLevelReduce(Arrays.asList(aggsBuffer), reduceContext);
655+
Arrays.fill(aggsBuffer, null);
656+
aggsBuffer[0] = reducedAggs;
657+
}
658+
if (hasTopDocs) {
659+
TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer),
660+
// we have to merge here in the same way we collect on a shard
661+
topNSize, 0);
662+
Arrays.fill(topDocsBuffer, null);
663+
topDocsBuffer[0] = reducedTopDocs;
664+
}
665+
numReducePhases++;
666+
index = 1;
667+
if (hasAggs) {
668+
progressListener.notifyPartialReduce(progressListener.searchShards(processedShards),
669+
topDocsStats.getTotalHits(), aggsBuffer[0], numReducePhases);
670+
}
671+
}
672+
final int i = index++;
626673
if (hasAggs) {
627-
ReduceContext reduceContext = controller.reduceContextFunction.apply(false);
628-
InternalAggregations reducedAggs = InternalAggregations.topLevelReduce(Arrays.asList(aggsBuffer), reduceContext);
629-
Arrays.fill(aggsBuffer, null);
630-
aggsBuffer[0] = reducedAggs;
674+
aggsBuffer[i] = (InternalAggregations) querySearchResult.consumeAggs();
631675
}
632676
if (hasTopDocs) {
633-
TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer),
634-
// we have to merge here in the same way we collect on a shard
635-
querySearchResult.from() + querySearchResult.size(), 0);
636-
Arrays.fill(topDocsBuffer, null);
637-
topDocsBuffer[0] = reducedTopDocs;
677+
final TopDocsAndMaxScore topDocs = querySearchResult.consumeTopDocs(); // can't be null
678+
topDocsStats.add(topDocs, querySearchResult.searchTimedOut(), querySearchResult.terminatedEarly());
679+
setShardIndex(topDocs.topDocs, querySearchResult.getShardIndex());
680+
topDocsBuffer[i] = topDocs.topDocs;
638681
}
639-
numReducePhases++;
640-
index = 1;
641-
if (hasAggs) {
642-
progressListener.notifyPartialReduce(progressListener.searchShards(processedShards),
643-
topDocsStats.getTotalHits(), aggsBuffer[0], numReducePhases);
644-
}
645-
}
646-
final int i = index++;
647-
if (hasAggs) {
648-
aggsBuffer[i] = (InternalAggregations) querySearchResult.consumeAggs();
649-
}
650-
if (hasTopDocs) {
651-
final TopDocsAndMaxScore topDocs = querySearchResult.consumeTopDocs(); // can't be null
652-
topDocsStats.add(topDocs, querySearchResult.searchTimedOut(), querySearchResult.terminatedEarly());
653-
setShardIndex(topDocs.topDocs, querySearchResult.getShardIndex());
654-
topDocsBuffer[i] = topDocs.topDocs;
655682
}
656683
processedShards[querySearchResult.getShardIndex()] = querySearchResult.getSearchShardTarget();
657684
}
@@ -706,9 +733,10 @@ ArraySearchPhaseResults<SearchPhaseResult> newSearchPhaseResults(SearchProgressL
706733
if (isScrollRequest == false && (hasAggs || hasTopDocs)) {
707734
// no incremental reduce if scroll is used - we only hit a single shard or sometimes more...
708735
if (request.getBatchedReduceSize() < numShards) {
736+
int topNSize = getTopDocsSize(request);
709737
// only use this if there are aggs and if there are more shards than we should reduce at once
710738
return new QueryPhaseResultConsumer(listener, this, numShards, request.getBatchedReduceSize(), hasTopDocs, hasAggs,
711-
trackTotalHitsUpTo, request.isFinalReduce());
739+
trackTotalHitsUpTo, topNSize, request.isFinalReduce());
712740
}
713741
}
714742
return new ArraySearchPhaseResults<SearchPhaseResult>(numShards) {
@@ -731,7 +759,7 @@ ReducedQueryPhase reduce() {
731759

732760
static final class TopDocsStats {
733761
final int trackTotalHitsUpTo;
734-
private long totalHits;
762+
long totalHits;
735763
private TotalHits.Relation totalHitsRelation;
736764
long fetchHits;
737765
private float maxScore = Float.NEGATIVE_INFINITY;

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

+7
Original file line numberDiff line numberDiff line change
@@ -1254,6 +1254,13 @@ private Engine.Searcher acquireSearcher(String source, Engine.SearcherScope scop
12541254
markSearcherAccessed();
12551255
final Engine engine = getEngine();
12561256
final Engine.Searcher searcher = engine.acquireSearcher(source, scope);
1257+
return wrapSearcher(searcher);
1258+
}
1259+
1260+
/**
1261+
* Wraps the provided searcher acquired with {@link #acquireSearcherNoWrap(String)}.
1262+
*/
1263+
public Engine.Searcher wrapSearcher(Engine.Searcher searcher) {
12571264
assert ElasticsearchDirectoryReader.unwrap(searcher.getDirectoryReader())
12581265
!= null : "DirectoryReader must be an instance or ElasticsearchDirectoryReader";
12591266
boolean success = false;

0 commit comments

Comments
 (0)