Skip to content

Commit dafea3c

Browse files
authored
Minor search controller changes (#36479)
This commit contains a few minor changes to our search code: - adjust the visibility of a couple of methods in our search code to package private from public or protected. - make some of the `SearchPhaseController` methods static where possible - rename one of the `SearchPhaseController#reducedQueryPhase` methods (used only for scroll requests) to `reducedScrollQueryPhase` without the `isScrollRequest` argument which was always set to `true` - replace leniency in `SearchPhaseController#setShardIndex` with an assert to make sure that we never set the shard index twice - remove two null checks where the checked field can never be null - resolve an unchecked warning - replace `List#toArray` invocation that creates an array providing the true size with array creation of length 0 - correct a couple of typos in comments
1 parent fb18b35 commit dafea3c

File tree

8 files changed

+27
-42
lines changed

8 files changed

+27
-42
lines changed

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
7272
private final TransportSearchAction.SearchTimeProvider timeProvider;
7373
private final SearchResponse.Clusters clusters;
7474

75-
protected AbstractSearchAsyncAction(String name, Logger logger, SearchTransportService searchTransportService,
75+
AbstractSearchAsyncAction(String name, Logger logger, SearchTransportService searchTransportService,
7676
BiFunction<String, String, Transport.Connection> nodeIdToConnection,
7777
Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
7878
Map<String, Set<String>> indexRoutings,

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

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@
4444
* and collect the results. If a shard request returns a failure this class handles the advance to the next replica of the shard until
4545
* the shards replica iterator is exhausted. Each shard is referenced by position in the {@link GroupShardsIterator} which is later
4646
* referred to as the {@code shardIndex}.
47-
* The fan out and collect algorithm is traditionally used as the initial phase which can either be a query execution or collection
47+
* The fan out and collect algorithm is traditionally used as the initial phase which can either be a query execution or collection of
4848
* distributed frequencies
4949
*/
5050
abstract class InitialSearchPhase<FirstResult extends SearchPhaseResult> extends SearchPhase {
@@ -327,7 +327,6 @@ private void successfulShardExecution(SearchShardIterator shardsIt) {
327327
}
328328
}
329329

330-
331330
/**
332331
* Executed once all shard results have been received and processed
333332
* @see #onShardFailure(int, SearchShardTarget, Exception)
@@ -367,7 +366,7 @@ protected abstract void executePhaseOnShard(SearchShardIterator shardIt, ShardRo
367366
abstract static class SearchPhaseResults<Result extends SearchPhaseResult> {
368367
private final int numShards;
369368

370-
protected SearchPhaseResults(int numShards) {
369+
SearchPhaseResults(int numShards) {
371370
this.numShards = numShards;
372371
}
373372
/**

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

Lines changed: 14 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121

2222
import com.carrotsearch.hppc.IntArrayList;
2323
import com.carrotsearch.hppc.ObjectObjectHashMap;
24-
2524
import org.apache.lucene.index.Term;
2625
import org.apache.lucene.search.CollectionStatistics;
2726
import org.apache.lucene.search.FieldDoc;
@@ -154,7 +153,7 @@ public AggregatedDfs aggregateDfs(Collection<DfsSearchResult> results) {
154153
* @param from the offset into the search results top docs
155154
* @param size the number of hits to return from the merged top docs
156155
*/
157-
public SortedTopDocs sortDocs(boolean ignoreFrom, Collection<? extends SearchPhaseResult> results,
156+
static SortedTopDocs sortDocs(boolean ignoreFrom, Collection<? extends SearchPhaseResult> results,
158157
final Collection<TopDocs> bufferedTopDocs, final TopDocsStats topDocsStats, int from, int size) {
159158
if (results.isEmpty()) {
160159
return SortedTopDocs.EMPTY;
@@ -214,7 +213,7 @@ public SortedTopDocs sortDocs(boolean ignoreFrom, Collection<? extends SearchPha
214213
}
215214
final boolean isSortedByField;
216215
final SortField[] sortFields;
217-
if (mergedTopDocs != null && mergedTopDocs instanceof TopFieldDocs) {
216+
if (mergedTopDocs instanceof TopFieldDocs) {
218217
TopFieldDocs fieldDocs = (TopFieldDocs) mergedTopDocs;
219218
isSortedByField = (fieldDocs instanceof CollapseTopFieldDocs &&
220219
fieldDocs.fields.length == 1 && fieldDocs.fields[0].getType() == SortField.Type.SCORE) == false;
@@ -230,11 +229,10 @@ public SortedTopDocs sortDocs(boolean ignoreFrom, Collection<? extends SearchPha
230229
}
231230
}
232231

233-
TopDocs mergeTopDocs(Collection<TopDocs> results, int topN, int from) {
232+
static TopDocs mergeTopDocs(Collection<TopDocs> results, int topN, int from) {
234233
if (results.isEmpty()) {
235234
return null;
236235
}
237-
assert results.isEmpty() == false;
238236
final boolean setShardIndex = false;
239237
final TopDocs topDocs = results.stream().findFirst().get();
240238
final TopDocs mergedTopDocs;
@@ -259,12 +257,8 @@ TopDocs mergeTopDocs(Collection<TopDocs> results, int topN, int from) {
259257
}
260258

261259
private static void setShardIndex(TopDocs topDocs, int shardIndex) {
260+
assert topDocs.scoreDocs.length == 0 || topDocs.scoreDocs[0].shardIndex == -1 : "shardIndex is already set";
262261
for (ScoreDoc doc : topDocs.scoreDocs) {
263-
if (doc.shardIndex != -1) {
264-
// once there is a single shard index initialized all others will be initialized too
265-
// there are many asserts down in lucene land that this is actually true. we can shortcut it here.
266-
return;
267-
}
268262
doc.shardIndex = shardIndex;
269263
}
270264
}
@@ -283,7 +277,6 @@ public ScoreDoc[] getLastEmittedDocPerShard(ReducedQueryPhase reducedQueryPhase,
283277
}
284278
}
285279
return lastEmittedDocPerShard;
286-
287280
}
288281

289282
/**
@@ -402,15 +395,15 @@ private SearchHits getHits(ReducedQueryPhase reducedQueryPhase, boolean ignoreFr
402395
hits.add(searchHit);
403396
}
404397
}
405-
return new SearchHits(hits.toArray(new SearchHit[hits.size()]), reducedQueryPhase.totalHits, reducedQueryPhase.maxScore);
398+
return new SearchHits(hits.toArray(new SearchHit[0]), reducedQueryPhase.totalHits, reducedQueryPhase.maxScore);
406399
}
407400

408401
/**
409402
* Reduces the given query results and consumes all aggregations and profile results.
410403
* @param queryResults a list of non-null query shard results
411404
*/
412-
public ReducedQueryPhase reducedQueryPhase(Collection<? extends SearchPhaseResult> queryResults, boolean isScrollRequest) {
413-
return reducedQueryPhase(queryResults, isScrollRequest, true);
405+
public ReducedQueryPhase reducedScrollQueryPhase(Collection<? extends SearchPhaseResult> queryResults) {
406+
return reducedQueryPhase(queryResults, true, true);
414407
}
415408

416409
/**
@@ -422,7 +415,6 @@ public ReducedQueryPhase reducedQueryPhase(Collection<? extends SearchPhaseResul
422415
return reducedQueryPhase(queryResults, null, new ArrayList<>(), new TopDocsStats(trackTotalHits), 0, isScrollRequest);
423416
}
424417

425-
426418
/**
427419
* Reduces the given query results and consumes all aggregations and profile results.
428420
* @param queryResults a list of non-null query shard results
@@ -507,15 +499,13 @@ private ReducedQueryPhase reducedQueryPhase(Collection<? extends SearchPhaseResu
507499
final InternalAggregations aggregations = aggregationsList.isEmpty() ? null : reduceAggs(aggregationsList,
508500
firstResult.pipelineAggregators(), reduceContext);
509501
final SearchProfileShardResults shardResults = profileResults.isEmpty() ? null : new SearchProfileShardResults(profileResults);
510-
final SortedTopDocs scoreDocs = this.sortDocs(isScrollRequest, queryResults, bufferedTopDocs, topDocsStats, from, size);
502+
final SortedTopDocs scoreDocs = sortDocs(isScrollRequest, queryResults, bufferedTopDocs, topDocsStats, from, size);
511503
final TotalHits totalHits = topDocsStats.getTotalHits();
512504
return new ReducedQueryPhase(totalHits, topDocsStats.fetchHits, topDocsStats.maxScore,
513505
timedOut, terminatedEarly, suggest, aggregations, shardResults, scoreDocs.scoreDocs, scoreDocs.sortFields,
514-
firstResult != null ? firstResult.sortValueFormats() : null,
515-
numReducePhases, scoreDocs.isSortedByField, size, from, firstResult == null);
506+
firstResult.sortValueFormats(), numReducePhases, scoreDocs.isSortedByField, size, from, false);
516507
}
517508

518-
519509
/**
520510
* Performs an intermediate reduce phase on the aggregations. For instance with this reduce phase never prune information
521511
* that relevant for the final reduce step. For final reduce see {@link #reduceAggs(List, List, ReduceContext)}
@@ -526,7 +516,7 @@ private InternalAggregations reduceAggsIncrementally(List<InternalAggregations>
526516
null, reduceContext);
527517
}
528518

529-
private InternalAggregations reduceAggs(List<InternalAggregations> aggregationsList,
519+
private static InternalAggregations reduceAggs(List<InternalAggregations> aggregationsList,
530520
List<SiblingPipelineAggregator> pipelineAggregators, ReduceContext reduceContext) {
531521
InternalAggregations aggregations = InternalAggregations.reduce(aggregationsList, reduceContext);
532522
if (pipelineAggregators != null) {
@@ -657,7 +647,6 @@ private QueryPhaseResultConsumer(SearchPhaseController controller, int expectedR
657647
this.hasTopDocs = hasTopDocs;
658648
this.hasAggs = hasAggs;
659649
this.bufferSize = bufferSize;
660-
661650
}
662651

663652
@Override
@@ -675,10 +664,9 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) {
675664
aggsBuffer[0] = reducedAggs;
676665
}
677666
if (hasTopDocs) {
678-
TopDocs reducedTopDocs = controller.mergeTopDocs(Arrays.asList(topDocsBuffer),
667+
TopDocs reducedTopDocs = mergeTopDocs(Arrays.asList(topDocsBuffer),
679668
// we have to merge here in the same way we collect on a shard
680-
querySearchResult.from() + querySearchResult.size()
681-
, 0);
669+
querySearchResult.from() + querySearchResult.size(), 0);
682670
Arrays.fill(topDocsBuffer, null);
683671
topDocsBuffer[0] = reducedTopDocs;
684672
}
@@ -692,7 +680,7 @@ private synchronized void consumeInternal(QuerySearchResult querySearchResult) {
692680
if (hasTopDocs) {
693681
final TopDocsAndMaxScore topDocs = querySearchResult.consumeTopDocs(); // can't be null
694682
topDocsStats.add(topDocs);
695-
SearchPhaseController.setShardIndex(topDocs.topDocs, querySearchResult.getShardIndex());
683+
setShardIndex(topDocs.topDocs, querySearchResult.getShardIndex());
696684
topDocsBuffer[i] = topDocs.topDocs;
697685
}
698686
}
@@ -705,7 +693,6 @@ private synchronized List<TopDocs> getRemainingTopDocs() {
705693
return hasTopDocs ? Arrays.asList(topDocsBuffer).subList(0, index) : null;
706694
}
707695

708-
709696
@Override
710697
public ReducedQueryPhase reduce() {
711698
return controller.reducedQueryPhase(results.asList(), getRemainingAggs(), getRemainingTopDocs(), topDocsStats,
@@ -739,7 +726,7 @@ InitialSearchPhase.ArraySearchPhaseResults<SearchPhaseResult> newSearchPhaseResu
739726
return new QueryPhaseResultConsumer(this, numShards, request.getBatchedReduceSize(), hasTopDocs, hasAggs);
740727
}
741728
}
742-
return new InitialSearchPhase.ArraySearchPhaseResults(numShards) {
729+
return new InitialSearchPhase.ArraySearchPhaseResults<SearchPhaseResult>(numShards) {
743730
@Override
744731
public ReducedQueryPhase reduce() {
745732
return reducedQueryPhase(results.asList(), isScrollRequest, trackTotalHits);

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ protected void executeInitialPhase(Transport.Connection connection, InternalScro
5252

5353
@Override
5454
protected SearchPhase moveToNextPhase(BiFunction<String, String, DiscoveryNode> clusterNodeLookup) {
55-
return sendResponsePhase(searchPhaseController.reducedQueryPhase(queryFetchResults.asList(), true), queryFetchResults);
55+
return sendResponsePhase(searchPhaseController.reducedScrollQueryPhase(queryFetchResults.asList()), queryFetchResults);
5656
}
5757

5858
@Override

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

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -69,8 +69,8 @@ protected SearchPhase moveToNextPhase(BiFunction<String, String, DiscoveryNode>
6969
return new SearchPhase("fetch") {
7070
@Override
7171
public void run() throws IOException {
72-
final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = searchPhaseController.reducedQueryPhase(
73-
queryResults.asList(), true);
72+
final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = searchPhaseController.reducedScrollQueryPhase(
73+
queryResults.asList());
7474
if (reducedQueryPhase.scoreDocs.length == 0) {
7575
sendResponse(reducedQueryPhase, fetchResults);
7676
return;

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@
2525
import org.elasticsearch.transport.TransportResponse;
2626

2727
/**
28-
* This class is a base class for all search releated results. It contains the shard target it
28+
* This class is a base class for all search related results. It contains the shard target it
2929
* was executed against, a shard index used to reference the result on the coordinating node
3030
* and a request ID that is used to reference the request context on the executing node. The
3131
* request ID is particularly important since it is used to reference and maintain a context

server/src/main/java/org/elasticsearch/search/aggregations/metrics/InternalTopHits.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,6 @@ public InternalTopHits(StreamInput in) throws IOException {
6565
from = in.readVInt();
6666
size = in.readVInt();
6767
topDocs = Lucene.readTopDocs(in);
68-
assert topDocs != null;
6968
searchHits = SearchHits.readSearchHits(in);
7069
}
7170

server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,8 @@
3030
import org.elasticsearch.common.util.concurrent.AtomicArray;
3131
import org.elasticsearch.index.Index;
3232
import org.elasticsearch.search.DocValueFormat;
33+
import org.elasticsearch.search.SearchHit;
34+
import org.elasticsearch.search.SearchHits;
3335
import org.elasticsearch.search.SearchPhaseResult;
3436
import org.elasticsearch.search.SearchShardTarget;
3537
import org.elasticsearch.search.aggregations.AggregationBuilders;
@@ -38,8 +40,6 @@
3840
import org.elasticsearch.search.aggregations.metrics.InternalMax;
3941
import org.elasticsearch.search.builder.SearchSourceBuilder;
4042
import org.elasticsearch.search.fetch.FetchSearchResult;
41-
import org.elasticsearch.search.SearchHit;
42-
import org.elasticsearch.search.SearchHits;
4343
import org.elasticsearch.search.internal.InternalSearchResponse;
4444
import org.elasticsearch.search.query.QuerySearchResult;
4545
import org.elasticsearch.search.suggest.Suggest;
@@ -73,7 +73,7 @@ public void setup() {
7373
(b) -> new InternalAggregation.ReduceContext(BigArrays.NON_RECYCLING_INSTANCE, null, b));
7474
}
7575

76-
public void testSort() throws Exception {
76+
public void testSort() {
7777
List<CompletionSuggestion> suggestions = new ArrayList<>();
7878
for (int i = 0; i < randomIntBetween(1, 5); i++) {
7979
suggestions.add(new CompletionSuggestion(randomAlphaOfLength(randomIntBetween(1, 5)), randomIntBetween(1, 20), false));
@@ -88,7 +88,7 @@ public void testSort() throws Exception {
8888
size = first.get().queryResult().size();
8989
}
9090
int accumulatedLength = Math.min(queryResultSize, getTotalQueryHits(results));
91-
ScoreDoc[] sortedDocs = searchPhaseController.sortDocs(true, results.asList(), null, new SearchPhaseController.TopDocsStats(),
91+
ScoreDoc[] sortedDocs = SearchPhaseController.sortDocs(true, results.asList(), null, new SearchPhaseController.TopDocsStats(),
9292
from, size)
9393
.scoreDocs;
9494
for (Suggest.Suggestion<?> suggestion : reducedSuggest(results)) {
@@ -113,12 +113,12 @@ public void testSortIsIdempotent() throws Exception {
113113
size = first.get().queryResult().size();
114114
}
115115
SearchPhaseController.TopDocsStats topDocsStats = new SearchPhaseController.TopDocsStats();
116-
ScoreDoc[] sortedDocs = searchPhaseController.sortDocs(ignoreFrom, results.asList(), null, topDocsStats, from, size).scoreDocs;
116+
ScoreDoc[] sortedDocs = SearchPhaseController.sortDocs(ignoreFrom, results.asList(), null, topDocsStats, from, size).scoreDocs;
117117

118118
results = generateSeededQueryResults(randomSeed, nShards, Collections.emptyList(), queryResultSize,
119119
useConstantScore);
120120
SearchPhaseController.TopDocsStats topDocsStats2 = new SearchPhaseController.TopDocsStats();
121-
ScoreDoc[] sortedDocs2 = searchPhaseController.sortDocs(ignoreFrom, results.asList(), null, topDocsStats2, from, size).scoreDocs;
121+
ScoreDoc[] sortedDocs2 = SearchPhaseController.sortDocs(ignoreFrom, results.asList(), null, topDocsStats2, from, size).scoreDocs;
122122
assertEquals(sortedDocs.length, sortedDocs2.length);
123123
for (int i = 0; i < sortedDocs.length; i++) {
124124
assertEquals(sortedDocs[i].doc, sortedDocs2[i].doc);

0 commit comments

Comments
 (0)