| Index: lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (working copy) |
| @@ -320,7 +320,7 @@ |
| |
| doc = -1; |
| this.needsFreq = (flags & DocsEnum.FLAG_FREQS) != 0; |
| - if (!indexHasFreq) { |
| + if (indexHasFreq == false || needsFreq == false) { |
| Arrays.fill(freqBuffer, 1); |
| } |
| accum = 0; |
| Index: lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (working copy) |
| @@ -242,7 +242,7 @@ |
| for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) { |
| Weight w = wIter.next(); |
| BooleanClause c = cIter.next(); |
| - if (w.scorer(context, context.reader().getLiveDocs()) == null) { |
| + if (w.scorer(context, context.reader().getLiveDocs(), true) == null) { |
| if (c.isRequired()) { |
| fail = true; |
| Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")"); |
| @@ -307,12 +307,12 @@ |
| /** Try to build a boolean scorer for this weight. Returns null if {@link BooleanScorer} |
| * cannot be used. */ |
| // pkg-private for forcing use of BooleanScorer in tests |
| - BooleanScorer booleanScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + BooleanScorer booleanScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| List<BulkScorer> optional = new ArrayList<BulkScorer>(); |
| Iterator<BooleanClause> cIter = clauses.iterator(); |
| for (Weight w : weights) { |
| BooleanClause c = cIter.next(); |
| - BulkScorer subScorer = w.bulkScorer(context, acceptDocs); |
| + BulkScorer subScorer = w.bulkScorer(context, acceptDocs, needsScores); |
| if (subScorer == null) { |
| if (c.isRequired()) { |
| return null; |
| @@ -342,8 +342,8 @@ |
| } |
| |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs); |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs, needsScores); |
| if (bulkScorer != null) { // BooleanScorer is applicable |
| // TODO: what is the right heuristic here? |
| final long costThreshold; |
| @@ -366,12 +366,11 @@ |
| return bulkScorer; |
| } |
| } |
| - return super.bulkScorer(context, acceptDocs); |
| + return super.bulkScorer(context, acceptDocs, needsScores); |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) |
| - throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| // initially the user provided value, |
| // but if minNrShouldMatch == optional.size(), |
| // we will optimize and move these to required, making this 0 |
| @@ -383,7 +382,7 @@ |
| Iterator<BooleanClause> cIter = clauses.iterator(); |
| for (Weight w : weights) { |
| BooleanClause c = cIter.next(); |
| - Scorer subScorer = w.scorer(context, acceptDocs); |
| + Scorer subScorer = w.scorer(context, acceptDocs, needsScores && c.isProhibited() == false); |
| if (subScorer == null) { |
| if (c.isRequired()) { |
| return null; |
| @@ -416,6 +415,11 @@ |
| return null; |
| } |
| |
| + // we don't need scores, so if we have required clauses, drop optional clauses completely |
| + if (!needsScores && minShouldMatch == 0 && required.size() > 0) { |
| + optional.clear(); |
| + } |
| + |
| // three cases: conjunction, disjunction, or mix |
| |
| // pure conjunction |
| Index: lucene/core/src/java/org/apache/lucene/search/CachingCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (working copy) |
| @@ -297,6 +297,11 @@ |
| @Override |
| public void collect(int doc) {} |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| + |
| }; |
| return create(other, cacheScores, maxRAMMB); |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/Collector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/Collector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/Collector.java (working copy) |
| @@ -73,4 +73,10 @@ |
| */ |
| LeafCollector getLeafCollector(LeafReaderContext context) throws IOException; |
| |
| + /** |
| + * Indicates if document scores are needed by this collector. |
| + * |
| + * @return {@code true} if scores are needed. |
| + */ |
| + boolean needsScores(); |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (working copy) |
| @@ -134,14 +134,14 @@ |
| } |
| |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| final DocIdSetIterator disi; |
| if (filter != null) { |
| assert query == null; |
| - return super.bulkScorer(context, acceptDocs); |
| + return super.bulkScorer(context, acceptDocs, needsScores); |
| } else { |
| assert query != null && innerWeight != null; |
| - BulkScorer bulkScorer = innerWeight.bulkScorer(context, acceptDocs); |
| + BulkScorer bulkScorer = innerWeight.bulkScorer(context, acceptDocs, false); |
| if (bulkScorer == null) { |
| return null; |
| } |
| @@ -150,7 +150,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| final DocIdSetIterator disi; |
| if (filter != null) { |
| assert query == null; |
| @@ -161,7 +161,7 @@ |
| disi = dis.iterator(); |
| } else { |
| assert query != null && innerWeight != null; |
| - disi = innerWeight.scorer(context, acceptDocs); |
| + disi = innerWeight.scorer(context, acceptDocs, false); |
| } |
| |
| if (disi == null) { |
| @@ -172,7 +172,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - final Scorer cs = scorer(context, context.reader().getLiveDocs()); |
| + final Scorer cs = scorer(context, context.reader().getLiveDocs(), true); |
| final boolean exists = (cs != null && cs.advance(doc) == doc); |
| |
| final ComplexExplanation result = new ComplexExplanation(); |
| Index: lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (working copy) |
| @@ -153,11 +153,11 @@ |
| |
| /** Create the scorer used to score our associated DisjunctionMaxQuery */ |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| List<Scorer> scorers = new ArrayList<>(); |
| for (Weight w : weights) { |
| // we will advance() subscorers |
| - Scorer subScorer = w.scorer(context, acceptDocs); |
| + Scorer subScorer = w.scorer(context, acceptDocs, needsScores); |
| if (subScorer != null) { |
| scorers.add(subScorer); |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (working copy) |
| @@ -55,11 +55,13 @@ |
| private int freq; |
| |
| private final Similarity.SimScorer docScorer; |
| + private final boolean needsScores; |
| |
| ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings, |
| - Similarity.SimScorer docScorer) throws IOException { |
| + Similarity.SimScorer docScorer, boolean needsScores) throws IOException { |
| super(weight); |
| this.docScorer = docScorer; |
| + this.needsScores = needsScores; |
| |
| chunkStates = new ChunkState[postings.length]; |
| |
| @@ -233,6 +235,9 @@ |
| final int posIndex = cs.pos - chunkStart; |
| if (posIndex >= 0 && gens[posIndex] == gen && counts[posIndex] == endMinus1) { |
| freq++; |
| + if (!needsScores) { |
| + return freq; // we determined there was a match. |
| + } |
| } |
| } |
| |
| Index: lucene/core/src/java/org/apache/lucene/search/FilterCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/FilterCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/FilterCollector.java (working copy) |
| @@ -44,5 +44,9 @@ |
| public String toString() { |
| return getClass().getSimpleName() + "(" + in + ")"; |
| } |
| - |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return in.needsScores(); |
| + } |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (working copy) |
| @@ -119,7 +119,7 @@ |
| |
| // return a filtering scorer |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| assert filter != null; |
| |
| DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs); |
| @@ -128,12 +128,12 @@ |
| return null; |
| } |
| |
| - return strategy.filteredScorer(context, weight, filterDocIdSet); |
| + return strategy.filteredScorer(context, weight, filterDocIdSet, needsScores); |
| } |
| |
| // return a filtering top scorer |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| assert filter != null; |
| |
| DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs); |
| @@ -142,7 +142,7 @@ |
| return null; |
| } |
| |
| - return strategy.filteredBulkScorer(context, weight, filterDocIdSet); |
| + return strategy.filteredBulkScorer(context, weight, filterDocIdSet, needsScores); |
| } |
| }; |
| } |
| @@ -465,7 +465,7 @@ |
| * @throws IOException if an {@link IOException} occurs |
| */ |
| public abstract Scorer filteredScorer(LeafReaderContext context, |
| - Weight weight, DocIdSet docIdSet) throws IOException; |
| + Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException; |
| |
| /** |
| * Returns a filtered {@link BulkScorer} based on this |
| @@ -480,8 +480,8 @@ |
| * @return a filtered top scorer |
| */ |
| public BulkScorer filteredBulkScorer(LeafReaderContext context, |
| - Weight weight, DocIdSet docIdSet) throws IOException { |
| - Scorer scorer = filteredScorer(context, weight, docIdSet); |
| + Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException { |
| + Scorer scorer = filteredScorer(context, weight, docIdSet, needsScores); |
| if (scorer == null) { |
| return null; |
| } |
| @@ -502,7 +502,7 @@ |
| public static class RandomAccessFilterStrategy extends FilterStrategy { |
| |
| @Override |
| - public Scorer filteredScorer(LeafReaderContext context, Weight weight, DocIdSet docIdSet) throws IOException { |
| + public Scorer filteredScorer(LeafReaderContext context, Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException { |
| final DocIdSetIterator filterIter = docIdSet.iterator(); |
| if (filterIter == null) { |
| // this means the filter does not accept any documents. |
| @@ -514,11 +514,11 @@ |
| final boolean useRandomAccess = filterAcceptDocs != null && useRandomAccess(filterAcceptDocs, filterIter.cost()); |
| if (useRandomAccess) { |
| // if we are using random access, we return the inner scorer, just with other acceptDocs |
| - return weight.scorer(context, filterAcceptDocs); |
| + return weight.scorer(context, filterAcceptDocs, needsScores); |
| } else { |
| // we are gonna advance() this scorer, so we set inorder=true/toplevel=false |
| // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice |
| - final Scorer scorer = weight.scorer(context, null); |
| + final Scorer scorer = weight.scorer(context, null, needsScores); |
| return (scorer == null) ? null : new LeapFrogScorer(weight, filterIter, scorer, scorer); |
| } |
| } |
| @@ -551,7 +551,7 @@ |
| |
| @Override |
| public Scorer filteredScorer(LeafReaderContext context, |
| - Weight weight, DocIdSet docIdSet) throws IOException { |
| + Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException { |
| final DocIdSetIterator filterIter = docIdSet.iterator(); |
| if (filterIter == null) { |
| // this means the filter does not accept any documents. |
| @@ -558,7 +558,7 @@ |
| return null; |
| } |
| // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice |
| - final Scorer scorer = weight.scorer(context, null); |
| + final Scorer scorer = weight.scorer(context, null, needsScores); |
| if (scorer == null) { |
| return null; |
| } |
| @@ -587,15 +587,14 @@ |
| private static final class QueryFirstFilterStrategy extends FilterStrategy { |
| @Override |
| public Scorer filteredScorer(final LeafReaderContext context, |
| - Weight weight, |
| - DocIdSet docIdSet) throws IOException { |
| + Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException { |
| Bits filterAcceptDocs = docIdSet.bits(); |
| if (filterAcceptDocs == null) { |
| // Filter does not provide random-access Bits; we |
| // must fallback to leapfrog: |
| - return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet); |
| + return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet, needsScores); |
| } |
| - final Scorer scorer = weight.scorer(context, null); |
| + final Scorer scorer = weight.scorer(context, null, needsScores); |
| return scorer == null ? null : new QueryFirstScorer(weight, |
| filterAcceptDocs, scorer); |
| } |
| @@ -602,15 +601,14 @@ |
| |
| @Override |
| public BulkScorer filteredBulkScorer(final LeafReaderContext context, |
| - Weight weight, |
| - DocIdSet docIdSet) throws IOException { |
| + Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException { |
| Bits filterAcceptDocs = docIdSet.bits(); |
| if (filterAcceptDocs == null) { |
| // Filter does not provide random-access Bits; we |
| // must fallback to leapfrog: |
| - return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, docIdSet); |
| + return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, docIdSet, needsScores); |
| } |
| - final Scorer scorer = weight.scorer(context, null); |
| + final Scorer scorer = weight.scorer(context, null, needsScores); |
| return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs); |
| } |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (working copy) |
| @@ -586,7 +586,7 @@ |
| // continue with the following leaf |
| continue; |
| } |
| - BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs()); |
| + BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs(), collector.needsScores()); |
| if (scorer != null) { |
| try { |
| scorer.score(leafCollector); |
| Index: lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (working copy) |
| @@ -114,7 +114,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight); |
| } |
| |
| Index: lucene/core/src/java/org/apache/lucene/search/MultiCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (working copy) |
| @@ -93,6 +93,16 @@ |
| } |
| |
| @Override |
| + public boolean needsScores() { |
| + for (Collector collector : collectors) { |
| + if (collector.needsScores()) { |
| + return true; |
| + } |
| + } |
| + return false; |
| + } |
| + |
| + @Override |
| public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { |
| final LeafCollector[] leafCollectors = new LeafCollector[collectors.length]; |
| for (int i = 0; i < collectors.length; ++i) { |
| Index: lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (working copy) |
| @@ -179,7 +179,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| assert !termArrays.isEmpty(); |
| final LeafReader reader = context.reader(); |
| final Bits liveDocs = acceptDocs; |
| @@ -249,15 +249,15 @@ |
| } |
| |
| if (slop == 0) { |
| - return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context)); |
| + return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context), needsScores); |
| } else { |
| - return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context)); |
| + return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context), needsScores); |
| } |
| } |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - Scorer scorer = scorer(context, context.reader().getLiveDocs()); |
| + Scorer scorer = scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| int newDoc = scorer.advance(doc); |
| if (newDoc == doc) { |
| Index: lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (working copy) |
| @@ -245,7 +245,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| assert !terms.isEmpty(); |
| final LeafReader reader = context.reader(); |
| final Bits liveDocs = acceptDocs; |
| @@ -285,9 +285,9 @@ |
| } |
| |
| if (slop == 0) { // optimize exact case |
| - return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context)); |
| + return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context), needsScores); |
| } else { |
| - return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context)); |
| + return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context), needsScores); |
| } |
| } |
| |
| @@ -298,7 +298,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - Scorer scorer = scorer(context, context.reader().getLiveDocs()); |
| + Scorer scorer = scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| int newDoc = scorer.advance(doc); |
| if (newDoc == doc) { |
| Index: lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java (working copy) |
| @@ -82,7 +82,7 @@ |
| if (readerContext != null) { |
| // We advanced to another segment: |
| docBase = readerContext.docBase; |
| - scorer = weight.scorer(readerContext, null); |
| + scorer = weight.scorer(readerContext, null, true); |
| } |
| |
| if(scorer != null) { |
| Index: lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java (working copy) |
| @@ -56,7 +56,7 @@ |
| return new DocIdSet() { |
| @Override |
| public DocIdSetIterator iterator() throws IOException { |
| - return weight.scorer(privateContext, acceptDocs); |
| + return weight.scorer(privateContext, acceptDocs, false); |
| } |
| |
| @Override |
| Index: lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (working copy) |
| @@ -50,11 +50,13 @@ |
| |
| private int numMatches; |
| private final long cost; |
| + final boolean needsScores; |
| |
| SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings, |
| - int slop, Similarity.SimScorer docScorer) { |
| + int slop, Similarity.SimScorer docScorer, boolean needsScores) { |
| super(weight); |
| this.docScorer = docScorer; |
| + this.needsScores = needsScores; |
| this.slop = slop; |
| this.numPostings = postings==null ? 0 : postings.length; |
| pq = new PhraseQueue(postings.length); |
| @@ -114,6 +116,9 @@ |
| if (matchLength <= slop) { |
| freq += docScorer.computeSlopFactor(matchLength); // score match |
| numMatches++; |
| + if (!needsScores) { |
| + return freq; |
| + } |
| } |
| pq.add(pp); |
| pp = pq.pop(); |
| Index: lucene/core/src/java/org/apache/lucene/search/TermQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/TermQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/TermQuery.java (working copy) |
| @@ -75,13 +75,13 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termStates.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context); |
| final TermsEnum termsEnum = getTermsEnum(context); |
| if (termsEnum == null) { |
| return null; |
| } |
| - DocsEnum docs = termsEnum.docs(acceptDocs, null); |
| + DocsEnum docs = termsEnum.docs(acceptDocs, null, needsScores ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE); |
| assert docs != null; |
| return new TermScorer(this, docs, similarity.simScorer(stats, context)); |
| } |
| @@ -110,7 +110,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - Scorer scorer = scorer(context, context.reader().getLiveDocs()); |
| + Scorer scorer = scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| int newDoc = scorer.advance(doc); |
| if (newDoc == doc) { |
| Index: lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (working copy) |
| @@ -157,6 +157,11 @@ |
| }; |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return collector.needsScores(); |
| + } |
| + |
| /** |
| * This is so the same timer can be used with a multi-phase search process such as grouping. |
| * We don't want to create a new TimeLimitingCollector for each phase because that would |
| Index: lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (working copy) |
| @@ -129,8 +129,8 @@ |
| |
| final FieldValueHitQueue<Entry> queue; |
| |
| - public NonScoringCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) { |
| - super(queue, numHits, fillFields); |
| + public NonScoringCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) { |
| + super(queue, numHits, fillFields, sort.needsScores()); |
| this.queue = queue; |
| } |
| |
| @@ -216,8 +216,8 @@ |
| |
| final FieldValueHitQueue<Entry> queue; |
| |
| - public ScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) { |
| - super(queue, numHits, fillFields); |
| + public ScoringNoMaxScoreCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) { |
| + super(queue, numHits, fillFields, true); |
| this.queue = queue; |
| } |
| |
| @@ -315,8 +315,8 @@ |
| |
| final FieldValueHitQueue<Entry> queue; |
| |
| - public ScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) { |
| - super(queue, numHits, fillFields); |
| + public ScoringMaxScoreCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) { |
| + super(queue, numHits, fillFields, true); |
| this.queue = queue; |
| maxScore = Float.MIN_NORMAL; // otherwise we would keep NaN |
| } |
| @@ -414,9 +414,9 @@ |
| final boolean trackMaxScore; |
| final FieldDoc after; |
| |
| - public PagingFieldCollector(FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields, |
| + public PagingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields, |
| boolean trackDocScores, boolean trackMaxScore) { |
| - super(queue, numHits, fillFields); |
| + super(queue, numHits, fillFields, trackDocScores || trackMaxScore || sort.needsScores()); |
| this.queue = queue; |
| this.trackDocScores = trackDocScores; |
| this.trackMaxScore = trackMaxScore; |
| @@ -520,6 +520,7 @@ |
| FieldValueHitQueue.Entry bottom = null; |
| boolean queueFull; |
| int docBase; |
| + final boolean needsScores; |
| |
| // Declaring the constructor private prevents extending this class by anyone |
| // else. Note that the class cannot be final since it's extended by the |
| @@ -526,12 +527,18 @@ |
| // internal versions. If someone will define a constructor with any other |
| // visibility, then anyone will be able to extend the class, which is not what |
| // we want. |
| - private TopFieldCollector(PriorityQueue<Entry> pq, int numHits, boolean fillFields) { |
| + private TopFieldCollector(PriorityQueue<Entry> pq, int numHits, boolean fillFields, boolean needsScores) { |
| super(pq); |
| + this.needsScores = needsScores; |
| this.numHits = numHits; |
| this.fillFields = fillFields; |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return needsScores; |
| + } |
| + |
| /** |
| * Creates a new {@link TopFieldCollector} from the given |
| * arguments. |
| @@ -622,11 +629,11 @@ |
| |
| if (after == null) { |
| if (trackMaxScore) { |
| - return new ScoringMaxScoreCollector(queue, numHits, fillFields); |
| + return new ScoringMaxScoreCollector(sort, queue, numHits, fillFields); |
| } else if (trackDocScores) { |
| - return new ScoringNoMaxScoreCollector(queue, numHits, fillFields); |
| + return new ScoringNoMaxScoreCollector(sort, queue, numHits, fillFields); |
| } else { |
| - return new NonScoringCollector(queue, numHits, fillFields); |
| + return new NonScoringCollector(sort, queue, numHits, fillFields); |
| } |
| } else { |
| if (after.fields == null) { |
| @@ -637,7 +644,7 @@ |
| throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length); |
| } |
| |
| - return new PagingFieldCollector(queue, after, numHits, fillFields, trackDocScores, trackMaxScore); |
| + return new PagingFieldCollector(sort, queue, after, numHits, fillFields, trackDocScores, trackMaxScore); |
| } |
| } |
| |
| Index: lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (working copy) |
| @@ -207,4 +207,9 @@ |
| |
| return new TopDocs(totalHits, results, maxScore); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (working copy) |
| @@ -34,4 +34,9 @@ |
| public void collect(int doc) { |
| totalHits++; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| } |
| Index: lucene/core/src/java/org/apache/lucene/search/Weight.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/Weight.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/Weight.java (working copy) |
| @@ -34,7 +34,7 @@ |
| * {@link org.apache.lucene.index.LeafReader} dependent state should reside in the {@link Scorer}. |
| * <p> |
| * Since {@link Weight} creates {@link Scorer} instances for a given |
| - * {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)}) |
| + * {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits, boolean)}) |
| * callers must maintain the relationship between the searcher's top-level |
| * {@link IndexReaderContext} and the context used to create a {@link Scorer}. |
| * <p> |
| @@ -49,7 +49,7 @@ |
| * <li>The query normalization factor is passed to {@link #normalize(float, float)}. At |
| * this point the weighting is complete. |
| * <li>A <code>Scorer</code> is constructed by |
| - * {@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)}. |
| + * {@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits, boolean)}. |
| * </ol> |
| * |
| * @since 2.9 |
| @@ -87,11 +87,13 @@ |
| * @param acceptDocs |
| * Bits that represent the allowable docs to match (typically deleted docs |
| * but possibly filtering other documents) |
| + * @param needsScores |
| + * True if document scores ({@link Scorer#score}) or match frequencies ({@link Scorer#freq}) are needed. |
| * |
| * @return a {@link Scorer} which scores documents in/out-of order. |
| * @throws IOException if there is a low-level I/O error |
| */ |
| - public abstract Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException; |
| + public abstract Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException; |
| |
| /** |
| * Optional method, to return a {@link BulkScorer} to |
| @@ -106,14 +108,16 @@ |
| * @param acceptDocs |
| * Bits that represent the allowable docs to match (typically deleted docs |
| * but possibly filtering other documents) |
| + * @param needsScores |
| + * True if document scores are needed. |
| * |
| * @return a {@link BulkScorer} which scores documents and |
| * passes them to a collector. |
| * @throws IOException if there is a low-level I/O error |
| */ |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| |
| - Scorer scorer = scorer(context, acceptDocs); |
| + Scorer scorer = scorer(context, acceptDocs, needsScores); |
| if (scorer == null) { |
| // No docs match |
| return null; |
| Index: lucene/core/src/java/org/apache/lucene/search/package.html |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/package.html (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/package.html (working copy) |
| @@ -436,15 +436,13 @@ |
| that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will just defer to the Similarity's implementation: |
| {@link org.apache.lucene.search.similarities.Similarity.SimWeight#normalize SimWeight#normalize(float,float)}.</li> |
| <li> |
| - {@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits) |
| - scorer(LeafReaderContext context, Bits acceptDocs)} — |
| + {@link org.apache.lucene.search.Weight#scorer scorer()} — |
| Construct a new {@link org.apache.lucene.search.Scorer Scorer} for this Weight. See <a href="#scorerClass">The Scorer Class</a> |
| below for help defining a Scorer. As the name implies, the Scorer is responsible for doing the actual scoring of documents |
| given the Query. |
| </li> |
| <li> |
| - {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.LeafReaderContext, org.apache.lucene.util.Bits) |
| - scorer(LeafReaderContext context, Bits acceptDocs)} — |
| + {@link org.apache.lucene.search.Weight#bulkScorer bulkScorer()} — |
| Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a> |
| below for help defining a BulkScorer. This is an optional method, and most queries do not implement it. |
| </li> |
| Index: lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (working copy) |
| @@ -148,7 +148,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this, |
| similarity, similarity.simScorer(stats, context)); |
| } |
| @@ -155,7 +155,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs()); |
| + PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| int newDoc = scorer.advance(doc); |
| if (newDoc == doc) { |
| Index: lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (working copy) |
| @@ -79,7 +79,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts), |
| this, similarity.simScorer(stats, context)); |
| } |
| @@ -176,7 +176,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs()); |
| + PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| int newDoc = scorer.advance(doc); |
| if (newDoc == doc) { |
| Index: lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java |
| =================================================================== |
| --- lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (revision 1657441) |
| +++ lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (working copy) |
| @@ -81,7 +81,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| if (stats == null) { |
| return null; |
| } else { |
| @@ -91,7 +91,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs()); |
| + SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| int newDoc = scorer.advance(doc); |
| if (newDoc == doc) { |
| Index: lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (working copy) |
| @@ -437,6 +437,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| docBase = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| } |
| |
| /** test that when freqs are omitted, that totalTermFreq and sumTotalTermFreq are -1 */ |
| Index: lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java (working copy) |
| @@ -53,6 +53,10 @@ |
| throw new UnsupportedOperationException(UNSUPPORTED_MSG); |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + throw new UnsupportedOperationException(UNSUPPORTED_MSG); |
| + } |
| } |
| |
| static final class JustCompileDocIdSet extends DocIdSet { |
| @@ -253,7 +257,11 @@ |
| public TopDocs topDocs( int start, int end ) { |
| throw new UnsupportedOperationException( UNSUPPORTED_MSG ); |
| } |
| - |
| + |
| + @Override |
| + public boolean needsScores() { |
| + throw new UnsupportedOperationException( UNSUPPORTED_MSG ); |
| + } |
| } |
| |
| static final class JustCompileWeight extends Weight { |
| @@ -279,7 +287,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) { |
| throw new UnsupportedOperationException(UNSUPPORTED_MSG); |
| } |
| |
| Index: lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java (working copy) |
| @@ -46,6 +46,10 @@ |
| setScorerCalled = true; |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| |
| @Test |
| Index: lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java (working copy) |
| @@ -707,7 +707,7 @@ |
| private void assertScore(final float expected, Query query) throws Exception { |
| // test in-order |
| Weight weight = searcher.createNormalizedWeight(query); |
| - Scorer scorer = weight.scorer(reader.leaves().get(0), null); |
| + Scorer scorer = weight.scorer(reader.leaves().get(0), null, true); |
| assertTrue(scorer.docID() == -1 || scorer.docID() == DocIdSetIterator.NO_MORE_DOCS); |
| assertEquals(0, scorer.nextDoc()); |
| assertEquals(expected, scorer.score(), 0.0001f); |
| @@ -714,7 +714,7 @@ |
| |
| // test bulk scorer |
| final AtomicBoolean seen = new AtomicBoolean(false); |
| - BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), null); |
| + BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), null, true); |
| assertNotNull(bulkScorer); |
| bulkScorer.score(new LeafCollector() { |
| Scorer scorer; |
| Index: lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java (working copy) |
| @@ -189,7 +189,7 @@ |
| Weight w = s.createNormalizedWeight(bq); |
| |
| assertEquals(1, s.getIndexReader().leaves().size()); |
| - BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null); |
| + BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null, true); |
| |
| final FixedBitSet hits = new FixedBitSet(docCount); |
| final AtomicInteger end = new AtomicInteger(); |
| @@ -200,6 +200,11 @@ |
| assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue()); |
| hits.set(doc); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }; |
| |
| while (end.intValue() < docCount) { |
| Index: lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java (working copy) |
| @@ -235,7 +235,7 @@ |
| |
| Weight weight = s.createNormalizedWeight(q); |
| |
| - Scorer scorer = weight.scorer(s.leafContexts.get(0), null); |
| + Scorer scorer = weight.scorer(s.leafContexts.get(0), null, true); |
| |
| // First pass: just use .nextDoc() to gather all hits |
| final List<ScoreDoc> hits = new ArrayList<>(); |
| @@ -252,7 +252,7 @@ |
| for(int iter2=0;iter2<10;iter2++) { |
| |
| weight = s.createNormalizedWeight(q); |
| - scorer = weight.scorer(s.leafContexts.get(0), null); |
| + scorer = weight.scorer(s.leafContexts.get(0), null, true); |
| |
| if (VERBOSE) { |
| System.out.println(" iter2=" + iter2); |
| Index: lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java (working copy) |
| @@ -234,6 +234,11 @@ |
| public List<String> getSummaries() { |
| return summaries; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| |
| @Override |
| public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { |
| @@ -282,8 +287,8 @@ |
| public Weight createWeight(IndexSearcher searcher) throws IOException { |
| return new BooleanWeight(searcher, false) { |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - Scorer scorer = scorer(context, acceptDocs); |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + Scorer scorer = scorer(context, acceptDocs, needsScores); |
| if (scorer == null) { |
| return null; |
| } |
| Index: lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java (working copy) |
| @@ -92,12 +92,12 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) { |
| throw new UnsupportedOperationException(); |
| } |
| |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) { |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) { |
| return new BulkScorer() { |
| @Override |
| public int score(LeafCollector collector, int min, int max) throws IOException { |
| Index: lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java (working copy) |
| @@ -56,6 +56,11 @@ |
| |
| @Override |
| public void collect(int doc) throws IOException {} |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| |
| } |
| |
| @@ -79,6 +84,11 @@ |
| assertEquals(prevDocID + 1, doc); |
| prevDocID = doc; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }); |
| } |
| } |
| Index: lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java (working copy) |
| @@ -67,6 +67,11 @@ |
| assertEquals("Score differs from expected", expectedScore, this.scorer.score(), 0); |
| count[0]++; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| assertEquals("invalid number of results", 1, count[0]); |
| } |
| Index: lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java (working copy) |
| @@ -180,7 +180,7 @@ |
| assertTrue(s.getTopReaderContext() instanceof LeafReaderContext); |
| final Weight dw = s.createNormalizedWeight(dq); |
| LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext(); |
| - final Scorer ds = dw.scorer(context, context.reader().getLiveDocs()); |
| + final Scorer ds = dw.scorer(context, context.reader().getLiveDocs(), true); |
| final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS; |
| if (skipOk) { |
| fail("firsttime skipTo found a match? ... " |
| @@ -196,7 +196,7 @@ |
| QueryUtils.check(random(), dq, s); |
| final Weight dw = s.createNormalizedWeight(dq); |
| LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext(); |
| - final Scorer ds = dw.scorer(context, context.reader().getLiveDocs()); |
| + final Scorer ds = dw.scorer(context, context.reader().getLiveDocs(), true); |
| assertTrue("firsttime skipTo found no match", |
| ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS); |
| assertEquals("found wrong docid", "d4", r.document(ds.docID()).get("id")); |
| Index: lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java (working copy) |
| @@ -74,6 +74,10 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| base = context.docBase; |
| } |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| |
| float lastScore = 0.0f; |
| Index: lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java (working copy) |
| @@ -80,7 +80,11 @@ |
| collectionTerminated = false; |
| } |
| } |
| - |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }; |
| |
| searcher.search(new MatchAllDocsQuery(), collector); |
| Index: lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java (working copy) |
| @@ -131,11 +131,11 @@ |
| case DOC_VALUES: |
| return new SlowMinShouldMatchScorer(weight, reader, searcher); |
| case SCORER: |
| - return weight.scorer(reader.getContext(), null); |
| + return weight.scorer(reader.getContext(), null, true); |
| case BULK_SCORER: |
| - final BulkScorer bulkScorer = weight.booleanScorer(reader.getContext(), null); |
| + final BulkScorer bulkScorer = weight.booleanScorer(reader.getContext(), null, true); |
| if (bulkScorer == null) { |
| - if (weight.scorer(reader.getContext(), null) != null) { |
| + if (weight.scorer(reader.getContext(), null, true) != null) { |
| throw new AssertionError("BooleanScorer should be applicable for this query"); |
| } |
| return null; |
| Index: lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java (working copy) |
| @@ -241,6 +241,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| base = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| |
| // |
| Index: lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java (revision 0) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java (working copy) |
| @@ -0,0 +1,185 @@ |
| +package org.apache.lucene.search; |
| + |
| +/* |
| + * Licensed to the Apache Software Foundation (ASF) under one or more |
| + * contributor license agreements. See the NOTICE file distributed with |
| + * this work for additional information regarding copyright ownership. |
| + * The ASF 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. |
| + */ |
| + |
| +import java.io.IOException; |
| +import java.util.Set; |
| + |
| +import org.apache.lucene.document.Document; |
| +import org.apache.lucene.document.Field; |
| +import org.apache.lucene.document.TextField; |
| +import org.apache.lucene.index.IndexReader; |
| +import org.apache.lucene.index.LeafReaderContext; |
| +import org.apache.lucene.index.RandomIndexWriter; |
| +import org.apache.lucene.index.Term; |
| +import org.apache.lucene.store.Directory; |
| +import org.apache.lucene.util.Bits; |
| +import org.apache.lucene.util.IOUtils; |
| +import org.apache.lucene.util.LuceneTestCase; |
| + |
| +public class TestNeedsScores extends LuceneTestCase { |
| + Directory dir; |
| + IndexReader reader; |
| + IndexSearcher searcher; |
| + |
| + @Override |
| + public void setUp() throws Exception { |
| + super.setUp(); |
| + dir = newDirectory(); |
| + RandomIndexWriter iw = new RandomIndexWriter(random(), dir); |
| + for (int i = 0; i < 5; i++) { |
| + Document doc = new Document(); |
| + doc.add(new TextField("field", "this is document " + i, Field.Store.NO)); |
| + iw.addDocument(doc); |
| + } |
| + reader = iw.getReader(); |
| + searcher = newSearcher(reader); |
| + iw.close(); |
| + } |
| + |
| + @Override |
| + public void tearDown() throws Exception { |
| + IOUtils.close(reader, dir); |
| + super.tearDown(); |
| + } |
| + |
| + /** prohibited clauses in booleanquery don't need scoring */ |
| + public void testProhibitedClause() throws Exception { |
| + Query required = new TermQuery(new Term("field", "this")); |
| + Query prohibited = new TermQuery(new Term("field", "3")); |
| + BooleanQuery bq = new BooleanQuery(); |
| + bq.add(new AssertNeedsScores(required, true), BooleanClause.Occur.MUST); |
| + bq.add(new AssertNeedsScores(prohibited, false), BooleanClause.Occur.MUST_NOT); |
| + assertEquals(4, searcher.search(bq, 5).totalHits); // we exclude 3 |
| + } |
| + |
| + /** nested inside constant score query */ |
| + public void testConstantScoreQuery() throws Exception { |
| + Query term = new TermQuery(new Term("field", "this")); |
| + Query constantScore = new ConstantScoreQuery(new AssertNeedsScores(term, false)); |
| + assertEquals(5, searcher.search(constantScore, 5).totalHits); |
| + } |
| + |
| + /** when converted to a filter */ |
| + public void testQueryWrapperFilter() throws Exception { |
| + Query query = new MatchAllDocsQuery(); |
| + Query term = new TermQuery(new Term("field", "this")); |
| + Filter filter = new QueryWrapperFilter(new AssertNeedsScores(term, false)); |
| + assertEquals(5, searcher.search(query, filter, 5).totalHits); |
| + } |
| + |
| + /** when not sorting by score */ |
| + public void testSortByField() throws Exception { |
| + Query query = new AssertNeedsScores(new MatchAllDocsQuery(), false); |
| + assertEquals(5, searcher.search(query, 5, Sort.INDEXORDER).totalHits); |
| + } |
| + |
| + /** when sorting by score */ |
| + public void testSortByScore() throws Exception { |
| + Query query = new AssertNeedsScores(new MatchAllDocsQuery(), true); |
| + assertEquals(5, searcher.search(query, 5, Sort.RELEVANCE).totalHits); |
| + } |
| + |
| + /** |
| + * Wraps a query, checking that the needsScores param |
| + * passed to Weight.scorer is the expected value. |
| + */ |
| + static class AssertNeedsScores extends Query { |
| + final Query in; |
| + final boolean value; |
| + |
| + AssertNeedsScores(Query in, boolean value) { |
| + this.in = in; |
| + this.value = value; |
| + } |
| + |
| + @Override |
| + public Weight createWeight(IndexSearcher searcher) throws IOException { |
| + final Weight w = in.createWeight(searcher); |
| + return new Weight() { |
| + @Override |
| + public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| + return w.explain(context, doc); |
| + } |
| + |
| + @Override |
| + public Query getQuery() { |
| + return AssertNeedsScores.this; |
| + } |
| + |
| + @Override |
| + public float getValueForNormalization() throws IOException { |
| + return w.getValueForNormalization(); |
| + } |
| + |
| + @Override |
| + public void normalize(float norm, float topLevelBoost) { |
| + w.normalize(norm, topLevelBoost); |
| + } |
| + |
| + @Override |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + assertEquals("query=" + in, value, needsScores); |
| + return w.scorer(context, acceptDocs, needsScores); |
| + } |
| + }; |
| + } |
| + |
| + @Override |
| + public Query rewrite(IndexReader reader) throws IOException { |
| + Query in2 = in.rewrite(reader); |
| + if (in2 == in) { |
| + return this; |
| + } else { |
| + return new AssertNeedsScores(in2, value); |
| + } |
| + } |
| + |
| + @Override |
| + public void extractTerms(Set<Term> terms) { |
| + in.extractTerms(terms); |
| + } |
| + |
| + @Override |
| + public int hashCode() { |
| + final int prime = 31; |
| + int result = super.hashCode(); |
| + result = prime * result + ((in == null) ? 0 : in.hashCode()); |
| + result = prime * result + (value ? 1231 : 1237); |
| + return result; |
| + } |
| + |
| + @Override |
| + public boolean equals(Object obj) { |
| + if (this == obj) return true; |
| + if (!super.equals(obj)) return false; |
| + if (getClass() != obj.getClass()) return false; |
| + AssertNeedsScores other = (AssertNeedsScores) obj; |
| + if (in == null) { |
| + if (other.in != null) return false; |
| + } else if (!in.equals(other.in)) return false; |
| + if (value != other.value) return false; |
| + return true; |
| + } |
| + |
| + @Override |
| + public String toString(String field) { |
| + return "asserting(" + in.toString(field) + ")"; |
| + } |
| + } |
| +} |
| |
| Property changes on: lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java |
| ___________________________________________________________________ |
| Added: svn:eol-style |
| ## -0,0 +1 ## |
| +native |
| \ No newline at end of property |
| Index: lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java (working copy) |
| @@ -443,7 +443,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(final LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(final LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| |
| return new Scorer(null) { |
| int docID = -1; |
| Index: lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java (working copy) |
| @@ -90,6 +90,11 @@ |
| @Override public void setScorer(Scorer scorer) { |
| this.scorer = new ScoreCachingWrappingScorer(scorer); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| |
| } |
| |
| Index: lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java (working copy) |
| @@ -115,6 +115,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| docBase = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| } |
| |
| |
| Index: lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java (working copy) |
| @@ -86,7 +86,8 @@ |
| public final void collect(int doc) throws IOException { |
| assertEquals(1.0f, scorer.score(), 0); |
| } |
| - public boolean acceptsDocsOutOfOrder() { |
| + @Override |
| + public boolean needsScores() { |
| return true; |
| } |
| }); |
| @@ -111,6 +112,10 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| base = context.docBase; |
| } |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| |
| PhraseQuery pq = new PhraseQuery(); |
| @@ -129,6 +134,10 @@ |
| //System.out.println("Doc=" + doc + " score=" + score); |
| assertEquals(1.0f, scorer.score(), 0); |
| } |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| |
| pq.setSlop(2); |
| @@ -144,6 +153,10 @@ |
| //System.out.println("Doc=" + doc + " score=" + score); |
| assertEquals(2.0f, scorer.score(), 0); |
| } |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| |
| reader.close(); |
| Index: lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.java (working copy) |
| @@ -191,6 +191,11 @@ |
| totalHits++; |
| max = Math.max(max, scorer.freq()); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| |
| /** checks that no scores or freqs are infinite */ |
| @@ -208,6 +213,11 @@ |
| assertFalse(Float.isInfinite(scorer.freq())); |
| assertFalse(Float.isInfinite(scorer.score())); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| QueryUtils.check(random(), pq, searcher); |
| } |
| Index: lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java (working copy) |
| @@ -78,7 +78,7 @@ |
| Weight weight = indexSearcher.createNormalizedWeight(termQuery); |
| assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext); |
| LeafReaderContext context = (LeafReaderContext)indexSearcher.getTopReaderContext(); |
| - BulkScorer ts = weight.bulkScorer(context, context.reader().getLiveDocs()); |
| + BulkScorer ts = weight.bulkScorer(context, context.reader().getLiveDocs(), true); |
| // we have 2 documents with the term all in them, one document for all the |
| // other values |
| final List<TestHit> docs = new ArrayList<>(); |
| @@ -107,6 +107,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| base = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| assertTrue("docs Size: " + docs.size() + " is not: " + 2, docs.size() == 2); |
| TestHit doc0 = docs.get(0); |
| @@ -135,7 +140,7 @@ |
| Weight weight = indexSearcher.createNormalizedWeight(termQuery); |
| assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext); |
| LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext(); |
| - Scorer ts = weight.scorer(context, context.reader().getLiveDocs()); |
| + Scorer ts = weight.scorer(context, context.reader().getLiveDocs(), true); |
| assertTrue("next did not return a doc", |
| ts.nextDoc() != DocIdSetIterator.NO_MORE_DOCS); |
| assertTrue("score is not correct", ts.score() == 1.6931472f); |
| @@ -154,7 +159,7 @@ |
| Weight weight = indexSearcher.createNormalizedWeight(termQuery); |
| assertTrue(indexSearcher.getTopReaderContext() instanceof LeafReaderContext); |
| LeafReaderContext context = (LeafReaderContext) indexSearcher.getTopReaderContext(); |
| - Scorer ts = weight.scorer(context, context.reader().getLiveDocs()); |
| + Scorer ts = weight.scorer(context, context.reader().getLiveDocs(), true); |
| assertTrue("Didn't skip", ts.advance(3) != DocIdSetIterator.NO_MORE_DOCS); |
| // The next doc should be doc 5 |
| assertTrue("doc should be number 5", ts.docID() == 5); |
| Index: lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestTimeLimitingCollector.java (working copy) |
| @@ -356,6 +356,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| docBase = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| |
| } |
| |
| Index: lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.java (working copy) |
| @@ -70,6 +70,11 @@ |
| } |
| }; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| |
| } |
| |
| Index: lucene/core/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java (working copy) |
| @@ -183,7 +183,7 @@ |
| Weight w = searcher.createNormalizedWeight(q); |
| IndexReaderContext topReaderContext = searcher.getTopReaderContext(); |
| LeafReaderContext leave = topReaderContext.leaves().get(0); |
| - Scorer s = w.scorer(leave, leave.reader().getLiveDocs()); |
| + Scorer s = w.scorer(leave, leave.reader().getLiveDocs(), true); |
| assertEquals(1, s.advance(1)); |
| } |
| |
| Index: lucene/core/src/test/org/apache/lucene/search/spans/TestSpans.java |
| =================================================================== |
| --- lucene/core/src/test/org/apache/lucene/search/spans/TestSpans.java (revision 1657441) |
| +++ lucene/core/src/test/org/apache/lucene/search/spans/TestSpans.java (working copy) |
| @@ -429,7 +429,7 @@ |
| slop, |
| ordered); |
| |
| - spanScorer = searcher.createNormalizedWeight(snq).scorer(ctx, ctx.reader().getLiveDocs()); |
| + spanScorer = searcher.createNormalizedWeight(snq).scorer(ctx, ctx.reader().getLiveDocs(), true); |
| } finally { |
| searcher.setSimilarity(oldSim); |
| } |
| Index: lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java |
| =================================================================== |
| --- lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java (revision 1657441) |
| +++ lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java (working copy) |
| @@ -111,17 +111,17 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| // We can only run as a top scorer: |
| throw new UnsupportedOperationException(); |
| } |
| |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| |
| // TODO: it could be better if we take acceptDocs |
| // into account instead of baseScorer? |
| - Scorer baseScorer = baseWeight.scorer(context, acceptDocs); |
| + Scorer baseScorer = baseWeight.scorer(context, acceptDocs, needsScores); |
| |
| DrillSidewaysScorer.DocsAndCost[] dims = new DrillSidewaysScorer.DocsAndCost[drillDowns.length]; |
| int nullCount = 0; |
| @@ -166,7 +166,7 @@ |
| dims[dim].disi = disi; |
| } |
| } else { |
| - DocIdSetIterator disi = ((Weight) drillDowns[dim]).scorer(context, null); |
| + DocIdSetIterator disi = ((Weight) drillDowns[dim]).scorer(context, null, needsScores); |
| if (disi == null) { |
| nullCount++; |
| continue; |
| Index: lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java |
| =================================================================== |
| --- lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java (revision 1657441) |
| +++ lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java (working copy) |
| @@ -169,6 +169,11 @@ |
| } |
| |
| @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| + |
| + @Override |
| public final void setScorer(Scorer scorer) throws IOException { |
| this.scorer = scorer; |
| } |
| Index: lucene/facet/src/test/org/apache/lucene/facet/AssertingSubDocsAtOnceCollector.java |
| =================================================================== |
| --- lucene/facet/src/test/org/apache/lucene/facet/AssertingSubDocsAtOnceCollector.java (revision 1657441) |
| +++ lucene/facet/src/test/org/apache/lucene/facet/AssertingSubDocsAtOnceCollector.java (working copy) |
| @@ -54,4 +54,9 @@ |
| } |
| } |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| } |
| Index: lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java |
| =================================================================== |
| --- lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java (revision 1657441) |
| +++ lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java (working copy) |
| @@ -682,6 +682,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| lastDocID = -1; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }); |
| |
| // Also separately verify that DS respects the |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java (working copy) |
| @@ -528,4 +528,9 @@ |
| leafComparators[i] = comparators[i].getLeafComparator(readerContext); |
| } |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return needsScores; |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java (working copy) |
| @@ -152,4 +152,9 @@ |
| this.doc = doc + readerContext.docBase; |
| } |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't: e.g. return sortWithinGroup.needsScores() |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java (working copy) |
| @@ -80,4 +80,9 @@ |
| filler = values.getValueFiller(); |
| mval = filler.getValue(); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java (working copy) |
| @@ -89,4 +89,9 @@ |
| } |
| |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java (working copy) |
| @@ -84,4 +84,8 @@ |
| mval = filler.getValue(); |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java (working copy) |
| @@ -78,4 +78,8 @@ |
| mval = filler.getValue(); |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java (working copy) |
| @@ -623,4 +623,9 @@ |
| } |
| |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java (working copy) |
| @@ -118,4 +118,9 @@ |
| } |
| } |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java (working copy) |
| @@ -141,4 +141,8 @@ |
| } |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java (working copy) |
| @@ -91,4 +91,9 @@ |
| super.doSetNextReader(readerContext); |
| index = DocValues.getSorted(readerContext.reader(), groupField); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java (working copy) |
| @@ -75,6 +75,11 @@ |
| groupedFacetHits = new ArrayList<>(initialSize); |
| segmentGroupedFacetHits = new SentinelIntSet(initialSize, Integer.MIN_VALUE); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| |
| // Implementation for single valued facet fields. |
| static class SV extends TermGroupFacetCollector { |
| Index: lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java |
| =================================================================== |
| --- lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java (revision 1657441) |
| +++ lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java (working copy) |
| @@ -76,4 +76,9 @@ |
| } |
| return null; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO, maybe we don't? |
| + } |
| } |
| Index: lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java |
| =================================================================== |
| --- lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java (revision 1657441) |
| +++ lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java (working copy) |
| @@ -133,6 +133,11 @@ |
| public void setScorer(org.apache.lucene.search.Scorer scorer) { |
| // Do Nothing |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }); |
| assertEquals(1, bitset.cardinality()); |
| final int maxDoc = indexReader.maxDoc(); |
| Index: lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java (working copy) |
| @@ -103,4 +103,8 @@ |
| } |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| } |
| Index: lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java (working copy) |
| @@ -167,7 +167,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| Terms terms = context.reader().terms(field); |
| if (terms == null) { |
| return null; |
| Index: lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/TermsWithScoreCollector.java (working copy) |
| @@ -247,4 +247,8 @@ |
| } |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| Index: lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java (working copy) |
| @@ -128,9 +128,9 @@ |
| // NOTE: acceptDocs applies (and is checked) only in the |
| // child document space |
| @Override |
| - public Scorer scorer(LeafReaderContext readerContext, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext readerContext, Bits acceptDocs, boolean needsScores) throws IOException { |
| |
| - final Scorer parentScorer = parentWeight.scorer(readerContext, null); |
| + final Scorer parentScorer = parentWeight.scorer(readerContext, null, needsScores); |
| |
| if (parentScorer == null) { |
| // No matches |
| Index: lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java (working copy) |
| @@ -482,4 +482,9 @@ |
| public float getMaxScore() { |
| return maxScore; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return trackScores || trackMaxScore || sort.needsScores(); |
| + } |
| } |
| Index: lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinIndexSearcher.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinIndexSearcher.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinIndexSearcher.java (working copy) |
| @@ -55,7 +55,7 @@ |
| // we force the use of Scorer (not BulkScorer) to make sure |
| // that the scorer passed to LeafCollector.setScorer supports |
| // Scorer.getChildren |
| - Scorer scorer = weight.scorer(ctx, ctx.reader().getLiveDocs()); |
| + Scorer scorer = weight.scorer(ctx, ctx.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| final LeafCollector leafCollector = collector.getLeafCollector(ctx); |
| leafCollector.setScorer(scorer); |
| Index: lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java |
| =================================================================== |
| --- lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java (revision 1657441) |
| +++ lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java (working copy) |
| @@ -160,9 +160,9 @@ |
| // NOTE: acceptDocs applies (and is checked) only in the |
| // parent document space |
| @Override |
| - public Scorer scorer(LeafReaderContext readerContext, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext readerContext, Bits acceptDocs, boolean needsScores) throws IOException { |
| |
| - final Scorer childScorer = childWeight.scorer(readerContext, readerContext.reader().getLiveDocs()); |
| + final Scorer childScorer = childWeight.scorer(readerContext, readerContext.reader().getLiveDocs(), needsScores); |
| if (childScorer == null) { |
| // No matches |
| return null; |
| @@ -188,7 +188,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - BlockJoinScorer scorer = (BlockJoinScorer) scorer(context, context.reader().getLiveDocs()); |
| + BlockJoinScorer scorer = (BlockJoinScorer) scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer != null && scorer.advance(doc) == doc) { |
| return scorer.explain(context.docBase); |
| } |
| Index: lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java |
| =================================================================== |
| --- lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (revision 1657441) |
| +++ lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java (working copy) |
| @@ -1148,7 +1148,7 @@ |
| |
| ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg); |
| Weight weight = s.createNormalizedWeight(q); |
| - DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null); |
| + DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null, true); |
| assertEquals(1, disi.advance(1)); |
| r.close(); |
| dir.close(); |
| @@ -1182,7 +1182,7 @@ |
| |
| ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg); |
| Weight weight = s.createNormalizedWeight(q); |
| - DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null); |
| + DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null, true); |
| assertEquals(2, disi.advance(0)); |
| r.close(); |
| dir.close(); |
| Index: lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java |
| =================================================================== |
| --- lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (revision 1657441) |
| +++ lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (working copy) |
| @@ -310,6 +310,11 @@ |
| assertFalse("optimized bulkScorer was not used for join query embedded in boolean query!", sawFive); |
| } |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }); |
| |
| indexSearcher.getIndexReader().close(); |
| @@ -495,6 +500,11 @@ |
| } |
| }; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return topScoreDocCollector.needsScores(); |
| + } |
| }); |
| // Asserting bit set... |
| if (VERBOSE) { |
| @@ -673,6 +683,11 @@ |
| public void setScorer(Scorer scorer) { |
| this.scorer = scorer; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| } else { |
| fromSearcher.search(new TermQuery(new Term("value", uniqueRandomValue)), new SimpleCollector() { |
| @@ -705,6 +720,11 @@ |
| public void setScorer(Scorer scorer) { |
| this.scorer = scorer; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| } |
| |
| @@ -757,6 +777,11 @@ |
| |
| @Override |
| public void setScorer(Scorer scorer) {} |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }); |
| } |
| queryVals.put(uniqueRandomValue, docToJoinScore); |
| Index: lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java |
| =================================================================== |
| --- lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (revision 1657441) |
| +++ lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (working copy) |
| @@ -568,7 +568,11 @@ |
| public void setScorer(Scorer scorer) { |
| this.scorer = scorer; |
| } |
| - |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| float score = scores[0]; |
| return score; |
| Index: lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java |
| =================================================================== |
| --- lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java (revision 1657441) |
| +++ lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java (working copy) |
| @@ -234,14 +234,14 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - Scorer subQueryScorer = subQueryWeight.scorer(context, acceptDocs); |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + Scorer subQueryScorer = subQueryWeight.scorer(context, acceptDocs, needsScores); |
| if (subQueryScorer == null) { |
| return null; |
| } |
| Scorer[] valSrcScorers = new Scorer[valSrcWeights.length]; |
| for(int i = 0; i < valSrcScorers.length; i++) { |
| - valSrcScorers[i] = valSrcWeights[i].scorer(context, acceptDocs); |
| + valSrcScorers[i] = valSrcWeights[i].scorer(context, acceptDocs, needsScores); |
| } |
| return new CustomScorer(CustomScoreQuery.this.getCustomScoreProvider(context), this, queryWeight, subQueryScorer, valSrcScorers); |
| } |
| Index: lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java |
| =================================================================== |
| --- lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java (revision 1657441) |
| +++ lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java (working copy) |
| @@ -97,8 +97,8 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - Scorer subQueryScorer = qWeight.scorer(context, acceptDocs); |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + Scorer subQueryScorer = qWeight.scorer(context, acceptDocs, needsScores); |
| if (subQueryScorer == null) { |
| return null; |
| } |
| Index: lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java |
| =================================================================== |
| --- lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java (revision 1657441) |
| +++ lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java (working copy) |
| @@ -89,13 +89,13 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| return new AllScorer(context, acceptDocs, this, queryWeight); |
| } |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - return ((AllScorer)scorer(context, context.reader().getLiveDocs())).explain(doc); |
| + return ((AllScorer)scorer(context, context.reader().getLiveDocs(), true)).explain(doc); |
| } |
| } |
| |
| Index: lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java |
| =================================================================== |
| --- lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java (revision 1657441) |
| +++ lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java (working copy) |
| @@ -123,7 +123,7 @@ |
| try { |
| if (doc < lastDocRequested) { |
| if (noMatches) return defVal; |
| - scorer = weight.scorer(readerContext, acceptDocs); |
| + scorer = weight.scorer(readerContext, acceptDocs, true); |
| if (scorer==null) { |
| noMatches = true; |
| return defVal; |
| @@ -154,7 +154,7 @@ |
| try { |
| if (doc < lastDocRequested) { |
| if (noMatches) return false; |
| - scorer = weight.scorer(readerContext, acceptDocs); |
| + scorer = weight.scorer(readerContext, acceptDocs, true); |
| scorerDoc = -1; |
| if (scorer==null) { |
| noMatches = true; |
| @@ -212,7 +212,7 @@ |
| mval.exists = false; |
| return; |
| } |
| - scorer = weight.scorer(readerContext, acceptDocs); |
| + scorer = weight.scorer(readerContext, acceptDocs, true); |
| scorerDoc = -1; |
| if (scorer==null) { |
| noMatches = true; |
| Index: lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java |
| =================================================================== |
| --- lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java (revision 1657441) |
| +++ lucene/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java (working copy) |
| @@ -95,6 +95,11 @@ |
| } |
| totalMatched++; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| |
| void checkNrHits() { |
| Assert.assertEquals(queryText + ": nr of hits", expectedDocNrs.length, totalMatched); |
| Index: lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java |
| =================================================================== |
| --- lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java (revision 1657441) |
| +++ lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java (working copy) |
| @@ -385,7 +385,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| |
| // Initialize the enums; null for a given slot means that term didn't appear in this reader |
| EnumAndScorer[] enums = new EnumAndScorer[idToTerm.size()]; |
| Index: lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java |
| =================================================================== |
| --- lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java (revision 1657441) |
| +++ lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java (working copy) |
| @@ -302,6 +302,11 @@ |
| public void collect(int docID) throws IOException { |
| assertEquals(3, scorer.freq()); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }); |
| |
| w.close(); |
| Index: lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java |
| =================================================================== |
| --- lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (revision 1657441) |
| +++ lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (working copy) |
| @@ -1457,6 +1457,29 @@ |
| dir.close(); |
| } |
| |
| + public void testDidntWantFreqsButAskedAnyway() throws Exception { |
| + Directory dir = newDirectory(); |
| + IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); |
| + iwc.setCodec(getCodec()); |
| + RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); |
| + Document doc = new Document(); |
| + doc.add(newTextField("field", "value", Field.Store.NO)); |
| + iw.addDocument(doc); |
| + iw.addDocument(doc); |
| + DirectoryReader ir = iw.getReader(); |
| + LeafReader ar = getOnlySegmentReader(ir); |
| + TermsEnum termsEnum = ar.terms("field").iterator(null); |
| + assertTrue(termsEnum.seekExact(new BytesRef("value"))); |
| + DocsEnum docsEnum = termsEnum.docs(null, null, DocsEnum.FLAG_NONE); |
| + assertEquals(0, docsEnum.nextDoc()); |
| + assertEquals(1, docsEnum.freq()); |
| + assertEquals(1, docsEnum.nextDoc()); |
| + assertEquals(1, docsEnum.freq()); |
| + ir.close(); |
| + iw.close(); |
| + dir.close(); |
| + } |
| + |
| // tests that ghost fields still work |
| // TODO: can this be improved? |
| public void testGhosts() throws Exception { |
| Index: lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java |
| =================================================================== |
| --- lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java (revision 1657441) |
| +++ lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java (working copy) |
| @@ -58,15 +58,15 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - final Scorer inScorer = in.scorer(context, acceptDocs); |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + final Scorer inScorer = in.scorer(context, acceptDocs, needsScores); |
| assert inScorer == null || inScorer.docID() == -1; |
| return AssertingScorer.wrap(new Random(random.nextLong()), inScorer); |
| } |
| |
| @Override |
| - public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - BulkScorer inScorer = in.bulkScorer(context, acceptDocs); |
| + public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + BulkScorer inScorer = in.bulkScorer(context, acceptDocs, needsScores); |
| if (inScorer == null) { |
| return null; |
| } |
| Index: lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java |
| =================================================================== |
| --- lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java (revision 1657441) |
| +++ lucene/test-framework/src/java/org/apache/lucene/search/CheckHits.java (working copy) |
| @@ -139,6 +139,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| base = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| } |
| |
| /** |
| @@ -507,6 +512,11 @@ |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| base = context.docBase; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| |
| } |
| Index: lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java |
| =================================================================== |
| --- lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (revision 1657441) |
| +++ lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java (working copy) |
| @@ -264,7 +264,7 @@ |
| if (scorer == null) { |
| Weight w = s.createNormalizedWeight(q); |
| LeafReaderContext context = readerContextArray.get(leafPtr); |
| - scorer = w.scorer(context, context.reader().getLiveDocs()); |
| + scorer = w.scorer(context, context.reader().getLiveDocs(), true); |
| } |
| |
| int op = order[(opidx[0]++) % order.length]; |
| @@ -302,6 +302,11 @@ |
| } |
| |
| @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| + |
| + @Override |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| // confirm that skipping beyond the last doc, on the |
| // previous reader, hits NO_MORE_DOCS |
| @@ -311,7 +316,7 @@ |
| indexSearcher.setSimilarity(s.getSimilarity()); |
| Weight w = indexSearcher.createNormalizedWeight(q); |
| LeafReaderContext ctx = (LeafReaderContext)indexSearcher.getTopReaderContext(); |
| - Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs()); |
| + Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS; |
| Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more); |
| @@ -333,7 +338,7 @@ |
| indexSearcher.setSimilarity(s.getSimilarity()); |
| Weight w = indexSearcher.createNormalizedWeight(q); |
| LeafReaderContext ctx = previousReader.getContext(); |
| - Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs()); |
| + Scorer scorer = w.scorer(ctx, ctx.reader().getLiveDocs(), true); |
| if (scorer != null) { |
| boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS; |
| Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more); |
| @@ -364,7 +369,7 @@ |
| long startMS = System.currentTimeMillis(); |
| for (int i=lastDoc[0]+1; i<=doc; i++) { |
| Weight w = s.createNormalizedWeight(q); |
| - Scorer scorer = w.scorer(context.get(leafPtr), liveDocs); |
| + Scorer scorer = w.scorer(context.get(leafPtr), liveDocs, true); |
| Assert.assertTrue("query collected "+doc+" but skipTo("+i+") says no more docs!",scorer.advance(i) != DocIdSetIterator.NO_MORE_DOCS); |
| Assert.assertEquals("query collected "+doc+" but skipTo("+i+") got to "+scorer.docID(),doc,scorer.docID()); |
| float skipToScore = scorer.score(); |
| @@ -382,6 +387,11 @@ |
| throw new RuntimeException(e); |
| } |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| |
| @Override |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| @@ -392,7 +402,7 @@ |
| IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader); |
| indexSearcher.setSimilarity(s.getSimilarity()); |
| Weight w = indexSearcher.createNormalizedWeight(q); |
| - Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs()); |
| + Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs(), true); |
| if (scorer != null) { |
| boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS; |
| Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more); |
| @@ -413,7 +423,7 @@ |
| IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader); |
| indexSearcher.setSimilarity(s.getSimilarity()); |
| Weight w = indexSearcher.createNormalizedWeight(q); |
| - Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs()); |
| + Scorer scorer = w.scorer((LeafReaderContext)indexSearcher.getTopReaderContext(), previousReader.getLiveDocs(), true); |
| if (scorer != null) { |
| boolean more = scorer.advance(lastDoc[0] + 1) != DocIdSetIterator.NO_MORE_DOCS; |
| Assert.assertFalse("query's last doc was "+ lastDoc[0] +" but skipTo("+(lastDoc[0]+1)+") got to "+scorer.docID(),more); |
| @@ -425,8 +435,8 @@ |
| public static void checkBulkScorerSkipTo(Random r, Query query, IndexSearcher searcher) throws IOException { |
| Weight weight = searcher.createNormalizedWeight(query); |
| for (LeafReaderContext context : searcher.getIndexReader().leaves()) { |
| - final Scorer scorer = weight.scorer(context, context.reader().getLiveDocs()); |
| - final BulkScorer bulkScorer = weight.bulkScorer(context, context.reader().getLiveDocs()); |
| + final Scorer scorer = weight.scorer(context, context.reader().getLiveDocs(), true); |
| + final BulkScorer bulkScorer = weight.bulkScorer(context, context.reader().getLiveDocs(), true); |
| if (scorer == null && bulkScorer == null) { |
| continue; |
| } |
| Index: lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java |
| =================================================================== |
| --- lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (revision 1657441) |
| +++ lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java (working copy) |
| @@ -178,8 +178,8 @@ |
| } |
| |
| // not efficient, but simple! |
| - TopDocs td1 = s1.search(q1, filter, reader.maxDoc()); |
| - TopDocs td2 = s2.search(q2, filter, reader.maxDoc()); |
| + TopDocs td1 = s1.search(q1, filter, reader.maxDoc(), random().nextBoolean() ? Sort.INDEXORDER : Sort.RELEVANCE); |
| + TopDocs td2 = s2.search(q2, filter, reader.maxDoc(), random().nextBoolean() ? Sort.INDEXORDER : Sort.RELEVANCE); |
| assertTrue(td1.totalHits <= td2.totalHits); |
| |
| // fill the superset into a bitset |
| Index: solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java |
| =================================================================== |
| --- solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java (revision 1657441) |
| +++ solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java (working copy) |
| @@ -167,4 +167,9 @@ |
| public long getNumQueries() { |
| return 0l; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO: is this true? |
| + } |
| } |
| Index: solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FieldFacetAccumulator.java |
| =================================================================== |
| --- solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FieldFacetAccumulator.java (revision 1657441) |
| +++ solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FieldFacetAccumulator.java (working copy) |
| @@ -146,4 +146,8 @@ |
| @Override |
| public NamedList<?> export() { return null; } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO: is this true? |
| + } |
| } |
| Index: solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/QueryFacetAccumulator.java |
| =================================================================== |
| --- solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/QueryFacetAccumulator.java (revision 1657441) |
| +++ solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/QueryFacetAccumulator.java (working copy) |
| @@ -66,4 +66,8 @@ |
| return null; |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO: is this true? |
| + } |
| } |
| Index: solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.java (working copy) |
| @@ -510,6 +510,11 @@ |
| } |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO: is this always true? |
| + } |
| + |
| public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { |
| final int docBase = context.docBase; |
| |
| @@ -581,6 +586,11 @@ |
| this.field = field; |
| this.collapsedSet = collapsedSet; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO: is this always true? |
| + } |
| |
| public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { |
| final int docBase = context.docBase; |
| Index: solr/core/src/java/org/apache/solr/schema/LatLonType.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/schema/LatLonType.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/schema/LatLonType.java (working copy) |
| @@ -343,13 +343,13 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| return new SpatialScorer(context, acceptDocs, this, queryWeight); |
| } |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - return ((SpatialScorer)scorer(context, context.reader().getLiveDocs())).explain(doc); |
| + return ((SpatialScorer)scorer(context, context.reader().getLiveDocs(), true)).explain(doc); |
| } |
| } |
| |
| Index: solr/core/src/java/org/apache/solr/search/DelegatingCollector.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/DelegatingCollector.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/DelegatingCollector.java (working copy) |
| @@ -64,6 +64,11 @@ |
| } |
| |
| @Override |
| + public boolean needsScores() { |
| + return delegate.needsScores(); |
| + } |
| + |
| + @Override |
| public void collect(int doc) throws IOException { |
| leafDelegate.collect(doc); |
| } |
| Index: solr/core/src/java/org/apache/solr/search/DocSetCollector.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/DocSetCollector.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/DocSetCollector.java (working copy) |
| @@ -84,6 +84,11 @@ |
| } |
| |
| @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| + |
| + @Override |
| protected void doSetNextReader(LeafReaderContext context) throws IOException { |
| this.base = context.docBase; |
| } |
| Index: solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.java (working copy) |
| @@ -174,5 +174,9 @@ |
| return new TopDocs(totalHits, scoreDocs, 0.0f); |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; // TODO: is this the case? |
| + } |
| } |
| } |
| \ No newline at end of file |
| Index: solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (working copy) |
| @@ -304,7 +304,7 @@ |
| |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| if (filter == null) { |
| boolean debug = rb != null && rb.isDebug(); |
| long start = debug ? System.currentTimeMillis() : 0; |
| @@ -573,7 +573,7 @@ |
| |
| @Override |
| public Explanation explain(LeafReaderContext context, int doc) throws IOException { |
| - Scorer scorer = scorer(context, context.reader().getLiveDocs()); |
| + Scorer scorer = scorer(context, context.reader().getLiveDocs(), true); |
| boolean exists = scorer.advance(doc) == doc; |
| |
| ComplexExplanation result = new ComplexExplanation(); |
| Index: solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/ReRankQParserPlugin.java (working copy) |
| @@ -196,8 +196,8 @@ |
| return mainWeight.getValueForNormalization(); |
| } |
| |
| - public Scorer scorer(LeafReaderContext context, Bits bits) throws IOException { |
| - return mainWeight.scorer(context, bits); |
| + public Scorer scorer(LeafReaderContext context, Bits bits, boolean needsScores) throws IOException { |
| + return mainWeight.scorer(context, bits, needsScores); |
| } |
| |
| public Query getQuery() { |
| @@ -266,6 +266,11 @@ |
| return mainCollector.getLeafCollector(context); |
| } |
| |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| + |
| public TopDocs topDocs(int start, int howMany) { |
| |
| try { |
| Index: solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (working copy) |
| @@ -119,7 +119,7 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| return new ConstantScorer(context, this, queryWeight, acceptDocs); |
| } |
| |
| Index: solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (working copy) |
| @@ -1641,6 +1641,11 @@ |
| public void collect(int doc) { |
| numHits[0]++; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return false; |
| + } |
| }; |
| } else { |
| collector = new SimpleCollector() { |
| @@ -1655,6 +1660,11 @@ |
| float score = scorer.score(); |
| if (score > topscore[0]) topscore[0]=score; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }; |
| } |
| |
| @@ -1739,6 +1749,11 @@ |
| float score = scorer.score(); |
| if (score > topscore[0]) topscore[0] = score; |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| }; |
| |
| collector = MultiCollector.wrap(setCollector, topScoreCollector); |
| @@ -2489,7 +2504,7 @@ |
| iterators.add(iter); |
| } |
| for (Weight w : weights) { |
| - Scorer scorer = w.scorer(context, context.reader().getLiveDocs()); |
| + Scorer scorer = w.scorer(context, context.reader().getLiveDocs(), true); |
| if (scorer == null) return null; |
| iterators.add(scorer); |
| } |
| Index: solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java (working copy) |
| @@ -86,8 +86,8 @@ |
| } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - return w.scorer(context, null); |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + return w.scorer(context, null, needsScores); |
| } |
| } |
| |
| Index: solr/core/src/java/org/apache/solr/update/DeleteByQueryWrapper.java |
| =================================================================== |
| --- solr/core/src/java/org/apache/solr/update/DeleteByQueryWrapper.java (revision 1657441) |
| +++ solr/core/src/java/org/apache/solr/update/DeleteByQueryWrapper.java (working copy) |
| @@ -82,8 +82,8 @@ |
| public void normalize(float norm, float topLevelBoost) { inner.normalize(norm, topLevelBoost); } |
| |
| @Override |
| - public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException { |
| - return inner.scorer(privateContext.getIndexReader().leaves().get(0), acceptDocs); |
| + public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException { |
| + return inner.scorer(privateContext.getIndexReader().leaves().get(0), acceptDocs, needsScores); |
| } |
| }; |
| } |
| Index: solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java |
| =================================================================== |
| --- solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java (revision 1657441) |
| +++ solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java (working copy) |
| @@ -760,6 +760,11 @@ |
| public int getTotalHits() { |
| return list.size(); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| |
| class TestCollector1 extends TopDocsCollector { |
| @@ -817,6 +822,11 @@ |
| public int getTotalHits() { |
| return list.size(); |
| } |
| + |
| + @Override |
| + public boolean needsScores() { |
| + return true; |
| + } |
| } |
| |
| } |