diff --git a/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java b/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java index a76493464784a..16865891b28fd 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java @@ -132,12 +132,67 @@ public Weight createWeight(Query query, ScoreMode scoreMode, float boost) throws } } + private void checkCancelled() { + if (checkCancelled != null) { + checkCancelled.run(); + } + } + @Override protected void search(List leaves, Weight weight, Collector collector) throws IOException { - final Weight cancellableWeight; - if (checkCancelled != null) { - cancellableWeight = new Weight(weight.getQuery()) { + for (LeafReaderContext ctx : leaves) { // search each subreader + searchLeaf(ctx, weight, collector); + } + } + /** + * Lower-level search API. + * + * {@link LeafCollector#collect(int)} is called for every matching document in + * the provided ctx. + */ + public void searchLeaf(LeafReaderContext ctx, Weight weight, Collector collector) throws IOException { + checkCancelled(); + weight = wrapWeight(weight); + final LeafCollector leafCollector; + try { + leafCollector = collector.getLeafCollector(ctx); + } catch (CollectionTerminatedException e) { + // there is no doc of interest in this reader context + // continue with the following leaf + return; + } + Bits liveDocs = ctx.reader().getLiveDocs(); + BitSet liveDocsBitSet = getSparseBitSetOrNull(ctx.reader().getLiveDocs()); + if (liveDocsBitSet == null) { + BulkScorer bulkScorer = weight.bulkScorer(ctx); + if (bulkScorer != null) { + try { + bulkScorer.score(leafCollector, liveDocs); + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf + } + } + } else { + // if the role query result set is sparse then we should use the SparseFixedBitSet for advancing: + Scorer scorer = weight.scorer(ctx); + if (scorer != null) { + try { + intersectScorerAndBitSet(scorer, liveDocsBitSet, leafCollector, + checkCancelled == null ? () -> { + } : checkCancelled); + } catch (CollectionTerminatedException e) { + // collection was terminated prematurely + // continue with the following leaf + } + } + } + } + + private Weight wrapWeight(Weight weight) { + if (checkCancelled != null) { + return new Weight(weight.getQuery()) { @Override public void extractTerms(Set terms) { throw new UnsupportedOperationException(); @@ -169,48 +224,10 @@ public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { } }; } else { - cancellableWeight = weight; + return weight; } - searchInternal(leaves, cancellableWeight, collector); } - private void searchInternal(List leaves, Weight weight, Collector collector) throws IOException { - for (LeafReaderContext ctx : leaves) { // search each subreader - final LeafCollector leafCollector; - try { - leafCollector = collector.getLeafCollector(ctx); - } catch (CollectionTerminatedException e) { - // there is no doc of interest in this reader context - // continue with the following leaf - continue; - } - Bits liveDocs = ctx.reader().getLiveDocs(); - BitSet liveDocsBitSet = getSparseBitSetOrNull(ctx.reader().getLiveDocs()); - if (liveDocsBitSet == null) { - BulkScorer bulkScorer = weight.bulkScorer(ctx); - if (bulkScorer != null) { - try { - bulkScorer.score(leafCollector, liveDocs); - } catch (CollectionTerminatedException e) { - // collection was terminated prematurely - // continue with the following leaf - } - } - } else { - // if the role query result set is sparse then we should use the SparseFixedBitSet for advancing: - Scorer scorer = weight.scorer(ctx); - if (scorer != null) { - try { - intersectScorerAndBitSet(scorer, liveDocsBitSet, leafCollector, - checkCancelled == null ? () -> {} : checkCancelled); - } catch (CollectionTerminatedException e) { - // collection was terminated prematurely - // continue with the following leaf - } - } - } - } - } private static BitSet getSparseBitSetOrNull(Bits liveDocs) { if (liveDocs instanceof SparseFixedBitSet) { diff --git a/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java b/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java index 19d382dd8f380..99490e3009fd7 100644 --- a/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java +++ b/server/src/main/java/org/elasticsearch/search/profile/query/CollectorResult.java @@ -49,8 +49,6 @@ public class CollectorResult implements ToXContentObject, Writeable { public static final String REASON_SEARCH_POST_FILTER = "search_post_filter"; public static final String REASON_SEARCH_MIN_SCORE = "search_min_score"; public static final String REASON_SEARCH_MULTI = "search_multi"; - public static final String REASON_SEARCH_TIMEOUT = "search_timeout"; - public static final String REASON_SEARCH_CANCELLED = "search_cancelled"; public static final String REASON_AGGREGATION = "aggregation"; public static final String REASON_AGGREGATION_GLOBAL = "aggregation_global"; diff --git a/server/src/main/java/org/elasticsearch/search/query/CancellableCollector.java b/server/src/main/java/org/elasticsearch/search/query/CancellableCollector.java deleted file mode 100644 index 504a7f3d13da5..0000000000000 --- a/server/src/main/java/org/elasticsearch/search/query/CancellableCollector.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.query; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.search.Collector; -import org.apache.lucene.search.FilterCollector; -import org.apache.lucene.search.LeafCollector; -import org.elasticsearch.tasks.TaskCancelledException; - -import java.io.IOException; -import java.util.function.BooleanSupplier; - -/** - * Collector that checks if the task it is executed under is cancelled. - */ -public class CancellableCollector extends FilterCollector { - private final BooleanSupplier cancelled; - - /** - * Constructor - * @param cancelled supplier of the cancellation flag, the supplier will be called for each segment - * @param in wrapped collector - */ - public CancellableCollector(BooleanSupplier cancelled, Collector in) { - super(in); - this.cancelled = cancelled; - } - - @Override - public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { - if (cancelled.getAsBoolean()) { - throw new TaskCancelledException("cancelled"); - } - return super.getLeafCollector(context); - } -} diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/QueryCollectorContext.java index f0c94bd822edf..b63739df76bfe 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryCollectorContext.java @@ -28,16 +28,13 @@ import org.elasticsearch.common.lucene.MinimumScoreCollector; import org.elasticsearch.common.lucene.search.FilteredCollector; import org.elasticsearch.search.profile.query.InternalProfileCollector; -import org.elasticsearch.tasks.TaskCancelledException; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.function.BooleanSupplier; -import static org.elasticsearch.search.profile.query.CollectorResult.REASON_SEARCH_CANCELLED; import static org.elasticsearch.search.profile.query.CollectorResult.REASON_SEARCH_MIN_SCORE; import static org.elasticsearch.search.profile.query.CollectorResult.REASON_SEARCH_MULTI; import static org.elasticsearch.search.profile.query.CollectorResult.REASON_SEARCH_POST_FILTER; @@ -150,18 +147,6 @@ protected InternalProfileCollector createWithProfiler(InternalProfileCollector i }; } - /** - * Creates a collector that throws {@link TaskCancelledException} if the search is cancelled - */ - static QueryCollectorContext createCancellableCollectorContext(BooleanSupplier cancelled) { - return new QueryCollectorContext(REASON_SEARCH_CANCELLED) { - @Override - Collector create(Collector in) throws IOException { - return new CancellableCollector(cancelled, in); - } - }; - } - /** * Creates collector limiting the collection to the first numHits documents */ diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java index b3ca4a04ea75f..11f28c48f6f6c 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -34,7 +34,6 @@ import org.apache.lucene.search.ConstantScoreQuery; import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.FieldDoc; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; @@ -43,8 +42,10 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.search.TotalHits; +import org.apache.lucene.search.Weight; import org.elasticsearch.action.search.SearchTask; import org.elasticsearch.common.Booleans; +import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.concurrent.QueueResizingEsThreadPoolExecutor; @@ -68,12 +69,14 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; import java.util.LinkedList; +import java.util.List; import java.util.concurrent.ExecutorService; -import java.util.function.Consumer; -import static org.elasticsearch.search.query.QueryCollectorContext.createCancellableCollectorContext; import static org.elasticsearch.search.query.QueryCollectorContext.createEarlyTerminationCollectorContext; import static org.elasticsearch.search.query.QueryCollectorContext.createFilteredCollectorContext; import static org.elasticsearch.search.query.QueryCollectorContext.createMinScoreCollectorContext; @@ -89,7 +92,7 @@ public class QueryPhase implements SearchPhase { private static final Logger LOGGER = LogManager.getLogger(QueryPhase.class); public static final boolean SYS_PROP_LONG_SORT_OPTIMIZED = - Booleans.parseBoolean(System.getProperty("es.search.long_sort_optimized", "false")); + Booleans.parseBoolean(System.getProperty("es.search.long_sort_optimized", "true")); private final AggregationPhase aggregationPhase; private final SuggestPhase suggestPhase; @@ -124,8 +127,7 @@ public void execute(SearchContext searchContext) throws QueryPhaseExecutionExcep // 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); - final ContextIndexSearcher searcher = searchContext.searcher(); - boolean rescore = execute(searchContext, searchContext.searcher(), searcher::setCheckCancelled); + boolean rescore = executeInternal(searchContext); if (rescore) { // only if we do a regular search rescorePhase.execute(searchContext); @@ -145,9 +147,8 @@ public void execute(SearchContext searchContext) throws QueryPhaseExecutionExcep * wire everything (mapperService, etc.) * @return whether the rescoring phase should be executed */ - static boolean execute(SearchContext searchContext, - final IndexSearcher searcher, - Consumer checkCancellationSetter) throws QueryPhaseExecutionException { + static boolean executeInternal(SearchContext searchContext) throws QueryPhaseExecutionException { + final ContextIndexSearcher searcher = searchContext.searcher(); SortAndFormats sortAndFormatsForRewrittenNumericSort = null; final IndexReader reader = searcher.getIndexReader(); QuerySearchResult queryResult = searchContext.queryResult(); @@ -220,6 +221,7 @@ static boolean execute(SearchContext searchContext, hasFilterCollector = true; } + CheckedConsumer, IOException> leafSorter = l -> {}; // try to rewrite numeric or date sort to the optimized distanceFeatureQuery if ((searchContext.sort() != null) && SYS_PROP_LONG_SORT_OPTIMIZED) { Query rewrittenQuery = tryRewriteLongSort(searchContext, searcher.getIndexReader(), query, hasFilterCollector); @@ -228,14 +230,20 @@ static boolean execute(SearchContext searchContext, // modify sorts: add sort on _score as 1st sort, and move the sort on the original field as the 2nd sort SortField[] oldSortFields = searchContext.sort().sort.getSort(); DocValueFormat[] oldFormats = searchContext.sort().formats; - SortField[] newSortFields = new SortField[oldSortFields.length + 1]; - DocValueFormat[] newFormats = new DocValueFormat[oldSortFields.length + 1]; + SortField[] newSortFields = new SortField[oldSortFields.length + 2]; + DocValueFormat[] newFormats = new DocValueFormat[oldSortFields.length + 2]; newSortFields[0] = SortField.FIELD_SCORE; newFormats[0] = DocValueFormat.RAW; + // Add a tiebreak on _doc in order to be able to search + // the leaves in any order. This is needed since we reorder + // the leaves based on the minimum value in each segment. + newSortFields[newSortFields.length-1] = SortField.FIELD_DOC; + newFormats[newSortFields.length-1] = DocValueFormat.RAW; System.arraycopy(oldSortFields, 0, newSortFields, 1, oldSortFields.length); System.arraycopy(oldFormats, 0, newFormats, 1, oldFormats.length); sortAndFormatsForRewrittenNumericSort = searchContext.sort(); // stash SortAndFormats to restore it later searchContext.sort(new SortAndFormats(new Sort(newSortFields), newFormats)); + leafSorter = createLeafSorter(oldSortFields[0]); } } @@ -279,16 +287,11 @@ static boolean execute(SearchContext searchContext, checkCancelled = null; } - checkCancellationSetter.accept(checkCancelled); - - // add cancellable - // this only performs segment-level cancellation, which is cheap and checked regardless of - // searchContext.lowLevelCancellation() - collectors.add(createCancellableCollectorContext(searchContext.getTask()::isCancelled)); + searcher.setCheckCancelled(checkCancelled); final boolean doProfile = searchContext.getProfilers() != null; // create the top docs collector last when the other collectors are known - final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext, reader, hasFilterCollector); + final TopDocsCollectorContext topDocsFactory = createTopDocsCollectorContext(searchContext, hasFilterCollector); // add the top docs collector, the first collector context in the chain collectors.addFirst(topDocsFactory); @@ -302,7 +305,15 @@ static boolean execute(SearchContext searchContext, } try { - searcher.search(query, queryCollector); + Weight weight = searcher.createWeight(searcher.rewrite(query), queryCollector.scoreMode(), 1f); + // We search the leaves in a different order when the numeric sort optimization is + // activated. Collectors expect leaves in order when searching but this is fine in this + // case since we only have a TopFieldCollector and we force the tiebreak on _doc. + List leaves = new ArrayList<>(searcher.getIndexReader().leaves()); + leafSorter.accept(leaves); + for (LeafReaderContext ctx : leaves) { + searcher.searchLeaf(ctx, weight, queryCollector); + } } catch (EarlyTerminatingCollector.EarlyTerminationException e) { queryResult.terminatedEarly(true); } catch (TimeExceededException e) { @@ -427,13 +438,39 @@ private static Query tryRewriteLongSort(SearchContext searchContext, IndexReader return rewrittenQuery; } - // Restore fieldsDocs to remove the first _score sort - // updating in place without creating new FieldDoc objects + /** + * Creates a sorter of {@link LeafReaderContext} that orders leaves depending on the minimum + * value and the sort order of the provided sortField. + */ + static CheckedConsumer, IOException> createLeafSorter(SortField sortField) { + return leaves -> { + long[] sortValues = new long[leaves.size()]; + long missingValue = (long) sortField.getMissingValue(); + for (LeafReaderContext ctx : leaves) { + PointValues values = ctx.reader().getPointValues(sortField.getField()); + if (values == null) { + sortValues[ctx.ord] = missingValue; + } else { + byte[] sortValue = sortField.getReverse() ? values.getMaxPackedValue(): values.getMinPackedValue(); + sortValues[ctx.ord] = sortValue == null ? missingValue : LongPoint.decodeDimension(sortValue, 0); + } + } + Comparator comparator = Comparator.comparingLong(l -> sortValues[l.ord]); + if (sortField.getReverse()) { + comparator = comparator.reversed(); + } + Collections.sort(leaves, comparator); + }; + } + + /** + * Restore fieldsDocs to remove the first _score and last _doc sort. + */ static void restoreTopFieldDocs(QuerySearchResult result, SortAndFormats originalSortAndFormats) { TopDocs topDocs = result.topDocs().topDocs; for (ScoreDoc scoreDoc : topDocs.scoreDocs) { FieldDoc fieldDoc = (FieldDoc) scoreDoc; - fieldDoc.fields = Arrays.copyOfRange(fieldDoc.fields, 1, fieldDoc.fields.length); + fieldDoc.fields = Arrays.copyOfRange(fieldDoc.fields, 1, fieldDoc.fields.length-1); } TopFieldDocs newTopDocs = new TopFieldDocs(topDocs.totalHits, topDocs.scoreDocs, originalSortAndFormats.sort.getSort()); result.topDocs(new TopDocsAndMaxScore(newTopDocs, Float.NaN), originalSortAndFormats.formats); diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 5ae6cc739c362..8b795429959e6 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -428,8 +428,8 @@ static int shortcutTotalHitCount(IndexReader reader, Query query) throws IOExcep * @param hasFilterCollector True if the collector chain contains at least one collector that can filters document. */ static TopDocsCollectorContext createTopDocsCollectorContext(SearchContext searchContext, - IndexReader reader, boolean hasFilterCollector) throws IOException { + final IndexReader reader = searchContext.searcher().getIndexReader(); final Query query = searchContext.query(); // top collectors don't like a size of 0 final int totalNumDocs = Math.max(1, reader.numDocs()); diff --git a/server/src/test/java/org/elasticsearch/search/SearchCancellationTests.java b/server/src/test/java/org/elasticsearch/search/SearchCancellationTests.java index eba4a03e72cfa..4dbacc8ec87de 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchCancellationTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchCancellationTests.java @@ -22,14 +22,18 @@ import org.apache.lucene.document.Field; import org.apache.lucene.document.StringField; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.RandomIndexWriter; -import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.TotalHitCountCollector; +import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.elasticsearch.core.internal.io.IOUtils; import org.apache.lucene.util.TestUtil; -import org.elasticsearch.search.query.CancellableCollector; +import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.test.ESTestCase; import org.junit.AfterClass; @@ -38,6 +42,8 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.Matchers.equalTo; + public class SearchCancellationTests extends ESTestCase { static Directory dir; @@ -75,12 +81,22 @@ public static void cleanup() throws IOException { public void testCancellableCollector() throws IOException { TotalHitCountCollector collector = new TotalHitCountCollector(); AtomicBoolean cancelled = new AtomicBoolean(); - CancellableCollector cancellableCollector = new CancellableCollector(cancelled::get, collector); - final LeafCollector leafCollector = cancellableCollector.getLeafCollector(reader.leaves().get(0)); - leafCollector.collect(0); + ContextIndexSearcher searcher = new ContextIndexSearcher(reader, + IndexSearcher.getDefaultSimilarity(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy()); + searcher.setCheckCancelled(() -> { + if (cancelled.get()) { + throw new TaskCancelledException("cancelled"); + } + }); + LeafReaderContext leafContext = reader.leaves().get(0); + final Weight weight = searcher.createWeight(new MatchAllDocsQuery(), ScoreMode.COMPLETE, 1f); + searcher.searchLeaf(searcher.getIndexReader().leaves().get(0), weight, collector); + assertThat(collector.getTotalHits(), equalTo(leafContext.reader().numDocs())); cancelled.set(true); - leafCollector.collect(1); - expectThrows(TaskCancelledException.class, () -> cancellableCollector.getLeafCollector(reader.leaves().get(1))); + expectThrows(TaskCancelledException.class, + () -> searcher.searchLeaf(searcher.getIndexReader().leaves().get(0), weight, collector)); + expectThrows(TaskCancelledException.class, + () -> searcher.search(new MatchAllDocsQuery(), collector)); } } diff --git a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java index 1e885b4cabc70..cfb1083a9cbae 100644 --- a/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/QueryPhaseTests.java @@ -84,6 +84,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.internal.ContextIndexSearcher; import org.elasticsearch.search.internal.ScrollContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.sort.SortAndFormats; @@ -96,6 +97,7 @@ import static org.elasticsearch.search.query.QueryPhase.estimateMedianValue; import static org.elasticsearch.search.query.QueryPhase.estimatePointCount; +import static org.elasticsearch.search.query.TopDocsCollectorContext.hasInfMaxScore; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -128,18 +130,17 @@ public void tearDown() throws Exception { } private void countTestCase(Query query, IndexReader reader, boolean shouldCollectSearch, boolean shouldCollectCount) throws Exception { - TestSearchContext context = new TestSearchContext(null, indexShard); + ContextIndexSearcher searcher = shouldCollectSearch ? newContextSearcher(reader) : + newEarlyTerminationContextSearcher(reader, 0); + TestSearchContext context = new TestSearchContext(null, indexShard, searcher); context.parsedQuery(new ParsedQuery(query)); context.setSize(0); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); - - final IndexSearcher searcher = shouldCollectSearch ? new IndexSearcher(reader) : - getAssertingEarlyTerminationSearcher(reader, 0); - - final boolean rescore = QueryPhase.execute(context, searcher, checkCancelled -> {}); + final boolean rescore = QueryPhase.executeInternal(context); assertFalse(rescore); - IndexSearcher countSearcher = shouldCollectCount ? new IndexSearcher(reader) : - getAssertingEarlyTerminationSearcher(reader, 0); + + ContextIndexSearcher countSearcher = shouldCollectCount ? newContextSearcher(reader) : + newEarlyTerminationContextSearcher(reader, 0); assertEquals(countSearcher.count(query), context.queryResult().topDocs().topDocs.totalHits.value); } @@ -217,17 +218,17 @@ public void testPostFilterDisablesCountOptimization() throws Exception { w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = getAssertingEarlyTerminationSearcher(reader, 0); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = + new TestSearchContext(null, indexShard, newEarlyTerminationContextSearcher(reader, 0)); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(1, context.queryResult().topDocs().topDocs.totalHits.value); - contextSearcher = new IndexSearcher(reader); + context.setSearcher(newContextSearcher(reader)); context.parsedPostFilter(new ParsedQuery(new MatchNoDocsQuery())); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(0, context.queryResult().topDocs().topDocs.totalHits.value); reader.close(); dir.close(); @@ -247,15 +248,14 @@ public void testTerminateAfterWithFilter() throws Exception { w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); context.terminateAfter(1); context.setSize(10); for (int i = 0; i < 10; i++) { context.parsedPostFilter(new ParsedQuery(new TermQuery(new Term("foo", Integer.toString(i))))); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(1, context.queryResult().topDocs().topDocs.totalHits.value); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); } @@ -274,27 +274,22 @@ public void testMinScoreDisablesCountOptimization() throws Exception { w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = getAssertingEarlyTerminationSearcher(reader, 0); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = + new TestSearchContext(null, indexShard, newEarlyTerminationContextSearcher(reader, 0)); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); context.setSize(0); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(1, context.queryResult().topDocs().topDocs.totalHits.value); - contextSearcher = new IndexSearcher(reader); context.minimumScore(100); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(0, context.queryResult().topDocs().topDocs.totalHits.value); reader.close(); dir.close(); } public void testQueryCapturesThreadPoolStats() throws Exception { - TestSearchContext context = new TestSearchContext(null, indexShard); - context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); - context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); - Directory dir = newDirectory(); IndexWriterConfig iwc = newIndexWriterConfig(); RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc); @@ -304,9 +299,11 @@ public void testQueryCapturesThreadPoolStats() throws Exception { } w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); + context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); + context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); QuerySearchResult results = context.queryResult(); assertThat(results.serviceTimeEWMA(), greaterThanOrEqualTo(0L)); assertThat(results.nodeQueueSize(), greaterThanOrEqualTo(0)); @@ -326,8 +323,7 @@ public void testInOrderScrollOptimization() throws Exception { } w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); ScrollContext scrollContext = new ScrollContext(); scrollContext.lastEmittedDoc = null; @@ -338,14 +334,14 @@ public void testInOrderScrollOptimization() throws Exception { int size = randomIntBetween(2, 5); context.setSize(size); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertNull(context.queryResult().terminatedEarly()); assertThat(context.terminateAfter(), equalTo(0)); assertThat(context.queryResult().getTotalHits().value, equalTo((long) numDocs)); - contextSearcher = getAssertingEarlyTerminationSearcher(reader, size); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + context.setSearcher(newEarlyTerminationContextSearcher(reader, size)); + QueryPhase.executeInternal(context); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertThat(context.terminateAfter(), equalTo(size)); assertThat(context.queryResult().getTotalHits().value, equalTo((long) numDocs)); @@ -371,19 +367,17 @@ public void testTerminateAfterEarlyTermination() throws Exception { w.addDocument(doc); } w.close(); - TestSearchContext context = new TestSearchContext(null, indexShard); + final IndexReader reader = DirectoryReader.open(dir); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); - final IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); - context.terminateAfter(numDocs); { context.setSize(10); TotalHitCountCollector collector = new TotalHitCountCollector(); context.queryCollectors().put(TotalHitCountCollector.class, collector); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertFalse(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(10)); @@ -393,13 +387,13 @@ public void testTerminateAfterEarlyTermination() throws Exception { context.terminateAfter(1); { context.setSize(1); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); context.setSize(0); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(0)); @@ -407,7 +401,7 @@ public void testTerminateAfterEarlyTermination() throws Exception { { context.setSize(1); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); @@ -419,14 +413,14 @@ public void testTerminateAfterEarlyTermination() throws Exception { .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) .build(); context.parsedQuery(new ParsedQuery(bq)); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); context.setSize(0); context.parsedQuery(new ParsedQuery(bq)); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(0)); @@ -435,7 +429,7 @@ public void testTerminateAfterEarlyTermination() throws Exception { context.setSize(1); TotalHitCountCollector collector = new TotalHitCountCollector(); context.queryCollectors().put(TotalHitCountCollector.class, collector); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); @@ -446,7 +440,7 @@ public void testTerminateAfterEarlyTermination() throws Exception { context.setSize(0); TotalHitCountCollector collector = new TotalHitCountCollector(); context.queryCollectors().put(TotalHitCountCollector.class, collector); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertTrue(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(0)); @@ -476,15 +470,15 @@ public void testIndexSortingEarlyTermination() throws Exception { } w.close(); - TestSearchContext context = new TestSearchContext(null, indexShard); + final IndexReader reader = DirectoryReader.open(dir); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); context.setSize(1); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); context.sort(new SortAndFormats(sort, new DocValueFormat[] {DocValueFormat.RAW})); - final IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + + QueryPhase.executeInternal(context); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs[0], instanceOf(FieldDoc.class)); @@ -493,7 +487,7 @@ public void testIndexSortingEarlyTermination() throws Exception { { context.parsedPostFilter(new ParsedQuery(new MinDocQuery(1))); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertNull(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo(numDocs - 1L)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); @@ -503,7 +497,7 @@ public void testIndexSortingEarlyTermination() throws Exception { final TotalHitCountCollector totalHitCountCollector = new TotalHitCountCollector(); context.queryCollectors().put(TotalHitCountCollector.class, totalHitCountCollector); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertNull(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); @@ -514,15 +508,15 @@ public void testIndexSortingEarlyTermination() throws Exception { } { - contextSearcher = getAssertingEarlyTerminationSearcher(reader, 1); + context.setSearcher(newEarlyTerminationContextSearcher(reader, 1)); context.trackTotalHitsUpTo(SearchContext.TRACK_TOTAL_HITS_DISABLED); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertNull(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs[0], instanceOf(FieldDoc.class)); assertThat(fieldDoc.fields[0], anyOf(equalTo(1), equalTo(2))); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertNull(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(1)); assertThat(context.queryResult().topDocs().topDocs.scoreDocs[0], instanceOf(FieldDoc.class)); @@ -558,8 +552,7 @@ public void testIndexSortScrollOptimization() throws Exception { // search sort is a prefix of the index sort searchSortAndFormats.add(new SortAndFormats(new Sort(indexSort.getSort()[0]), new DocValueFormat[]{DocValueFormat.RAW})); for (SortAndFormats searchSortAndFormat : searchSortAndFormats) { - IndexSearcher contextSearcher = new IndexSearcher(reader); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); ScrollContext scrollContext = new ScrollContext(); scrollContext.lastEmittedDoc = null; @@ -570,7 +563,7 @@ public void testIndexSortScrollOptimization() throws Exception { context.setSize(10); context.sort(searchSortAndFormat); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertNull(context.queryResult().terminatedEarly()); assertThat(context.terminateAfter(), equalTo(0)); @@ -578,8 +571,8 @@ public void testIndexSortScrollOptimization() throws Exception { int sizeMinus1 = context.queryResult().topDocs().topDocs.scoreDocs.length - 1; FieldDoc lastDoc = (FieldDoc) context.queryResult().topDocs().topDocs.scoreDocs[sizeMinus1]; - contextSearcher = getAssertingEarlyTerminationSearcher(reader, 10); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + context.setSearcher(newEarlyTerminationContextSearcher(reader, 10)); + QueryPhase.executeInternal(context); assertNull(context.queryResult().terminatedEarly()); assertThat(context.queryResult().topDocs().topDocs.totalHits.value, equalTo((long) numDocs)); assertThat(context.terminateAfter(), equalTo(0)); @@ -617,8 +610,7 @@ public void testDisableTopScoreCollection() throws Exception { w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); Query q = new SpanNearQuery.Builder("title", true) .addClause(new SpanTermQuery(new Term("title", "foo"))) @@ -627,10 +619,9 @@ public void testDisableTopScoreCollection() throws Exception { context.parsedQuery(new ParsedQuery(q)); context.setSize(10); - TopDocsCollectorContext topDocsContext = - TopDocsCollectorContext.createTopDocsCollectorContext(context, reader, false); + TopDocsCollectorContext topDocsContext = TopDocsCollectorContext.createTopDocsCollectorContext(context, false); assertEquals(topDocsContext.create(null).scoreMode(), org.apache.lucene.search.ScoreMode.COMPLETE); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(5, context.queryResult().topDocs().topDocs.totalHits.value); assertEquals(context.queryResult().topDocs().topDocs.totalHits.relation, TotalHits.Relation.EQUAL_TO); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(5)); @@ -638,10 +629,9 @@ public void testDisableTopScoreCollection() throws Exception { context.sort(new SortAndFormats(new Sort(new SortField("other", SortField.Type.INT)), new DocValueFormat[] { DocValueFormat.RAW })); - topDocsContext = - TopDocsCollectorContext.createTopDocsCollectorContext(context, reader, false); + topDocsContext = TopDocsCollectorContext.createTopDocsCollectorContext(context, false); assertEquals(topDocsContext.create(null).scoreMode(), org.apache.lucene.search.ScoreMode.COMPLETE_NO_SCORES); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(5, context.queryResult().topDocs().topDocs.totalHits.value); assertThat(context.queryResult().topDocs().topDocs.scoreDocs.length, equalTo(5)); assertEquals(context.queryResult().topDocs().topDocs.totalHits.relation, TotalHits.Relation.EQUAL_TO); @@ -659,8 +649,6 @@ public void testNumericLongOrDateSortOptimization() throws Exception { MapperService mapperService = mock(MapperService.class); when(mapperService.fullName(fieldNameLong)).thenReturn(fieldTypeLong); when(mapperService.fullName(fieldNameDate)).thenReturn(fieldTypeDate); - TestSearchContext searchContext = spy(new TestSearchContext(null, indexShard)); - when(searchContext.mapperService()).thenReturn(mapperService); final int numDocs = 4000; Directory dir = newDirectory(); @@ -677,7 +665,10 @@ public void testNumericLongOrDateSortOptimization() throws Exception { } writer.close(); final IndexReader reader = DirectoryReader.open(dir); - IndexSearcher searcher = getAssertingSortOptimizedSearcher(reader, 0); + + TestSearchContext searchContext = + spy(new TestSearchContext(null, indexShard, newOptimizedContextSearcher(reader, 0))); + when(searchContext.mapperService()).thenReturn(mapperService); // 1. Test a sort on long field final SortField sortFieldLong = new SortField(fieldNameLong, SortField.Type.LONG); @@ -688,7 +679,7 @@ public void testNumericLongOrDateSortOptimization() throws Exception { searchContext.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); searchContext.setTask(new SearchTask(123L, "", "", "", null, Collections.emptyMap())); searchContext.setSize(10); - QueryPhase.execute(searchContext, searcher, checkCancelled -> {}); + QueryPhase.executeInternal(searchContext); assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false); // 2. Test a sort on long field + date field @@ -697,7 +688,7 @@ public void testNumericLongOrDateSortOptimization() throws Exception { final Sort longDateSort = new Sort(sortFieldLong, sortFieldDate); sortAndFormats = new SortAndFormats(longDateSort, new DocValueFormat[]{DocValueFormat.RAW, dateFormat}); searchContext.sort(sortAndFormats); - QueryPhase.execute(searchContext, searcher, checkCancelled -> {}); + QueryPhase.executeInternal(searchContext); assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true); // 3. Test a sort on date field @@ -705,14 +696,14 @@ public void testNumericLongOrDateSortOptimization() throws Exception { final Sort dateSort = new Sort(sortFieldDate); sortAndFormats = new SortAndFormats(dateSort, new DocValueFormat[]{dateFormat}); searchContext.sort(sortAndFormats); - QueryPhase.execute(searchContext, searcher, checkCancelled -> {}); + QueryPhase.executeInternal(searchContext); assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, false); // 4. Test a sort on date field + long field final Sort dateLongSort = new Sort(sortFieldDate, sortFieldLong); sortAndFormats = new SortAndFormats(dateLongSort, new DocValueFormat[]{dateFormat, DocValueFormat.RAW}); searchContext.sort(sortAndFormats); - QueryPhase.execute(searchContext, searcher, checkCancelled -> {}); + QueryPhase.executeInternal(searchContext); assertSortResults(searchContext.queryResult().topDocs().topDocs, (long) numDocs, true); reader.close(); dir.close(); @@ -784,10 +775,10 @@ public void testIndexHasNotDuplicateData() throws IOException { public void testMaxScoreQueryVisitor() { BitSetProducer producer = context -> new FixedBitSet(1); Query query = new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"); - assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + assertTrue(hasInfMaxScore(query)); query = new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.None, "nested"); - assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + assertFalse(hasInfMaxScore(query)); for (Occur occur : Occur.values()) { @@ -795,9 +786,9 @@ public void testMaxScoreQueryVisitor() { .add(new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"), occur) .build(); if (occur == Occur.MUST) { - assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + assertTrue(hasInfMaxScore(query)); } else { - assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + assertFalse(hasInfMaxScore(query)); } query = new BooleanQuery.Builder() @@ -806,9 +797,9 @@ public void testMaxScoreQueryVisitor() { .build(), occur) .build(); if (occur == Occur.MUST) { - assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + assertTrue(hasInfMaxScore(query)); } else { - assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + assertFalse(hasInfMaxScore(query)); } query = new BooleanQuery.Builder() @@ -816,7 +807,7 @@ public void testMaxScoreQueryVisitor() { .add(new ESToParentBlockJoinQuery(new MatchAllDocsQuery(), producer, ScoreMode.Avg, "nested"), occur) .build(), Occur.FILTER) .build(); - assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + assertFalse(hasInfMaxScore(query)); query = new BooleanQuery.Builder() .add(new BooleanQuery.Builder() @@ -825,34 +816,13 @@ public void testMaxScoreQueryVisitor() { .build(), occur) .build(); if (occur == Occur.MUST) { - assertTrue(TopDocsCollectorContext.hasInfMaxScore(query)); + assertTrue(hasInfMaxScore(query)); } else { - assertFalse(TopDocsCollectorContext.hasInfMaxScore(query)); + assertFalse(hasInfMaxScore(query)); } } } - // used to check that numeric long or date sort optimization was run - private static IndexSearcher getAssertingSortOptimizedSearcher(IndexReader reader, int queryType) { - return new IndexSearcher(reader) { - @Override - public void search(Query query, Collector results) throws IOException { - assertTrue(query instanceof BooleanQuery); - List clauses = ((BooleanQuery) query).clauses(); - assertTrue(clauses.size() == 2); - assertTrue(clauses.get(0).getOccur() == Occur.FILTER); - assertTrue(clauses.get(1).getOccur() == Occur.SHOULD); - if (queryType == 0) { - assertTrue (clauses.get(1).getQuery().getClass() == - LongPoint.newDistanceFeatureQuery("random_field", 1, 1, 1).getClass() - ); - } - if (queryType == 1) assertTrue(clauses.get(1).getQuery() instanceof DocValuesFieldExistsQuery); - super.search(query, results); - } - }; - } - // assert score docs are in order and their number is as expected private void assertSortResults(TopDocs topDocs, long expectedNumDocs, boolean isDoubleSort) { assertEquals(topDocs.totalHits.value, expectedNumDocs); @@ -886,8 +856,7 @@ public void testMinScore() throws Exception { w.close(); IndexReader reader = DirectoryReader.open(dir); - IndexSearcher contextSearcher = new IndexSearcher(reader); - TestSearchContext context = new TestSearchContext(null, indexShard); + TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader)); context.parsedQuery(new ParsedQuery( new BooleanQuery.Builder() .add(new TermQuery(new Term("foo", "bar")), Occur.MUST) @@ -899,7 +868,7 @@ public void testMinScore() throws Exception { context.setSize(1); context.trackTotalHitsUpTo(5); - QueryPhase.execute(context, contextSearcher, checkCancelled -> {}); + QueryPhase.executeInternal(context); assertEquals(10, context.queryResult().topDocs().topDocs.totalHits.value); reader.close(); @@ -907,12 +876,53 @@ public void testMinScore() throws Exception { } - private static IndexSearcher getAssertingEarlyTerminationSearcher(IndexReader reader, int size) { - return new IndexSearcher(reader) { + private static ContextIndexSearcher newContextSearcher(IndexReader reader) { + return new ContextIndexSearcher(reader, IndexSearcher.getDefaultSimilarity(), + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy()); + } + + private static ContextIndexSearcher newEarlyTerminationContextSearcher(IndexReader reader, int size) { + return new ContextIndexSearcher(reader, IndexSearcher.getDefaultSimilarity(), + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy()) { + @Override protected void search(List leaves, Weight weight, Collector collector) throws IOException { - final Collector in = new AssertingEarlyTerminationFilterCollector(collector, size); - super.search(leaves, weight, in); + throw new AssertionError(); + } + + @Override + public void searchLeaf(LeafReaderContext ctx, Weight weight, Collector collector) throws IOException { + collector = new AssertingEarlyTerminationFilterCollector(collector, size); + super.searchLeaf(ctx, weight, collector); + } + }; + } + + // used to check that numeric long or date sort optimization was run + private static ContextIndexSearcher newOptimizedContextSearcher(IndexReader reader, int queryType) { + return new ContextIndexSearcher(reader, IndexSearcher.getDefaultSimilarity(), + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy()) { + + @Override + public void search(Query query, Collector results) throws IOException { + throw new AssertionError(); + } + + @Override + public void searchLeaf(LeafReaderContext ctx, Weight weight, Collector collector) throws IOException { + final Query query = weight.getQuery(); + assertTrue(query instanceof BooleanQuery); + List clauses = ((BooleanQuery) query).clauses(); + assertTrue(clauses.size() == 2); + assertTrue(clauses.get(0).getOccur() == Occur.FILTER); + assertTrue(clauses.get(1).getOccur() == Occur.SHOULD); + if (queryType == 0) { + assertTrue (clauses.get(1).getQuery().getClass() == + LongPoint.newDistanceFeatureQuery("random_field", 1, 1, 1).getClass() + ); + } + if (queryType == 1) assertTrue(clauses.get(1).getQuery() instanceof DocValuesFieldExistsQuery); + super.searchLeaf(ctx, weight, collector); } }; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index 07b5f67db380d..522ab0da85200 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -102,11 +102,20 @@ public TestSearchContext(QueryShardContext queryShardContext) { } public TestSearchContext(QueryShardContext queryShardContext, IndexShard indexShard) { + this(queryShardContext, indexShard, null); + } + + public TestSearchContext(QueryShardContext queryShardContext, IndexShard indexShard, ContextIndexSearcher searcher) { this.bigArrays = null; this.indexService = null; this.fixedBitSetFilterCache = null; this.indexShard = indexShard; this.queryShardContext = queryShardContext; + this.searcher = searcher; + } + + public void setSearcher(ContextIndexSearcher searcher) { + this.searcher = searcher; } @Override