mirror of https://github.com/apache/lucene.git
LUCENE-6218: don't decode freqs or enumerate all positions when scoring is not needed
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1657554 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0beba279a5
commit
010e352f60
|
@ -60,6 +60,9 @@ Optimizations
|
|||
are greater than one and is used when queries produce dense result sets.
|
||||
(Adrien Grand)
|
||||
|
||||
* LUCENE-6218: Don't decode frequencies or match all positions when scoring
|
||||
is not needed. (Robert Muir)
|
||||
|
||||
API Changes
|
||||
|
||||
* LUCENE-6204, LUCENE-6208: Simplify CompoundFormat: remove files()
|
||||
|
@ -68,6 +71,9 @@ API Changes
|
|||
* LUCENE-6217: Add IndexWriter.isOpen and getTragicException. (Simon
|
||||
Willnauer, Mike McCandless)
|
||||
|
||||
* LUCENE-6218: Add Collector.needsScores() and needsScores parameter
|
||||
to Weight.scorer(). (Robert Muir)
|
||||
|
||||
Other
|
||||
|
||||
* LUCENE-6193: Collapse identical catch branches in try-catch statements.
|
||||
|
|
|
@ -320,7 +320,7 @@ public final class Lucene50PostingsReader extends PostingsReaderBase {
|
|||
|
||||
doc = -1;
|
||||
this.needsFreq = (flags & DocsEnum.FLAG_FREQS) != 0;
|
||||
if (!indexHasFreq) {
|
||||
if (indexHasFreq == false || needsFreq == false) {
|
||||
Arrays.fill(freqBuffer, 1);
|
||||
}
|
||||
accum = 0;
|
||||
|
|
|
@ -242,7 +242,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
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 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
/** 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 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
|
||||
@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 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
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 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
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 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
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
|
||||
|
|
|
@ -297,6 +297,11 @@ public abstract class CachingCollector extends FilterCollector {
|
|||
@Override
|
||||
public void collect(int doc) {}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
};
|
||||
return create(other, cacheScores, maxRAMMB);
|
||||
}
|
||||
|
|
|
@ -73,4 +73,10 @@ public interface Collector {
|
|||
*/
|
||||
LeafCollector getLeafCollector(LeafReaderContext context) throws IOException;
|
||||
|
||||
/**
|
||||
* Indicates if document scores are needed by this collector.
|
||||
*
|
||||
* @return {@code true} if scores are needed.
|
||||
*/
|
||||
boolean needsScores();
|
||||
}
|
||||
|
|
|
@ -134,14 +134,14 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@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 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@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 @@ public class ConstantScoreQuery extends Query {
|
|||
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 @@ public class ConstantScoreQuery extends Query {
|
|||
|
||||
@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();
|
||||
|
|
|
@ -153,11 +153,11 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
|
|||
|
||||
/** 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);
|
||||
}
|
||||
|
|
|
@ -55,11 +55,13 @@ final class ExactPhraseScorer extends Scorer {
|
|||
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 class ExactPhraseScorer extends Scorer {
|
|||
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.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -45,4 +45,8 @@ public class FilterCollector implements Collector {
|
|||
return getClass().getSimpleName() + "(" + in + ")";
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return in.needsScores();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -119,7 +119,7 @@ public class FilteredQuery extends Query {
|
|||
|
||||
// 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 @@ public class FilteredQuery extends Query {
|
|||
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 @@ public class FilteredQuery extends Query {
|
|||
return null;
|
||||
}
|
||||
|
||||
return strategy.filteredBulkScorer(context, weight, filterDocIdSet);
|
||||
return strategy.filteredBulkScorer(context, weight, filterDocIdSet, needsScores);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -465,7 +465,7 @@ public class FilteredQuery extends Query {
|
|||
* @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 @@ public class FilteredQuery extends Query {
|
|||
* @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 class FilteredQuery extends Query {
|
|||
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 @@ public class FilteredQuery extends Query {
|
|||
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,14 +551,14 @@ public class FilteredQuery extends Query {
|
|||
|
||||
@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.
|
||||
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,30 +587,28 @@ public class FilteredQuery extends Query {
|
|||
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);
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -586,7 +586,7 @@ public class IndexSearcher {
|
|||
// 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);
|
||||
|
|
|
@ -114,7 +114,7 @@ public class MatchAllDocsQuery extends Query {
|
|||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
||||
|
|
|
@ -92,6 +92,16 @@ public class MultiCollector implements Collector {
|
|||
this.collectors = collectors;
|
||||
}
|
||||
|
||||
@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];
|
||||
|
|
|
@ -179,7 +179,7 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@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 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
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) {
|
||||
|
|
|
@ -245,7 +245,7 @@ public class PhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@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 @@ public class PhraseQuery extends Query {
|
|||
}
|
||||
|
||||
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 @@ public class PhraseQuery extends Query {
|
|||
|
||||
@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) {
|
||||
|
|
|
@ -82,7 +82,7 @@ public abstract class QueryRescorer extends Rescorer {
|
|||
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) {
|
||||
|
|
|
@ -56,7 +56,7 @@ public class QueryWrapperFilter extends Filter {
|
|||
return new DocIdSet() {
|
||||
@Override
|
||||
public DocIdSetIterator iterator() throws IOException {
|
||||
return weight.scorer(privateContext, acceptDocs);
|
||||
return weight.scorer(privateContext, acceptDocs, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -50,11 +50,13 @@ final class SloppyPhraseScorer extends Scorer {
|
|||
|
||||
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 @@ final class SloppyPhraseScorer extends Scorer {
|
|||
if (matchLength <= slop) {
|
||||
freq += docScorer.computeSlopFactor(matchLength); // score match
|
||||
numMatches++;
|
||||
if (!needsScores) {
|
||||
return freq;
|
||||
}
|
||||
}
|
||||
pq.add(pp);
|
||||
pp = pq.pop();
|
||||
|
|
|
@ -75,13 +75,13 @@ public class TermQuery extends Query {
|
|||
}
|
||||
|
||||
@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 @@ public class TermQuery extends Query {
|
|||
|
||||
@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) {
|
||||
|
|
|
@ -157,6 +157,11 @@ public class TimeLimitingCollector implements Collector {
|
|||
};
|
||||
}
|
||||
|
||||
@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
|
||||
|
|
|
@ -129,8 +129,8 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
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 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
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 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
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 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
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,18 +520,25 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
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
|
||||
// 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 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
|
||||
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 @@ public abstract class TopFieldCollector extends TopDocsCollector<Entry> {
|
|||
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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -207,4 +207,9 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
|||
|
||||
return new TopDocs(totalHits, results, maxScore);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,4 +34,9 @@ public class TotalHitCountCollector extends SimpleCollector {
|
|||
public void collect(int doc) {
|
||||
totalHits++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.util.Bits;
|
|||
* {@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 @@ import org.apache.lucene.util.Bits;
|
|||
* <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 @@ public abstract class Weight {
|
|||
* @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 @@ public abstract class Weight {
|
|||
* @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;
|
||||
|
|
|
@ -436,15 +436,13 @@ on the built-in available scoring models and extending or changing Similarity.
|
|||
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>
|
||||
|
|
|
@ -148,14 +148,14 @@ public class PayloadNearQuery extends SpanNearQuery {
|
|||
}
|
||||
|
||||
@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));
|
||||
}
|
||||
|
||||
@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) {
|
||||
|
|
|
@ -79,7 +79,7 @@ public class PayloadTermQuery extends SpanTermQuery {
|
|||
}
|
||||
|
||||
@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 @@ public class PayloadTermQuery extends SpanTermQuery {
|
|||
|
||||
@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) {
|
||||
|
|
|
@ -81,7 +81,7 @@ public class SpanWeight extends Weight {
|
|||
}
|
||||
|
||||
@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 @@ public class SpanWeight extends Weight {
|
|||
|
||||
@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) {
|
||||
|
|
|
@ -437,6 +437,11 @@ public class TestOmitTf extends LuceneTestCase {
|
|||
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 */
|
||||
|
|
|
@ -53,6 +53,10 @@ final class JustCompileSearch {
|
|||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
}
|
||||
|
||||
static final class JustCompileDocIdSet extends DocIdSet {
|
||||
|
@ -254,6 +258,10 @@ final class JustCompileSearch {
|
|||
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
throw new UnsupportedOperationException( UNSUPPORTED_MSG );
|
||||
}
|
||||
}
|
||||
|
||||
static final class JustCompileWeight extends Weight {
|
||||
|
@ -279,7 +287,7 @@ final class JustCompileSearch {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs) {
|
||||
public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
|
|
|
@ -46,6 +46,10 @@ public class MultiCollectorTest extends LuceneTestCase {
|
|||
setScorerCalled = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -707,14 +707,14 @@ public class TestBooleanCoord extends LuceneTestCase {
|
|||
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);
|
||||
|
||||
// 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;
|
||||
|
|
|
@ -189,7 +189,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
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 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue());
|
||||
hits.set(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
while (end.intValue() < docCount) {
|
||||
|
|
|
@ -235,7 +235,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
|
||||
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 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
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);
|
||||
|
|
|
@ -235,6 +235,11 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
return summaries;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
return new LeafCollector() {
|
||||
|
@ -282,8 +287,8 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -92,12 +92,12 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@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 {
|
||||
|
|
|
@ -57,6 +57,11 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
@Override
|
||||
public void collect(int doc) throws IOException {}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void testBasic() throws Exception {
|
||||
|
@ -79,6 +84,11 @@ public class TestCachingCollector extends LuceneTestCase {
|
|||
assertEquals(prevDocID + 1, doc);
|
||||
prevDocID = doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -67,6 +67,11 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
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]);
|
||||
}
|
||||
|
|
|
@ -180,7 +180,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
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 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
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"));
|
||||
|
|
|
@ -74,6 +74,10 @@ public class TestDocBoost extends LuceneTestCase {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
|
||||
float lastScore = 0.0f;
|
||||
|
|
|
@ -81,6 +81,10 @@ public class TestEarlyTermination extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
|
|
|
@ -131,11 +131,11 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
|
|||
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;
|
||||
|
|
|
@ -241,6 +241,11 @@ public class TestMultiTermConstantScore extends BaseTestRangeFilter {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
|
||||
//
|
||||
|
|
|
@ -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) + ")";
|
||||
}
|
||||
}
|
||||
}
|
|
@ -443,7 +443,7 @@ public class TestQueryRescorer extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@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;
|
||||
|
|
|
@ -91,6 +91,11 @@ public class TestScoreCachingWrappingScorer extends LuceneTestCase {
|
|||
this.scorer = new ScoreCachingWrappingScorer(scorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private static final float[] scores = new float[] { 0.7767749f, 1.7839992f,
|
||||
|
|
|
@ -115,6 +115,11 @@ public class TestScorerPerf extends LuceneTestCase {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
docBase = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -86,7 +86,8 @@ public class TestSimilarity extends LuceneTestCase {
|
|||
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 @@ public class TestSimilarity extends LuceneTestCase {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
|
||||
PhraseQuery pq = new PhraseQuery();
|
||||
|
@ -129,6 +134,10 @@ public class TestSimilarity extends LuceneTestCase {
|
|||
//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 @@ public class TestSimilarity extends LuceneTestCase {
|
|||
//System.out.println("Doc=" + doc + " score=" + score);
|
||||
assertEquals(2.0f, scorer.score(), 0);
|
||||
}
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
|
||||
reader.close();
|
||||
|
|
|
@ -191,6 +191,11 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
|
|||
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 @@ public class TestSloppyPhraseQuery extends LuceneTestCase {
|
|||
assertFalse(Float.isInfinite(scorer.freq()));
|
||||
assertFalse(Float.isInfinite(scorer.score()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
QueryUtils.check(random(), pq, searcher);
|
||||
}
|
||||
|
|
|
@ -78,7 +78,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
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 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
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 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
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 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
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);
|
||||
|
|
|
@ -357,6 +357,11 @@ public class TestTimeLimitingCollector extends LuceneTestCase {
|
|||
docBase = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -71,6 +71,11 @@ public class TestTopDocsCollector extends LuceneTestCase {
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
// Scores array to be used by MyTopDocsCollector. If it is changed, MAX_SCORE
|
||||
|
|
|
@ -183,7 +183,7 @@ public class TestNearSpansOrdered extends LuceneTestCase {
|
|||
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));
|
||||
}
|
||||
|
||||
|
|
|
@ -429,7 +429,7 @@ public class TestSpans extends LuceneTestCase {
|
|||
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);
|
||||
}
|
||||
|
|
|
@ -111,17 +111,17 @@ class DrillSidewaysQuery extends Query {
|
|||
}
|
||||
|
||||
@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 @@ class DrillSidewaysQuery extends Query {
|
|||
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;
|
||||
|
|
|
@ -168,6 +168,11 @@ public class FacetsCollector extends SimpleCollector {
|
|||
totalHits++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final void setScorer(Scorer scorer) throws IOException {
|
||||
this.scorer = scorer;
|
||||
|
|
|
@ -54,4 +54,9 @@ class AssertingSubDocsAtOnceCollector extends SimpleCollector {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -682,6 +682,11 @@ public class TestDrillSideways extends FacetTestCase {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
lastDocID = -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
});
|
||||
|
||||
// Also separately verify that DS respects the
|
||||
|
|
|
@ -528,4 +528,9 @@ public class BlockGroupingCollector extends SimpleCollector {
|
|||
leafComparators[i] = comparators[i].getLeafComparator(readerContext);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return needsScores;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -152,4 +152,9 @@ public class FunctionAllGroupHeadsCollector extends AbstractAllGroupHeadsCollect
|
|||
this.doc = doc + readerContext.docBase;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't: e.g. return sortWithinGroup.needsScores()
|
||||
}
|
||||
}
|
||||
|
|
|
@ -80,4 +80,9 @@ public class FunctionAllGroupsCollector extends AbstractAllGroupsCollector<Mutab
|
|||
filler = values.getValueFiller();
|
||||
mval = filler.getValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -89,4 +89,9 @@ public class FunctionDistinctValuesCollector extends AbstractDistinctValuesColle
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -84,4 +84,8 @@ public class FunctionFirstPassGroupingCollector extends AbstractFirstPassGroupin
|
|||
mval = filler.getValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -78,4 +78,8 @@ public class FunctionSecondPassGroupingCollector extends AbstractSecondPassGroup
|
|||
mval = filler.getValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -623,4 +623,9 @@ public abstract class TermAllGroupHeadsCollector<GH extends AbstractAllGroupHead
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -118,4 +118,9 @@ public class TermAllGroupsCollector extends AbstractAllGroupsCollector<BytesRef>
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -141,4 +141,8 @@ public class TermDistinctValuesCollector extends AbstractDistinctValuesCollector
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -91,4 +91,9 @@ public class TermFirstPassGroupingCollector extends AbstractFirstPassGroupingCol
|
|||
super.doSetNextReader(readerContext);
|
||||
index = DocValues.getSorted(readerContext.reader(), groupField);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -76,6 +76,11 @@ public abstract class TermGroupFacetCollector extends AbstractGroupFacetCollecto
|
|||
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 {
|
||||
|
||||
|
|
|
@ -76,4 +76,9 @@ public class TermSecondPassGroupingCollector extends AbstractSecondPassGroupingC
|
|||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO, maybe we don't?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -133,6 +133,11 @@ public class HighlighterPhraseTest extends LuceneTestCase {
|
|||
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();
|
||||
|
|
|
@ -103,4 +103,8 @@ abstract class TermsCollector extends SimpleCollector {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -167,7 +167,7 @@ class TermsIncludingScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@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;
|
||||
|
|
|
@ -247,4 +247,8 @@ abstract class TermsWithScoreCollector extends SimpleCollector {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -128,9 +128,9 @@ public class ToChildBlockJoinQuery extends Query {
|
|||
// 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
|
||||
|
|
|
@ -482,4 +482,9 @@ public class ToParentBlockJoinCollector implements Collector {
|
|||
public float getMaxScore() {
|
||||
return maxScore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return trackScores || trackMaxScore || sort.needsScores();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -55,7 +55,7 @@ public class ToParentBlockJoinIndexSearcher extends IndexSearcher {
|
|||
// 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);
|
||||
|
|
|
@ -160,9 +160,9 @@ public class ToParentBlockJoinQuery extends Query {
|
|||
// 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 @@ public class ToParentBlockJoinQuery extends Query {
|
|||
|
||||
@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);
|
||||
}
|
||||
|
|
|
@ -1148,7 +1148,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
|
||||
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 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
|
||||
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();
|
||||
|
|
|
@ -310,6 +310,11 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
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 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return topScoreDocCollector.needsScores();
|
||||
}
|
||||
});
|
||||
// Asserting bit set...
|
||||
if (VERBOSE) {
|
||||
|
@ -673,6 +683,11 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
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 class TestJoinUtil extends LuceneTestCase {
|
|||
public void setScorer(Scorer scorer) {
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -757,6 +777,11 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) {}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
});
|
||||
}
|
||||
queryVals.put(uniqueRandomValue, docToJoinScore);
|
||||
|
|
|
@ -569,6 +569,10 @@ public class MemoryIndex {
|
|||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
float score = scores[0];
|
||||
return score;
|
||||
|
|
|
@ -234,14 +234,14 @@ public class CustomScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
|
|
|
@ -97,8 +97,8 @@ public class BoostedQuery extends Query {
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
|
|
@ -89,13 +89,13 @@ public class FunctionQuery extends Query {
|
|||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -123,7 +123,7 @@ class QueryDocValues extends FloatDocValues {
|
|||
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 @@ class QueryDocValues extends FloatDocValues {
|
|||
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 @@ class QueryDocValues extends FloatDocValues {
|
|||
mval.exists = false;
|
||||
return;
|
||||
}
|
||||
scorer = weight.scorer(readerContext, acceptDocs);
|
||||
scorer = weight.scorer(readerContext, acceptDocs, true);
|
||||
scorerDoc = -1;
|
||||
if (scorer==null) {
|
||||
noMatches = true;
|
||||
|
|
|
@ -96,6 +96,11 @@ public class BooleanQueryTst {
|
|||
totalMatched++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
void checkNrHits() {
|
||||
Assert.assertEquals(queryText + ": nr of hits", expectedDocNrs.length, totalMatched);
|
||||
}
|
||||
|
|
|
@ -385,7 +385,7 @@ public class TermAutomatonQuery extends Query {
|
|||
}
|
||||
|
||||
@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()];
|
||||
|
|
|
@ -302,6 +302,11 @@ public class TestTermAutomatonQuery extends LuceneTestCase {
|
|||
public void collect(int docID) throws IOException {
|
||||
assertEquals(3, scorer.freq());
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
});
|
||||
|
||||
w.close();
|
||||
|
|
|
@ -1457,6 +1457,29 @@ public abstract class BasePostingsFormatTestCase extends BaseIndexFileFormatTest
|
|||
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 {
|
||||
|
|
|
@ -58,15 +58,15 @@ class AssertingWeight extends Weight {
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
|
|
@ -139,6 +139,11 @@ public class CheckHits {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -507,6 +512,11 @@ public class CheckHits {
|
|||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
base = context.docBase;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -264,7 +264,7 @@ public class QueryUtils {
|
|||
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];
|
||||
|
@ -301,6 +301,11 @@ public class QueryUtils {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
// confirm that skipping beyond the last doc, on the
|
||||
|
@ -311,7 +316,7 @@ public class QueryUtils {
|
|||
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 @@ public class QueryUtils {
|
|||
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 @@ public class QueryUtils {
|
|||
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();
|
||||
|
@ -383,6 +388,11 @@ public class QueryUtils {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
// confirm that skipping beyond the last doc, on the
|
||||
|
@ -392,7 +402,7 @@ public class QueryUtils {
|
|||
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 @@ public class QueryUtils {
|
|||
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 class QueryUtils {
|
|||
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;
|
||||
}
|
||||
|
|
|
@ -178,8 +178,8 @@ public abstract class SearchEquivalenceTestBase extends LuceneTestCase {
|
|||
}
|
||||
|
||||
// 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
|
||||
|
|
|
@ -167,4 +167,9 @@ public class BasicAccumulator extends ValueAccumulator {
|
|||
public long getNumQueries() {
|
||||
return 0l;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO: is this true?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -146,4 +146,8 @@ public class FieldFacetAccumulator extends ValueAccumulator {
|
|||
@Override
|
||||
public NamedList<?> export() { return null; }
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO: is this true?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -66,4 +66,8 @@ public class QueryFacetAccumulator extends ValueAccumulator {
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO: is this true?
|
||||
}
|
||||
}
|
||||
|
|
|
@ -510,6 +510,11 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return true; // TODO: is this always true?
|
||||
}
|
||||
|
||||
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
|
||||
final int docBase = context.docBase;
|
||||
|
||||
|
@ -582,6 +587,11 @@ public class ExpandComponent extends SearchComponent implements PluginInfoInitia
|
|||
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;
|
||||
this.docValues = context.reader().getNumericDocValues(this.field);
|
||||
|
|
|
@ -343,13 +343,13 @@ class SpatialDistanceQuery extends ExtendedQueryBase implements PostFilter {
|
|||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -63,6 +63,11 @@ public class DelegatingCollector extends SimpleCollector {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return delegate.needsScores();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
leafDelegate.collect(doc);
|
||||
|
|
|
@ -83,6 +83,11 @@ public class DocSetCollector extends SimpleCollector {
|
|||
public void setScorer(Scorer scorer) throws IOException {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean needsScores() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doSetNextReader(LeafReaderContext context) throws IOException {
|
||||
this.base = context.docBase;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue