mirror of https://github.com/apache/lucene.git
LUCENE-5487: rename TopScorer -> BulkScorer
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5487@1575234 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
45649d2445
commit
74dd810aa7
|
@ -305,21 +305,21 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
Bits acceptDocs) throws IOException {
|
||||
|
||||
if (scoreDocsInOrder || minNrShouldMatch > 1) {
|
||||
// TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
|
||||
// but the same is even true of pure conjunctions...
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
}
|
||||
|
||||
List<TopScorer> prohibited = new ArrayList<TopScorer>();
|
||||
List<TopScorer> optional = new ArrayList<TopScorer>();
|
||||
List<BulkScorer> prohibited = new ArrayList<BulkScorer>();
|
||||
List<BulkScorer> optional = new ArrayList<BulkScorer>();
|
||||
Iterator<BooleanClause> cIter = clauses.iterator();
|
||||
for (Weight w : weights) {
|
||||
BooleanClause c = cIter.next();
|
||||
TopScorer subScorer = w.topScorer(context, false, acceptDocs);
|
||||
BulkScorer subScorer = w.bulkScorer(context, false, acceptDocs);
|
||||
if (subScorer == null) {
|
||||
if (c.isRequired()) {
|
||||
return null;
|
||||
|
@ -328,7 +328,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
// TODO: there are some cases where BooleanScorer
|
||||
// would handle conjunctions faster than
|
||||
// BooleanScorer2...
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
} else if (c.isProhibited()) {
|
||||
prohibited.add(subScorer);
|
||||
} else {
|
||||
|
|
|
@ -59,7 +59,7 @@ import org.apache.lucene.search.BooleanQuery.BooleanWeight;
|
|||
* conjunction can reduce the number of priority queue
|
||||
* updates for the optional terms. */
|
||||
|
||||
final class BooleanScorer extends TopScorer {
|
||||
final class BooleanScorer extends BulkScorer {
|
||||
|
||||
private static final class BooleanScorerCollector extends Collector {
|
||||
private BucketTable bucketTable;
|
||||
|
@ -190,7 +190,7 @@ final class BooleanScorer extends TopScorer {
|
|||
}
|
||||
|
||||
static final class SubScorer {
|
||||
public TopScorer scorer;
|
||||
public BulkScorer scorer;
|
||||
// TODO: re-enable this if BQ ever sends us required clauses
|
||||
//public boolean required = false;
|
||||
public boolean prohibited;
|
||||
|
@ -198,7 +198,7 @@ final class BooleanScorer extends TopScorer {
|
|||
public SubScorer next;
|
||||
public boolean more;
|
||||
|
||||
public SubScorer(TopScorer scorer, boolean required, boolean prohibited,
|
||||
public SubScorer(BulkScorer scorer, boolean required, boolean prohibited,
|
||||
Collector collector, SubScorer next) {
|
||||
if (required) {
|
||||
throw new IllegalArgumentException("this scorer cannot handle required=true");
|
||||
|
@ -227,15 +227,15 @@ final class BooleanScorer extends TopScorer {
|
|||
private final Weight weight;
|
||||
|
||||
BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
|
||||
List<TopScorer> optionalScorers, List<TopScorer> prohibitedScorers, int maxCoord) throws IOException {
|
||||
List<BulkScorer> optionalScorers, List<BulkScorer> prohibitedScorers, int maxCoord) throws IOException {
|
||||
this.minNrShouldMatch = minNrShouldMatch;
|
||||
this.weight = weight;
|
||||
|
||||
for (TopScorer scorer : optionalScorers) {
|
||||
for (BulkScorer scorer : optionalScorers) {
|
||||
scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers);
|
||||
}
|
||||
|
||||
for (TopScorer scorer : prohibitedScorers) {
|
||||
for (BulkScorer scorer : prohibitedScorers) {
|
||||
scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
|
||||
}
|
||||
|
||||
|
@ -313,11 +313,6 @@ final class BooleanScorer extends TopScorer {
|
|||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
score(collector, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder buffer = new StringBuilder();
|
||||
|
|
|
@ -19,7 +19,14 @@ package org.apache.lucene.search;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
public abstract class TopScorer {
|
||||
/** This class is used to score a range of documents at
|
||||
* once, and is returned by {@link Weight#bulkScorer}. Only
|
||||
* queries that have a more optimized means of scoring
|
||||
* across a range of documents need to override this.
|
||||
* Otherwise, a default implementation is wrapped around
|
||||
* the {@link Scorer} returned by {@link Weight#scorer}. */
|
||||
|
||||
public abstract class BulkScorer {
|
||||
|
||||
/** Scores and collects all matching documents.
|
||||
* @param collector The collector to which all matching documents are passed.
|
||||
|
@ -29,14 +36,10 @@ public abstract class TopScorer {
|
|||
}
|
||||
|
||||
/**
|
||||
* Expert: Collects matching documents in a range. Hook for optimization.
|
||||
* Note, <code>firstDocID</code> is added to ensure that {@link #nextDoc()}
|
||||
* was called before this method.
|
||||
* Collects matching documents in a range.
|
||||
*
|
||||
* @param collector
|
||||
* The collector to which all matching documents are passed.
|
||||
* @param max
|
||||
* Score up to, but not including, this doc
|
||||
* @param collector The collector to which all matching documents are passed.
|
||||
* @param max Score up to, but not including, this doc
|
||||
* @return true if more matching documents may remain.
|
||||
*/
|
||||
public abstract boolean score(Collector collector, int max) throws IOException;
|
|
@ -134,18 +134,18 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
final DocIdSetIterator disi;
|
||||
if (filter != null) {
|
||||
assert query == null;
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
} else {
|
||||
assert query != null && innerWeight != null;
|
||||
TopScorer topScorer = innerWeight.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
if (topScorer == null) {
|
||||
BulkScorer bulkScorer = innerWeight.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
if (bulkScorer == null) {
|
||||
return null;
|
||||
}
|
||||
return new ConstantTopScorer(topScorer, this, queryWeight);
|
||||
return new ConstantBulkScorer(bulkScorer, this, queryWeight);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -196,24 +196,24 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
}
|
||||
|
||||
/** We return this as our {@link TopScorer} so that if the CSQ
|
||||
/** We return this as our {@link BulkScorer} so that if the CSQ
|
||||
* wraps a query with its own optimized top-level
|
||||
* scorer (e.g. BooleanScorer) we can use that
|
||||
* top-level scorer. */
|
||||
protected class ConstantTopScorer extends TopScorer {
|
||||
final TopScorer topScorer;
|
||||
protected class ConstantBulkScorer extends BulkScorer {
|
||||
final BulkScorer bulkScorer;
|
||||
final Weight weight;
|
||||
final float theScore;
|
||||
|
||||
public ConstantTopScorer(TopScorer topScorer, Weight weight, float theScore) {
|
||||
this.topScorer = topScorer;
|
||||
public ConstantBulkScorer(BulkScorer bulkScorer, Weight weight, float theScore) {
|
||||
this.bulkScorer = bulkScorer;
|
||||
this.weight = weight;
|
||||
this.theScore = theScore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max) throws IOException {
|
||||
return topScorer.score(wrapCollector(collector), max);
|
||||
return bulkScorer.score(wrapCollector(collector), max);
|
||||
}
|
||||
|
||||
private Collector wrapCollector(final Collector collector) {
|
||||
|
|
|
@ -138,7 +138,7 @@ public class FilteredQuery extends Query {
|
|||
|
||||
// return a filtering top scorer
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
assert filter != null;
|
||||
|
||||
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
|
||||
|
@ -147,7 +147,7 @@ public class FilteredQuery extends Query {
|
|||
return null;
|
||||
}
|
||||
|
||||
return strategy.filteredTopScorer(context, weight, scoreDocsInOrder, filterDocIdSet);
|
||||
return strategy.filteredBulkScorer(context, weight, scoreDocsInOrder, filterDocIdSet);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -214,12 +214,12 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
}
|
||||
|
||||
private static class QueryFirstTopScorer extends TopScorer {
|
||||
private static class QueryFirstBulkScorer extends BulkScorer {
|
||||
|
||||
private final Scorer scorer;
|
||||
private final Bits filterBits;
|
||||
|
||||
public QueryFirstTopScorer(Scorer scorer, Bits filterBits) {
|
||||
public QueryFirstBulkScorer(Scorer scorer, Bits filterBits) {
|
||||
this.scorer = scorer;
|
||||
this.filterBits = filterBits;
|
||||
}
|
||||
|
@ -324,12 +324,12 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
}
|
||||
|
||||
private static final class LeapFrogTopScorer extends TopScorer {
|
||||
private static final class LeapFrogBulkScorer extends BulkScorer {
|
||||
private final DocIdSetIterator primary;
|
||||
private final DocIdSetIterator secondary;
|
||||
private final Scorer scorer;
|
||||
|
||||
public LeapFrogTopScorer(DocIdSetIterator primary, DocIdSetIterator secondary, Scorer scorer) {
|
||||
public LeapFrogBulkScorer(DocIdSetIterator primary, DocIdSetIterator secondary, Scorer scorer) {
|
||||
this.primary = primary;
|
||||
this.secondary = secondary;
|
||||
this.scorer = scorer;
|
||||
|
@ -530,10 +530,10 @@ public class FilteredQuery extends Query {
|
|||
Weight weight, DocIdSet docIdSet) throws IOException;
|
||||
|
||||
/**
|
||||
* Returns a filtered {@link TopScorer} based on this
|
||||
* Returns a filtered {@link BulkScorer} based on this
|
||||
* strategy. This is an optional method: the default
|
||||
* implementation just calls {@link #filteredScorer} and
|
||||
* wraps that into a TopScorer.
|
||||
* wraps that into a BulkScorer.
|
||||
*
|
||||
* @param context
|
||||
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
|
||||
|
@ -541,7 +541,7 @@ public class FilteredQuery extends Query {
|
|||
* @param docIdSet the filter {@link DocIdSet} to apply
|
||||
* @return a filtered top scorer
|
||||
*/
|
||||
public TopScorer filteredTopScorer(AtomicReaderContext context,
|
||||
public BulkScorer filteredBulkScorer(AtomicReaderContext context,
|
||||
Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
|
||||
Scorer scorer = filteredScorer(context, weight, docIdSet);
|
||||
if (scorer == null) {
|
||||
|
@ -549,7 +549,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
// This impl always scores docs in order, so we can
|
||||
// ignore scoreDocsInOrder:
|
||||
return new Weight.DefaultTopScorer(scorer);
|
||||
return new Weight.DefaultBulkScorer(scorer);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -640,7 +640,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer filteredTopScorer(AtomicReaderContext context,
|
||||
public BulkScorer filteredBulkScorer(AtomicReaderContext context,
|
||||
Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
|
||||
final DocIdSetIterator filterIter = docIdSet.iterator();
|
||||
if (filterIter == null) {
|
||||
|
@ -653,9 +653,9 @@ public class FilteredQuery extends Query {
|
|||
return null;
|
||||
}
|
||||
if (scorerFirst) {
|
||||
return new LeapFrogTopScorer(scorer, filterIter, scorer);
|
||||
return new LeapFrogBulkScorer(scorer, filterIter, scorer);
|
||||
} else {
|
||||
return new LeapFrogTopScorer(filterIter, scorer, scorer);
|
||||
return new LeapFrogBulkScorer(filterIter, scorer, scorer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -690,7 +690,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer filteredTopScorer(final AtomicReaderContext context,
|
||||
public BulkScorer filteredBulkScorer(final AtomicReaderContext context,
|
||||
Weight weight,
|
||||
boolean scoreDocsInOrder, // ignored (we always top-score in order)
|
||||
DocIdSet docIdSet) throws IOException {
|
||||
|
@ -698,10 +698,10 @@ public class FilteredQuery extends Query {
|
|||
if (filterAcceptDocs == null) {
|
||||
// Filter does not provide random-access Bits; we
|
||||
// must fallback to leapfrog:
|
||||
return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredTopScorer(context, weight, scoreDocsInOrder, docIdSet);
|
||||
return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, scoreDocsInOrder, docIdSet);
|
||||
}
|
||||
final Scorer scorer = weight.scorer(context, null);
|
||||
return scorer == null ? null : new QueryFirstTopScorer(scorer, filterAcceptDocs);
|
||||
return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -607,7 +607,7 @@ public class IndexSearcher {
|
|||
// continue with the following leaf
|
||||
continue;
|
||||
}
|
||||
TopScorer scorer = weight.topScorer(ctx, !collector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
|
||||
BulkScorer scorer = weight.bulkScorer(ctx, !collector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
try {
|
||||
scorer.score(collector);
|
||||
|
|
|
@ -36,8 +36,7 @@ import org.apache.lucene.util.Bits;
|
|||
* {@link AtomicReader} dependent state should reside in the {@link Scorer}.
|
||||
* <p>
|
||||
* Since {@link Weight} creates {@link Scorer} instances for a given
|
||||
* {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext,
|
||||
* boolean, boolean, Bits)})
|
||||
* {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, Bits)})
|
||||
* callers must maintain the relationship between the searcher's top-level
|
||||
* {@link IndexReaderContext} and the context used to create a {@link Scorer}.
|
||||
* <p>
|
||||
|
@ -52,7 +51,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(AtomicReaderContext, boolean, boolean, Bits)}.
|
||||
* {@link #scorer(AtomicReaderContext, Bits)}.
|
||||
* </ol>
|
||||
*
|
||||
* @since 2.9
|
||||
|
@ -102,7 +101,7 @@ public abstract class Weight {
|
|||
public abstract Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException;
|
||||
|
||||
/**
|
||||
* Optional method, to return a {@link TopScorer} to
|
||||
* Optional method, to return a {@link BulkScorer} to
|
||||
* score the query and send hits to a {@link Collector}.
|
||||
* Only queries that have a different top-level approach
|
||||
* need to override this; the default implementation
|
||||
|
@ -124,11 +123,11 @@ public abstract class Weight {
|
|||
* Bits that represent the allowable docs to match (typically deleted docs
|
||||
* but possibly filtering other documents)
|
||||
*
|
||||
* @return a {@link TopScorer} which scores documents and
|
||||
* @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 TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
Scorer scorer = scorer(context, acceptDocs);
|
||||
if (scorer == null) {
|
||||
|
@ -138,14 +137,14 @@ public abstract class Weight {
|
|||
|
||||
// This impl always scores docs in order, so we can
|
||||
// ignore scoreDocsInOrder:
|
||||
return new DefaultTopScorer(scorer);
|
||||
return new DefaultBulkScorer(scorer);
|
||||
}
|
||||
|
||||
/** Just wraps a Scorer and performs top scoring using it. */
|
||||
static class DefaultTopScorer extends TopScorer {
|
||||
static class DefaultBulkScorer extends BulkScorer {
|
||||
private final Scorer scorer;
|
||||
|
||||
public DefaultTopScorer(Scorer scorer) {
|
||||
public DefaultBulkScorer(Scorer scorer) {
|
||||
assert scorer != null;
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
@ -174,7 +173,7 @@ public abstract class Weight {
|
|||
* Returns true iff this implementation scores docs only out of order. This
|
||||
* method is used in conjunction with {@link Collector}'s
|
||||
* {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
|
||||
* {@link #scorer(AtomicReaderContext, boolean, boolean, Bits)} to
|
||||
* {@link #scorer(AtomicReaderContext, Bits)} to
|
||||
* create a matching {@link Scorer} instance for a given {@link Collector}, or
|
||||
* vice versa.
|
||||
* <p>
|
||||
|
|
|
@ -53,8 +53,8 @@ org.apache.lucene.search.IndexSearcher#search(Query,Filter,int)}.
|
|||
<p>
|
||||
Once a Query has been created and submitted to the {@link org.apache.lucene.search.IndexSearcher IndexSearcher}, the scoring
|
||||
process begins. After some infrastructure setup, control finally passes to the {@link org.apache.lucene.search.Weight Weight}
|
||||
implementation and its {@link org.apache.lucene.search.Scorer Scorer} instances. See the <a href="#algorithm">Algorithm</a>
|
||||
section for more notes on the process.
|
||||
implementation and its {@link org.apache.lucene.search.Scorer Scorer} or {@link org.apache.lucene.search.BulkScorer BulkScore}
|
||||
instances. See the <a href="#algorithm">Algorithm</a> section for more notes on the process.
|
||||
</p>
|
||||
<!-- FILL IN MORE HERE -->
|
||||
<!-- TODO: this page over-links the same things too many times -->
|
||||
|
@ -370,6 +370,12 @@ on the built-in available scoring models and extending or changing Similarity.
|
|||
{@link org.apache.lucene.search.Scorer Scorer} — An abstract class containing common
|
||||
functionality for scoring. Provides both scoring and
|
||||
explanation capabilities. This is created per-segment.</li>
|
||||
<li>
|
||||
{@link org.apache.lucene.search.BulkScorer BulkScorer} — An abstract class that scores
|
||||
a range of documents. A default implementation simply iterates through the hits from
|
||||
{@link org.apache.lucene.search.Scorer Scorer}, but some queries such as
|
||||
{@link org.apache.lucene.search.BooleanQuery BooleanQuery} have more efficient
|
||||
implementations.</li>
|
||||
</ol>
|
||||
Details on each of these classes, and their children, can be found in the subsections below.
|
||||
</p>
|
||||
|
@ -430,12 +436,18 @@ 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.AtomicReaderContext, boolean, boolean, org.apache.lucene.util.Bits)
|
||||
scorer(AtomicReaderContext context, boolean scoresDocsInOrder, boolean topScorer, Bits acceptDocs)} —
|
||||
{@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, org.apache.lucene.util.Bits)
|
||||
scorer(AtomicReaderContext context, Bits acceptDocs)} —
|
||||
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.AtomicReaderContext, boolean, org.apache.lucene.util.Bits)
|
||||
scorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} —
|
||||
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>
|
||||
<li>
|
||||
{@link org.apache.lucene.search.Weight#explain(org.apache.lucene.index.AtomicReaderContext, int)
|
||||
explain(AtomicReaderContext context, int doc)} — Provide a means for explaining why a given document was
|
||||
|
@ -489,6 +501,18 @@ on the built-in available scoring models and extending or changing Similarity.
|
|||
</li>
|
||||
</ol>
|
||||
</p>
|
||||
<a name="bulkScorerClass"></a>
|
||||
<h4>The BulkScorer Class</h4>
|
||||
<p>The
|
||||
{@link org.apache.lucene.search.BulkScorer BulkScorer} scores a range of documents. There is only one
|
||||
abstract method:
|
||||
<ol>
|
||||
<li>
|
||||
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector,int) score(Collector,int)} —
|
||||
Score all documents up to but not including the specified max document.
|
||||
</li>
|
||||
</ol>
|
||||
</p>
|
||||
<h4>Why would I want to add my own Query?</h4>
|
||||
|
||||
<p>In a nutshell, you want to add your own custom Query implementation when you think that Lucene's
|
||||
|
@ -539,7 +563,7 @@ on the built-in available scoring models and extending or changing Similarity.
|
|||
<p>If a Filter is being used, some initial setup is done to determine which docs to include.
|
||||
Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each
|
||||
{@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling
|
||||
{@link org.apache.lucene.search.Scorer#score(org.apache.lucene.search.Collector) Scorer.score()}.
|
||||
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector) BulkScorer.score(Collector)}.
|
||||
</p>
|
||||
<p>At last, we are actually going to score some documents. The score method takes in the Collector
|
||||
(most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here
|
||||
|
|
|
@ -183,7 +183,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
Weight w = s.createNormalizedWeight(bq);
|
||||
|
||||
assertEquals(1, s.getIndexReader().leaves().size());
|
||||
TopScorer scorer = w.topScorer(s.getIndexReader().leaves().get(0), false, null);
|
||||
BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), false, null);
|
||||
|
||||
final FixedBitSet hits = new FixedBitSet(docCount);
|
||||
final AtomicInteger end = new AtomicInteger();
|
||||
|
|
|
@ -80,7 +80,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
BooleanWeight weight = (BooleanWeight) new BooleanQuery().createWeight(searcher);
|
||||
TopScorer[] scorers = new TopScorer[] {new TopScorer() {
|
||||
BulkScorer[] scorers = new BulkScorer[] {new BulkScorer() {
|
||||
private int doc = -1;
|
||||
|
||||
@Override
|
||||
|
@ -123,7 +123,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
}
|
||||
}};
|
||||
|
||||
BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), Collections.<TopScorer>emptyList(), scorers.length);
|
||||
BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), Collections.<BulkScorer>emptyList(), scorers.length);
|
||||
|
||||
final List<Integer> hits = new ArrayList<Integer>();
|
||||
bs.score(new Collector() {
|
||||
|
@ -243,11 +243,11 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
}
|
||||
|
||||
/** Throws UOE if Weight.scorer is called */
|
||||
private static class CrazyMustUseTopScorerQuery extends Query {
|
||||
private static class CrazyMustUseBulkScorerQuery extends Query {
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return "MustUseTopScorerQuery";
|
||||
return "MustUseBulkScorerQuery";
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -260,7 +260,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
|
||||
@Override
|
||||
public Query getQuery() {
|
||||
return CrazyMustUseTopScorerQuery.this;
|
||||
return CrazyMustUseBulkScorerQuery.this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -278,8 +278,8 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) {
|
||||
return new TopScorer() {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) {
|
||||
return new BulkScorer() {
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max) throws IOException {
|
||||
|
@ -311,7 +311,7 @@ public class TestBooleanScorer extends LuceneTestCase {
|
|||
|
||||
BooleanQuery q2 = new BooleanQuery();
|
||||
q2.add(q1, BooleanClause.Occur.SHOULD);
|
||||
q2.add(new CrazyMustUseTopScorerQuery(), BooleanClause.Occur.SHOULD);
|
||||
q2.add(new CrazyMustUseBulkScorerQuery(), BooleanClause.Occur.SHOULD);
|
||||
|
||||
assertEquals(1, s.search(q2, 10).totalHits);
|
||||
r.close();
|
||||
|
|
|
@ -158,7 +158,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
|
|||
}
|
||||
|
||||
// LUCENE-5307
|
||||
// don't reuse the scorer of filters since they have been created with topScorer=false
|
||||
// don't reuse the scorer of filters since they have been created with bulkScorer=false
|
||||
public void testQueryWrapperFilter() throws IOException {
|
||||
Directory d = newDirectory();
|
||||
RandomIndexWriter w = new RandomIndexWriter(random(), d);
|
||||
|
|
|
@ -78,7 +78,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof AtomicReaderContext);
|
||||
AtomicReaderContext context = (AtomicReaderContext)indexSearcher.getTopReaderContext();
|
||||
TopScorer ts = weight.topScorer(context, true, context.reader().getLiveDocs());
|
||||
BulkScorer ts = weight.bulkScorer(context, true, context.reader().getLiveDocs());
|
||||
// we have 2 documents with the term all in them, one document for all the
|
||||
// other values
|
||||
final List<TestHit> docs = new ArrayList<TestHit>();
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.apache.lucene.search.Filter;
|
|||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.TopScorer;
|
||||
import org.apache.lucene.search.BulkScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
|
@ -124,7 +124,7 @@ class DrillSidewaysQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
// TODO: it could be better if we take acceptDocs
|
||||
// into account instead of baseScorer?
|
||||
|
|
|
@ -26,12 +26,12 @@ import org.apache.lucene.index.DocsEnum;
|
|||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.TopScorer;
|
||||
import org.apache.lucene.search.BulkScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
class DrillSidewaysScorer extends TopScorer {
|
||||
class DrillSidewaysScorer extends BulkScorer {
|
||||
|
||||
//private static boolean DEBUG = false;
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.lucene.search.Explanation;
|
|||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.TopScorer;
|
||||
import org.apache.lucene.search.BulkScorer;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
@ -132,7 +132,7 @@ class TermsIncludingScoreQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
SVInnerScorer scorer = (SVInnerScorer) topScorer(context, false, null);
|
||||
SVInnerScorer scorer = (SVInnerScorer) bulkScorer(context, false, null);
|
||||
if (scorer != null) {
|
||||
return scorer.explain(doc);
|
||||
}
|
||||
|
@ -180,10 +180,10 @@ class TermsIncludingScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
if (scoreDocsInOrder) {
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
} else {
|
||||
Terms terms = context.reader().terms(field);
|
||||
if (terms == null) {
|
||||
|
@ -245,7 +245,7 @@ class TermsIncludingScoreQuery extends Query {
|
|||
}
|
||||
|
||||
// This impl assumes that the 'join' values are used uniquely per doc per field. Used for one to many relations.
|
||||
class SVInnerScorer extends TopScorer {
|
||||
class SVInnerScorer extends BulkScorer {
|
||||
|
||||
final BytesRef spare = new BytesRef();
|
||||
final Bits acceptDocs;
|
||||
|
|
|
@ -154,7 +154,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
}
|
||||
|
||||
/** LUCENE-5487: verify a join query inside a SHOULD BQ
|
||||
* will still use the join query's optimized TopScorers */
|
||||
* will still use the join query's optimized BulkScorers */
|
||||
public void testInsideBooleanQuery() throws Exception {
|
||||
final String idField = "id";
|
||||
final String toField = "productId";
|
||||
|
@ -235,7 +235,7 @@ public class TestJoinUtil extends LuceneTestCase {
|
|||
if (docID == 5) {
|
||||
sawFive = true;
|
||||
} else if (docID == 1) {
|
||||
assertFalse("optimized topScorer was not used for join query embedded in boolean query!", sawFive);
|
||||
assertFalse("optimized bulkScorer was not used for join query embedded in boolean query!", sawFive);
|
||||
}
|
||||
}
|
||||
@Override
|
||||
|
|
|
@ -29,38 +29,38 @@ import org.apache.lucene.index.DocsEnum;
|
|||
import org.apache.lucene.util.VirtualMethod;
|
||||
|
||||
/** Wraps a Scorer with additional checks */
|
||||
public class AssertingTopScorer extends TopScorer {
|
||||
public class AssertingBulkScorer extends BulkScorer {
|
||||
|
||||
private static final VirtualMethod<TopScorer> SCORE_COLLECTOR = new VirtualMethod<TopScorer>(TopScorer.class, "score", Collector.class);
|
||||
private static final VirtualMethod<TopScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<TopScorer>(TopScorer.class, "score", Collector.class, int.class);
|
||||
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", Collector.class);
|
||||
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", Collector.class, int.class);
|
||||
|
||||
// we need to track scorers using a weak hash map because otherwise we
|
||||
// could loose references because of eg.
|
||||
// AssertingScorer.score(Collector) which needs to delegate to work correctly
|
||||
private static Map<TopScorer, WeakReference<AssertingTopScorer>> ASSERTING_INSTANCES = Collections.synchronizedMap(new WeakHashMap<TopScorer, WeakReference<AssertingTopScorer>>());
|
||||
private static Map<BulkScorer, WeakReference<AssertingBulkScorer>> ASSERTING_INSTANCES = Collections.synchronizedMap(new WeakHashMap<BulkScorer, WeakReference<AssertingBulkScorer>>());
|
||||
|
||||
public static TopScorer wrap(Random random, TopScorer other) {
|
||||
if (other == null || other instanceof AssertingTopScorer) {
|
||||
public static BulkScorer wrap(Random random, BulkScorer other) {
|
||||
if (other == null || other instanceof AssertingBulkScorer) {
|
||||
return other;
|
||||
}
|
||||
final AssertingTopScorer assertScorer = new AssertingTopScorer(random, other);
|
||||
ASSERTING_INSTANCES.put(other, new WeakReference<AssertingTopScorer>(assertScorer));
|
||||
final AssertingBulkScorer assertScorer = new AssertingBulkScorer(random, other);
|
||||
ASSERTING_INSTANCES.put(other, new WeakReference<AssertingBulkScorer>(assertScorer));
|
||||
return assertScorer;
|
||||
}
|
||||
|
||||
public static boolean shouldWrap(TopScorer inScorer) {
|
||||
public static boolean shouldWrap(BulkScorer inScorer) {
|
||||
return SCORE_COLLECTOR.isOverriddenAsOf(inScorer.getClass()) || SCORE_COLLECTOR_RANGE.isOverriddenAsOf(inScorer.getClass());
|
||||
}
|
||||
|
||||
final Random random;
|
||||
final TopScorer in;
|
||||
final BulkScorer in;
|
||||
|
||||
private AssertingTopScorer(Random random, TopScorer in) {
|
||||
private AssertingBulkScorer(Random random, BulkScorer in) {
|
||||
this.random = random;
|
||||
this.in = in;
|
||||
}
|
||||
|
||||
public TopScorer getIn() {
|
||||
public BulkScorer getIn() {
|
||||
return in;
|
||||
}
|
||||
|
||||
|
@ -85,6 +85,6 @@ public class AssertingTopScorer extends TopScorer {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "AssertingTopScorer(" + in + ")";
|
||||
return "AssertingBulkScorer(" + in + ")";
|
||||
}
|
||||
}
|
|
@ -66,19 +66,19 @@ class AssertingWeight extends Weight {
|
|||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
// if the caller asks for in-order scoring or if the weight does not support
|
||||
// out-of order scoring then collection will have to happen in-order.
|
||||
TopScorer inScorer = in.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
BulkScorer inScorer = in.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
if (inScorer == null) {
|
||||
return null;
|
||||
}
|
||||
if (AssertingTopScorer.shouldWrap(inScorer)) {
|
||||
return AssertingTopScorer.wrap(new Random(random.nextLong()), inScorer);
|
||||
if (AssertingBulkScorer.shouldWrap(inScorer)) {
|
||||
return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
|
||||
} else {
|
||||
// Let super wrap this.scorer instead, so we use
|
||||
// AssertingScorer:
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue