|
@@ -19,35 +19,23 @@
|
|
|
|
|
|
package org.elasticsearch.search.query;
|
|
|
|
|
|
-import org.apache.lucene.index.LeafReaderContext;
|
|
|
-import org.apache.lucene.index.Term;
|
|
|
+import org.apache.lucene.index.IndexReader;
|
|
|
import org.apache.lucene.queries.MinDocQuery;
|
|
|
import org.apache.lucene.search.BooleanClause;
|
|
|
import org.apache.lucene.search.BooleanQuery;
|
|
|
import org.apache.lucene.search.Collector;
|
|
|
import org.apache.lucene.search.ConstantScoreQuery;
|
|
|
-import org.apache.lucene.search.FieldDoc;
|
|
|
+import org.apache.lucene.search.EarlyTerminatingSortingCollector;
|
|
|
import org.apache.lucene.search.IndexSearcher;
|
|
|
import org.apache.lucene.search.MatchAllDocsQuery;
|
|
|
-import org.apache.lucene.search.MultiCollector;
|
|
|
import org.apache.lucene.search.Query;
|
|
|
import org.apache.lucene.search.ScoreDoc;
|
|
|
import org.apache.lucene.search.Sort;
|
|
|
-import org.apache.lucene.search.TermQuery;
|
|
|
import org.apache.lucene.search.TimeLimitingCollector;
|
|
|
import org.apache.lucene.search.TopDocs;
|
|
|
-import org.apache.lucene.search.TopDocsCollector;
|
|
|
-import org.apache.lucene.search.TopFieldCollector;
|
|
|
-import org.apache.lucene.search.TopScoreDocCollector;
|
|
|
-import org.apache.lucene.search.TotalHitCountCollector;
|
|
|
-import org.apache.lucene.search.Weight;
|
|
|
-import org.apache.lucene.search.grouping.CollapsingTopDocsCollector;
|
|
|
-import org.elasticsearch.action.search.SearchType;
|
|
|
+import org.elasticsearch.common.Nullable;
|
|
|
import org.elasticsearch.common.lucene.Lucene;
|
|
|
-import org.elasticsearch.common.lucene.MinimumScoreCollector;
|
|
|
-import org.elasticsearch.common.lucene.search.FilteredCollector;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
|
-import org.elasticsearch.search.collapse.CollapseContext;
|
|
|
import org.elasticsearch.search.DocValueFormat;
|
|
|
import org.elasticsearch.search.SearchPhase;
|
|
|
import org.elasticsearch.search.SearchService;
|
|
@@ -56,18 +44,22 @@ import org.elasticsearch.search.internal.ScrollContext;
|
|
|
import org.elasticsearch.search.internal.SearchContext;
|
|
|
import org.elasticsearch.search.profile.ProfileShardResult;
|
|
|
import org.elasticsearch.search.profile.SearchProfileShardResults;
|
|
|
-import org.elasticsearch.search.profile.query.CollectorResult;
|
|
|
import org.elasticsearch.search.profile.query.InternalProfileCollector;
|
|
|
import org.elasticsearch.search.rescore.RescorePhase;
|
|
|
-import org.elasticsearch.search.rescore.RescoreSearchContext;
|
|
|
import org.elasticsearch.search.sort.SortAndFormats;
|
|
|
import org.elasticsearch.search.suggest.SuggestPhase;
|
|
|
|
|
|
-import java.util.AbstractList;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Collections;
|
|
|
+import java.util.LinkedList;
|
|
|
import java.util.List;
|
|
|
-import java.util.concurrent.Callable;
|
|
|
+
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createCancellableCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createEarlySortingTerminationCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createEarlyTerminationCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createFilteredCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createMinScoreCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createMultiCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.QueryCollectorContext.createTimeoutCollectorContext;
|
|
|
+import static org.elasticsearch.search.query.TopDocsCollectorContext.createTopDocsCollectorContext;
|
|
|
|
|
|
/**
|
|
|
* Query phase of a search request, used to run the query and get back from each shard information about the matching documents
|
|
@@ -104,8 +96,9 @@ public class QueryPhase implements SearchPhase {
|
|
|
// request, preProcess is called on the DFS phase phase, this is why we pre-process them
|
|
|
// here to make sure it happens during the QUERY phase
|
|
|
aggregationPhase.preProcess(searchContext);
|
|
|
-
|
|
|
- boolean rescore = execute(searchContext, searchContext.searcher());
|
|
|
+ Sort indexSort = searchContext.mapperService().getIndexSettings().getIndexSortConfig()
|
|
|
+ .buildIndexSort(searchContext.mapperService()::fullName, searchContext.fieldData()::getForField);
|
|
|
+ boolean rescore = execute(searchContext, searchContext.searcher(), indexSort);
|
|
|
|
|
|
if (rescore) { // only if we do a regular search
|
|
|
rescorePhase.execute(searchContext);
|
|
@@ -120,298 +113,149 @@ public class QueryPhase implements SearchPhase {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private static boolean returnsDocsInOrder(Query query, SortAndFormats sf) {
|
|
|
- if (sf == null || Sort.RELEVANCE.equals(sf.sort)) {
|
|
|
- // sort by score
|
|
|
- // queries that return constant scores will return docs in index
|
|
|
- // order since Lucene tie-breaks on the doc id
|
|
|
- return query.getClass() == ConstantScoreQuery.class
|
|
|
- || query.getClass() == MatchAllDocsQuery.class;
|
|
|
- } else {
|
|
|
- return Sort.INDEXORDER.equals(sf.sort);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* In a package-private method so that it can be tested without having to
|
|
|
* wire everything (mapperService, etc.)
|
|
|
* @return whether the rescoring phase should be executed
|
|
|
*/
|
|
|
- static boolean execute(SearchContext searchContext, final IndexSearcher searcher) throws QueryPhaseExecutionException {
|
|
|
+ static boolean execute(SearchContext searchContext, final IndexSearcher searcher, @Nullable Sort indexSort) throws QueryPhaseExecutionException {
|
|
|
QuerySearchResult queryResult = searchContext.queryResult();
|
|
|
queryResult.searchTimedOut(false);
|
|
|
|
|
|
- final boolean doProfile = searchContext.getProfilers() != null;
|
|
|
- boolean rescore = false;
|
|
|
try {
|
|
|
queryResult.from(searchContext.from());
|
|
|
queryResult.size(searchContext.size());
|
|
|
-
|
|
|
Query query = searchContext.query();
|
|
|
-
|
|
|
- final int totalNumDocs = searcher.getIndexReader().numDocs();
|
|
|
- int numDocs = Math.min(searchContext.from() + searchContext.size(), totalNumDocs);
|
|
|
-
|
|
|
- Collector collector;
|
|
|
- Callable<TopDocs> topDocsCallable;
|
|
|
- DocValueFormat[] sortValueFormats = new DocValueFormat[0];
|
|
|
-
|
|
|
assert query == searcher.rewrite(query); // already rewritten
|
|
|
|
|
|
- if (searchContext.size() == 0) { // no matter what the value of from is
|
|
|
- final TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector();
|
|
|
- collector = totalHitCountCollector;
|
|
|
- if (searchContext.getProfilers() != null) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_COUNT, Collections.emptyList());
|
|
|
- }
|
|
|
- topDocsCallable = () -> new TopDocs(totalHitCountCollector.getTotalHits(), Lucene.EMPTY_SCORE_DOCS, 0);
|
|
|
- } else {
|
|
|
- // Perhaps have a dedicated scroll phase?
|
|
|
- final ScrollContext scrollContext = searchContext.scrollContext();
|
|
|
- assert (scrollContext != null) == (searchContext.request().scroll() != null);
|
|
|
- final Collector topDocsCollector;
|
|
|
- ScoreDoc after = null;
|
|
|
- if (searchContext.request().scroll() != null) {
|
|
|
- numDocs = Math.min(searchContext.size(), totalNumDocs);
|
|
|
- after = scrollContext.lastEmittedDoc;
|
|
|
-
|
|
|
- if (returnsDocsInOrder(query, searchContext.sort())) {
|
|
|
- if (scrollContext.totalHits == -1) {
|
|
|
- // first round
|
|
|
- assert scrollContext.lastEmittedDoc == null;
|
|
|
- // there is not much that we can optimize here since we want to collect all
|
|
|
- // documents in order to get the total number of hits
|
|
|
- } else {
|
|
|
- // now this gets interesting: since we sort in index-order, we can directly
|
|
|
- // skip to the desired doc and stop collecting after ${size} matches
|
|
|
- if (scrollContext.lastEmittedDoc != null) {
|
|
|
- BooleanQuery bq = new BooleanQuery.Builder()
|
|
|
- .add(query, BooleanClause.Occur.MUST)
|
|
|
- .add(new MinDocQuery(after.doc + 1), BooleanClause.Occur.FILTER)
|
|
|
- .build();
|
|
|
- query = bq;
|
|
|
- }
|
|
|
- searchContext.terminateAfter(numDocs);
|
|
|
- }
|
|
|
- }
|
|
|
- } else {
|
|
|
- after = searchContext.searchAfter();
|
|
|
- }
|
|
|
- if (totalNumDocs == 0) {
|
|
|
- // top collectors don't like a size of 0
|
|
|
- numDocs = 1;
|
|
|
- }
|
|
|
- assert numDocs > 0;
|
|
|
- if (searchContext.collapse() == null) {
|
|
|
- if (searchContext.sort() != null) {
|
|
|
- SortAndFormats sf = searchContext.sort();
|
|
|
- topDocsCollector = TopFieldCollector.create(sf.sort, numDocs,
|
|
|
- (FieldDoc) after, true, searchContext.trackScores(), searchContext.trackScores());
|
|
|
- sortValueFormats = sf.formats;
|
|
|
+ final ScrollContext scrollContext = searchContext.scrollContext();
|
|
|
+ if (scrollContext != null) {
|
|
|
+ if (returnsDocsInOrder(query, searchContext.sort())) {
|
|
|
+ if (scrollContext.totalHits == -1) {
|
|
|
+ // first round
|
|
|
+ assert scrollContext.lastEmittedDoc == null;
|
|
|
+ // there is not much that we can optimize here since we want to collect all
|
|
|
+ // documents in order to get the total number of hits
|
|
|
} else {
|
|
|
- rescore = !searchContext.rescore().isEmpty();
|
|
|
- for (RescoreSearchContext rescoreContext : searchContext.rescore()) {
|
|
|
- numDocs = Math.max(rescoreContext.window(), numDocs);
|
|
|
+ // now this gets interesting: since we sort in index-order, we can directly
|
|
|
+ // skip to the desired doc
|
|
|
+ final ScoreDoc after = scrollContext.lastEmittedDoc;
|
|
|
+ if (after != null) {
|
|
|
+ BooleanQuery bq = new BooleanQuery.Builder()
|
|
|
+ .add(query, BooleanClause.Occur.MUST)
|
|
|
+ .add(new MinDocQuery(after.doc + 1), BooleanClause.Occur.FILTER)
|
|
|
+ .build();
|
|
|
+ query = bq;
|
|
|
}
|
|
|
- topDocsCollector = TopScoreDocCollector.create(numDocs, after);
|
|
|
- }
|
|
|
- } else {
|
|
|
- Sort sort = Sort.RELEVANCE;
|
|
|
- if (searchContext.sort() != null) {
|
|
|
- sort = searchContext.sort().sort;
|
|
|
+ // ... and stop collecting after ${size} matches
|
|
|
+ searchContext.terminateAfter(searchContext.size());
|
|
|
+ searchContext.trackTotalHits(false);
|
|
|
}
|
|
|
- CollapseContext collapse = searchContext.collapse();
|
|
|
- topDocsCollector = collapse.createTopDocs(sort, numDocs, searchContext.trackScores());
|
|
|
- if (searchContext.sort() == null) {
|
|
|
- sortValueFormats = new DocValueFormat[] {DocValueFormat.RAW};
|
|
|
- } else {
|
|
|
- sortValueFormats = searchContext.sort().formats;
|
|
|
- }
|
|
|
- }
|
|
|
- collector = topDocsCollector;
|
|
|
- if (doProfile) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_TOP_HITS, Collections.emptyList());
|
|
|
- }
|
|
|
- topDocsCallable = () -> {
|
|
|
- final TopDocs topDocs;
|
|
|
- if (topDocsCollector instanceof TopDocsCollector) {
|
|
|
- topDocs = ((TopDocsCollector<?>) topDocsCollector).topDocs();
|
|
|
- } else if (topDocsCollector instanceof CollapsingTopDocsCollector) {
|
|
|
- topDocs = ((CollapsingTopDocsCollector) topDocsCollector).getTopDocs();
|
|
|
- } else {
|
|
|
- throw new IllegalStateException("Unknown top docs collector " + topDocsCollector.getClass().getName());
|
|
|
- }
|
|
|
- if (scrollContext != null) {
|
|
|
- if (scrollContext.totalHits == -1) {
|
|
|
- // first round
|
|
|
- scrollContext.totalHits = topDocs.totalHits;
|
|
|
- scrollContext.maxScore = topDocs.getMaxScore();
|
|
|
- } else {
|
|
|
- // subsequent round: the total number of hits and
|
|
|
- // the maximum score were computed on the first round
|
|
|
- topDocs.totalHits = scrollContext.totalHits;
|
|
|
- topDocs.setMaxScore(scrollContext.maxScore);
|
|
|
- }
|
|
|
- if (searchContext.request().numberOfShards() == 1) {
|
|
|
- // if we fetch the document in the same roundtrip, we already know the last emitted doc
|
|
|
- if (topDocs.scoreDocs.length > 0) {
|
|
|
- // set the last emitted doc
|
|
|
- scrollContext.lastEmittedDoc = topDocs.scoreDocs[topDocs.scoreDocs.length - 1];
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return topDocs;
|
|
|
- };
|
|
|
- }
|
|
|
-
|
|
|
- final boolean terminateAfterSet = searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER;
|
|
|
- if (terminateAfterSet) {
|
|
|
- final Collector child = collector;
|
|
|
- // throws Lucene.EarlyTerminationException when given count is reached
|
|
|
- collector = Lucene.wrapCountBasedEarlyTerminatingCollector(collector, searchContext.terminateAfter());
|
|
|
- if (doProfile) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_TERMINATE_AFTER_COUNT,
|
|
|
- Collections.singletonList((InternalProfileCollector) child));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ final LinkedList<QueryCollectorContext> collectors = new LinkedList<>();
|
|
|
if (searchContext.parsedPostFilter() != null) {
|
|
|
- final Collector child = collector;
|
|
|
- // this will only get applied to the actual search collector and not
|
|
|
- // to any scoped collectors, also, it will only be applied to the main collector
|
|
|
- // since that is where the filter should only work
|
|
|
- final Weight filterWeight = searcher.createNormalizedWeight(searchContext.parsedPostFilter().query(), false);
|
|
|
- collector = new FilteredCollector(collector, filterWeight);
|
|
|
- if (doProfile) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_POST_FILTER,
|
|
|
- Collections.singletonList((InternalProfileCollector) child));
|
|
|
- }
|
|
|
+ // add post filters before aggregations
|
|
|
+ // it will only be applied to top hits
|
|
|
+ collectors.add(createFilteredCollectorContext(searcher, searchContext.parsedPostFilter().query()));
|
|
|
}
|
|
|
-
|
|
|
- // plug in additional collectors, like aggregations
|
|
|
- final List<Collector> subCollectors = new ArrayList<>();
|
|
|
- subCollectors.add(collector);
|
|
|
- subCollectors.addAll(searchContext.queryCollectors().values());
|
|
|
- collector = MultiCollector.wrap(subCollectors);
|
|
|
- if (doProfile && collector instanceof InternalProfileCollector == false) {
|
|
|
- // When there is a single collector to wrap, MultiCollector returns it
|
|
|
- // directly, so only wrap in the case that there are several sub collectors
|
|
|
- final List<InternalProfileCollector> children = new AbstractList<InternalProfileCollector>() {
|
|
|
- @Override
|
|
|
- public InternalProfileCollector get(int index) {
|
|
|
- return (InternalProfileCollector) subCollectors.get(index);
|
|
|
- }
|
|
|
- @Override
|
|
|
- public int size() {
|
|
|
- return subCollectors.size();
|
|
|
- }
|
|
|
- };
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_MULTI, children);
|
|
|
+ if (searchContext.queryCollectors().isEmpty() == false) {
|
|
|
+ // plug in additional collectors, like aggregations
|
|
|
+ collectors.add(createMultiCollectorContext(searchContext.queryCollectors().values()));
|
|
|
}
|
|
|
-
|
|
|
- // apply the minimum score after multi collector so we filter aggs as well
|
|
|
if (searchContext.minimumScore() != null) {
|
|
|
- final Collector child = collector;
|
|
|
- collector = new MinimumScoreCollector(collector, searchContext.minimumScore());
|
|
|
- if (doProfile) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_MIN_SCORE,
|
|
|
- Collections.singletonList((InternalProfileCollector) child));
|
|
|
- }
|
|
|
+ // apply the minimum score after multi collector so we filter aggs as well
|
|
|
+ collectors.add(createMinScoreCollectorContext(searchContext.minimumScore()));
|
|
|
}
|
|
|
-
|
|
|
- if (collector.getClass() == TotalHitCountCollector.class) {
|
|
|
- // Optimize counts in simple cases to return in constant time
|
|
|
- // instead of using a collector
|
|
|
- while (true) {
|
|
|
- // remove wrappers that don't matter for counts
|
|
|
- // this is necessary so that we don't only optimize match_all
|
|
|
- // queries but also match_all queries that are nested in
|
|
|
- // a constant_score query
|
|
|
- if (query instanceof ConstantScoreQuery) {
|
|
|
- query = ((ConstantScoreQuery) query).getQuery();
|
|
|
- } else {
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (query.getClass() == MatchAllDocsQuery.class) {
|
|
|
- collector = null;
|
|
|
- topDocsCallable = new Callable<TopDocs>() {
|
|
|
- @Override
|
|
|
- public TopDocs call() throws Exception {
|
|
|
- int count = searcher.getIndexReader().numDocs();
|
|
|
- return new TopDocs(count, Lucene.EMPTY_SCORE_DOCS, 0);
|
|
|
- }
|
|
|
- };
|
|
|
- } else if (query.getClass() == TermQuery.class && searcher.getIndexReader().hasDeletions() == false) {
|
|
|
- final Term term = ((TermQuery) query).getTerm();
|
|
|
- collector = null;
|
|
|
- topDocsCallable = new Callable<TopDocs>() {
|
|
|
- @Override
|
|
|
- public TopDocs call() throws Exception {
|
|
|
- int count = 0;
|
|
|
- for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
|
|
|
- count += context.reader().docFreq(term);
|
|
|
- }
|
|
|
- return new TopDocs(count, Lucene.EMPTY_SCORE_DOCS, 0);
|
|
|
- }
|
|
|
- };
|
|
|
- }
|
|
|
+ if (searchContext.terminateAfter() != SearchContext.DEFAULT_TERMINATE_AFTER) {
|
|
|
+ // apply terminate after after all filters collectors
|
|
|
+ collectors.add(createEarlyTerminationCollectorContext(searchContext.terminateAfter()));
|
|
|
}
|
|
|
|
|
|
- final boolean timeoutSet = searchContext.timeout() != null && !searchContext.timeout().equals(SearchService.NO_TIMEOUT);
|
|
|
- if (timeoutSet && collector != null) { // collector might be null if no collection is actually needed
|
|
|
- final Collector child = collector;
|
|
|
+ boolean timeoutSet = scrollContext == null && searchContext.timeout() != null &&
|
|
|
+ searchContext.timeout().equals(SearchService.NO_TIMEOUT) == false;
|
|
|
+ if (timeoutSet) {
|
|
|
// TODO: change to use our own counter that uses the scheduler in ThreadPool
|
|
|
// throws TimeLimitingCollector.TimeExceededException when timeout has reached
|
|
|
- collector = Lucene.wrapTimeLimitingCollector(collector, searchContext.timeEstimateCounter(), searchContext.timeout().millis());
|
|
|
- if (doProfile) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_TIMEOUT,
|
|
|
- Collections.singletonList((InternalProfileCollector) child));
|
|
|
- }
|
|
|
+ collectors.add(createTimeoutCollectorContext(searchContext.timeEstimateCounter(), searchContext.timeout().millis()));
|
|
|
}
|
|
|
-
|
|
|
- if (collector != null) {
|
|
|
- final Collector child = collector;
|
|
|
- collector = new CancellableCollector(searchContext.getTask()::isCancelled, searchContext.lowLevelCancellation(), collector);
|
|
|
- if (doProfile) {
|
|
|
- collector = new InternalProfileCollector(collector, CollectorResult.REASON_SEARCH_CANCELLED,
|
|
|
- Collections.singletonList((InternalProfileCollector) child));
|
|
|
- }
|
|
|
+ // add cancellable
|
|
|
+ collectors.add(createCancellableCollectorContext(searchContext.getTask()::isCancelled, searchContext.lowLevelCancellation()));
|
|
|
+
|
|
|
+ final IndexReader reader = searcher.getIndexReader();
|
|
|
+ final boolean doProfile = searchContext.getProfilers() != null;
|
|
|
+ // create the top docs collector last when the other collectors are known
|
|
|
+ final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext, reader,
|
|
|
+ collectors.stream().anyMatch(QueryCollectorContext::shouldCollect));
|
|
|
+ final boolean shouldCollect = topDocsFactory.shouldCollect();
|
|
|
+ if (scrollContext == null && topDocsFactory.numHits() > 0 && canEarlyTerminate(indexSort, searchContext)) {
|
|
|
+ // top docs collection can be early terminated based on index sort
|
|
|
+ // add the collector context first so we don't early terminate aggs but only top docs
|
|
|
+ collectors.addFirst(createEarlySortingTerminationCollectorContext(reader, searchContext.query(), indexSort,
|
|
|
+ topDocsFactory.numHits(), searchContext.trackTotalHits(), shouldCollect));
|
|
|
+ }
|
|
|
+ // add the top docs collector, the first collector context in the chain
|
|
|
+ collectors.addFirst(topDocsFactory);
|
|
|
+
|
|
|
+ final Collector queryCollector;
|
|
|
+ if (doProfile) {
|
|
|
+ InternalProfileCollector profileCollector = QueryCollectorContext.createQueryCollectorWithProfiler(collectors);
|
|
|
+ searchContext.getProfilers().getCurrentQueryProfiler().setCollector(profileCollector);
|
|
|
+ queryCollector = profileCollector;
|
|
|
+ } else {
|
|
|
+ queryCollector = QueryCollectorContext.createQueryCollector(collectors);
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- if (collector != null) {
|
|
|
- if (doProfile) {
|
|
|
- searchContext.getProfilers().getCurrentQueryProfiler().setCollector((InternalProfileCollector) collector);
|
|
|
- }
|
|
|
- searcher.search(query, collector);
|
|
|
+ if (shouldCollect) {
|
|
|
+ searcher.search(query, queryCollector);
|
|
|
}
|
|
|
} catch (TimeLimitingCollector.TimeExceededException e) {
|
|
|
assert timeoutSet : "TimeExceededException thrown even though timeout wasn't set";
|
|
|
queryResult.searchTimedOut(true);
|
|
|
- } catch (Lucene.EarlyTerminationException e) {
|
|
|
- assert terminateAfterSet : "EarlyTerminationException thrown even though terminateAfter wasn't set";
|
|
|
- queryResult.terminatedEarly(true);
|
|
|
} finally {
|
|
|
searchContext.clearReleasables(SearchContext.Lifetime.COLLECTION);
|
|
|
}
|
|
|
- if (terminateAfterSet && queryResult.terminatedEarly() == null) {
|
|
|
- queryResult.terminatedEarly(false);
|
|
|
- }
|
|
|
-
|
|
|
- queryResult.topDocs(topDocsCallable.call(), sortValueFormats);
|
|
|
|
|
|
+ final QuerySearchResult result = searchContext.queryResult();
|
|
|
+ for (QueryCollectorContext ctx : collectors) {
|
|
|
+ ctx.postProcess(result, shouldCollect);
|
|
|
+ }
|
|
|
if (searchContext.getProfilers() != null) {
|
|
|
- ProfileShardResult shardResults = SearchProfileShardResults
|
|
|
- .buildShardResults(searchContext.getProfilers());
|
|
|
- searchContext.queryResult().profileResults(shardResults);
|
|
|
+ ProfileShardResult shardResults = SearchProfileShardResults.buildShardResults(searchContext.getProfilers());
|
|
|
+ result.profileResults(shardResults);
|
|
|
}
|
|
|
-
|
|
|
- return rescore;
|
|
|
-
|
|
|
+ return topDocsFactory.shouldRescore();
|
|
|
} catch (Exception e) {
|
|
|
throw new QueryPhaseExecutionException(searchContext, "Failed to execute main query", e);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns true if the provided <code>query</code> returns docs in index order (internal doc ids).
|
|
|
+ * @param query The query to execute
|
|
|
+ * @param sf The query sort
|
|
|
+ */
|
|
|
+ static boolean returnsDocsInOrder(Query query, SortAndFormats sf) {
|
|
|
+ if (sf == null || Sort.RELEVANCE.equals(sf.sort)) {
|
|
|
+ // sort by score
|
|
|
+ // queries that return constant scores will return docs in index
|
|
|
+ // order since Lucene tie-breaks on the doc id
|
|
|
+ return query.getClass() == ConstantScoreQuery.class
|
|
|
+ || query.getClass() == MatchAllDocsQuery.class;
|
|
|
+ } else {
|
|
|
+ return Sort.INDEXORDER.equals(sf.sort);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns true if the provided <code>searchContext</code> can early terminate based on <code>indexSort</code>
|
|
|
+ * @param indexSort The index sort specification
|
|
|
+ * @param context The search context for the request
|
|
|
+ */
|
|
|
+ static boolean canEarlyTerminate(Sort indexSort, SearchContext context) {
|
|
|
+ final Sort sort = context.sort() == null ? Sort.RELEVANCE : context.sort().sort;
|
|
|
+ return indexSort != null && EarlyTerminatingSortingCollector.canEarlyTerminate(sort, indexSort);
|
|
|
+ }
|
|
|
}
|