LUCENE-2865: Pass a context struct to Weight#scorer instead of naked booleans

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1058592 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Simon Willnauer 2011-01-13 14:59:11 +00:00
parent 05565c3b29
commit 7913804b28
29 changed files with 153 additions and 71 deletions

View File

@ -134,6 +134,9 @@ Changes in backwards compatibility policy
* LUCENE-2839: Scorer#score(Collector,int,int) is now public because it is
called from other classes and part of public API. (Uwe Schindler)
* LUCENE-2865: Weight#scorer(AtomicReaderContext, boolean, boolean) now accepts
a ScorerContext struct instead of booleans.(Simon Willnauer)
Changes in Runtime Behavior

View File

@ -372,7 +372,7 @@ class BufferedDeletes {
Query query = entry.getKey();
int limit = entry.getValue().intValue();
Weight weight = query.weight(searcher);
Scorer scorer = weight.scorer(readerContext, true, false);
Scorer scorer = weight.scorer(readerContext, Weight.ScorerContext.def());
if (scorer != null) {
while(true) {
int doc = scorer.nextDoc();

View File

@ -226,7 +226,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, true, true) == null) {
if (w.scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true)) == null) {
if (c.isRequired()) {
fail = true;
Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@ -288,7 +288,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer)
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext)
throws IOException {
List<Scorer> required = new ArrayList<Scorer>();
List<Scorer> prohibited = new ArrayList<Scorer>();
@ -296,7 +296,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, true, false);
Scorer subScorer = w.scorer(context, ScorerContext.def());
if (subScorer == null) {
if (c.isRequired()) {
return null;
@ -311,7 +311,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
// Check if we can return a BooleanScorer
if (!scoreDocsInOrder && topScorer && required.size() == 0 && prohibited.size() < 32) {
if (!scorerContext.scoreDocsInOrder && scorerContext.topScorer && required.size() == 0 && prohibited.size() < 32) {
return new BooleanScorer(this, disableCoord, similarity, minNrShouldMatch, optional, prohibited, maxCoord);
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -133,7 +134,7 @@ public class ConstantScoreQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
final DocIdSetIterator disi;
if (filter != null) {
assert query == null;
@ -144,7 +145,7 @@ public class ConstantScoreQuery extends Query {
} else {
assert query != null && innerWeight != null;
disi =
innerWeight.scorer(context, scoreDocsInOrder, topScorer);
innerWeight.scorer(context, scorerContext);
}
if (disi == null)
return null;
@ -158,7 +159,7 @@ public class ConstantScoreQuery extends Query {
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
final Scorer cs = scorer(context, true, false);
final Scorer cs = scorer(context, ScorerContext.def());
final boolean exists = (cs != null && cs.advance(doc) == doc);
final ComplexExplanation result = new ComplexExplanation();

View File

@ -142,12 +142,11 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
/* Create the scorer used to score our associated DisjunctionMaxQuery */
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
Scorer[] scorers = new Scorer[weights.size()];
int idx = 0;
for (Weight w : weights) {
Scorer subScorer = w.scorer(context, true, false);
Scorer subScorer = w.scorer(context, ScorerContext.def());
if (subScorer != null && subScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
scorers[idx++] = subScorer;
}

View File

@ -112,13 +112,13 @@ extends Query {
// return a filtering scorer
@Override
public Scorer scorer(AtomicReaderContext indexReader, boolean scoreDocsInOrder, boolean topScorer)
public Scorer scorer(AtomicReaderContext context, ScorerContext scoreContext)
throws IOException {
final Scorer scorer = weight.scorer(indexReader, true, false);
final Scorer scorer = weight.scorer(context, ScorerContext.def());
if (scorer == null) {
return null;
}
DocIdSet docIdSet = filter.getDocIdSet(indexReader);
DocIdSet docIdSet = filter.getDocIdSet(context);
if (docIdSet == null) {
return null;
}

View File

@ -36,6 +36,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.NIOFSDirectory; // javadoc
import org.apache.lucene.util.ReaderUtil;
@ -496,12 +497,13 @@ public class IndexSearcher {
// TODO: should we make this
// threaded...? the Collector could be sync'd?
ScorerContext scorerContext = ScorerContext.def().scoreDocsInOrder(true).topScorer(true);
// always use single thread:
if (filter == null) {
for (int i = 0; i < leafContexts.length; i++) { // search each subreader
collector.setNextReader(leafContexts[i]);
Scorer scorer = weight.scorer(leafContexts[i], !collector.acceptsDocsOutOfOrder(), true);
scorerContext = scorerContext.scoreDocsInOrder(!collector.acceptsDocsOutOfOrder());
Scorer scorer = weight.scorer(leafContexts[i], scorerContext);
if (scorer != null) {
scorer.score(collector);
}
@ -519,7 +521,7 @@ public class IndexSearcher {
assert filter != null;
Scorer scorer = weight.scorer(context, true, false);
Scorer scorer = weight.scorer(context, ScorerContext.def());
if (scorer == null) {
return;
}

View File

@ -127,7 +127,7 @@ public class MatchAllDocsQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new MatchAllScorer(context.reader, similarity, this,
normsField != null ? context.reader.norms(normsField) : null);
}

View File

@ -25,6 +25,7 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ToStringUtils;
@ -168,7 +169,7 @@ public class MultiPhraseQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
if (termArrays.size() == 0) // optimize zero-term case
return null;
final IndexReader reader = context.reader;
@ -264,7 +265,7 @@ public class MultiPhraseQuery extends Query {
fieldExpl.setDescription("fieldWeight("+getQuery()+" in "+doc+
"), product of:");
Scorer scorer = scorer(context, true, false);
Scorer scorer = scorer(context, ScorerContext.def());
if (scorer == null) {
return new Explanation(0.0f, "no matching docs");
}

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Explanation.IDFExplanation;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
@ -175,7 +176,7 @@ public class PhraseQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
if (terms.size() == 0) // optimize zero-term case
return null;
final IndexReader reader = context.reader;
@ -268,7 +269,7 @@ public class PhraseQuery extends Query {
fieldExpl.setDescription("fieldWeight("+field+":"+query+" in "+doc+
"), product of:");
Scorer scorer = scorer(context, true, false);
Scorer scorer = scorer(context, ScorerContext.def());
if (scorer == null) {
return new Explanation(0.0f, "no matching docs");
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Weight.ScorerContext;
/**
* Constrains search results to only match those which also match a provided
@ -55,7 +56,7 @@ public class QueryWrapperFilter extends Filter {
return new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return weight.scorer(privateContext, true, false);
return weight.scorer(privateContext, ScorerContext.def());
}
@Override
public boolean isCacheable() { return false; }

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Explanation.IDFExplanation;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PerReaderTermState;
import org.apache.lucene.util.ReaderUtil;
@ -86,7 +87,7 @@ public class TermQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
final String field = term.field();
final IndexReader reader = context.reader;
assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight is not the same as the current reader's top-reader";

View File

@ -49,7 +49,7 @@ import org.apache.lucene.index.IndexReader.ReaderContext;
* <li>The query normalization factor is passed to {@link #normalize(float)}. At
* this point the weighting is complete.
* <li>A <code>Scorer</code> is constructed by
* {@link #scorer(AtomicReaderContext,boolean,boolean)}.
* {@link #scorer(AtomicReaderContext, ScorerContext)}.
* </ol>
*
* @since 2.9
@ -89,23 +89,11 @@ public abstract class Weight implements Serializable {
*
* @param context
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
* @param scoreDocsInOrder
* specifies whether in-order scoring of documents is required. Note
* that if set to false (i.e., out-of-order scoring is required),
* this method can return whatever scoring mode it supports, as every
* in-order scorer is also an out-of-order one. However, an
* out-of-order scorer may not support {@link Scorer#nextDoc()}
* and/or {@link Scorer#advance(int)}, therefore it is recommended to
* request an in-order scorer if use of these methods is required.
* @param topScorer
* if true, {@link Scorer#score(Collector)} will be called; if false,
* {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
* be called.
* @param scorercontext the {@link ScorerContext} holding the scores context variables
* @return a {@link Scorer} which scores documents in/out-of order.
* @throws IOException
*/
public abstract Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer) throws IOException;
public abstract Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException;
/** The sum of squared weights of contained query clauses. */
public abstract float sumOfSquaredWeights() throws IOException;
@ -123,4 +111,82 @@ public abstract class Weight implements Serializable {
*/
public boolean scoresDocsOutOfOrder() { return false; }
/**
* A struct like class encapsulating a scorer's context variables.
* ScorerContex is a strictly immutable struct that follows a
* <tt>create on modification</tt> pattern. If a context variable changes
* through one of the modifiers like {@link #topScorer(boolean)} a new
* {@link ScorerContext} instance is creates. If the modifier call doesn't
* change the instance the method call has no effect and the same instance is
* returned from the modifier.
*
* @lucene.experimental
*/
public static final class ScorerContext {
/**
* Specifies whether in-order scoring of documents is required. Note that if
* set to false (i.e., out-of-order scoring is required), this method can
* return whatever scoring mode it supports, as every in-order scorer is
* also an out-of-order one. However, an out-of-order scorer may not support
* {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)}, therefore it
* is recommended to request an in-order scorer if use of these methods is
* required.
*/
public final boolean scoreDocsInOrder;
/**
* if <code>true</code>, {@link Scorer#score(Collector)} will be called; if
* false, {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
* be called instead.
*/
public final boolean topScorer;
private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false);
/**
* Returns a default {@link ScorerContext} template initialized with:
* <ul>
* <li>{@link #scoreDocsInOrder} = <code>true</code></li>
* <li>{@link #topScorer} = <code>false</code></li>
* </ul>
*/
public static ScorerContext def() {
return DEFAULT_CONTEXT;
}
private ScorerContext(boolean scoreDocsInOrder, boolean topScorer) {
this.scoreDocsInOrder = scoreDocsInOrder;
this.topScorer = topScorer;
}
/**
* Creates and returns a copy of this context with the given value for
* {@link #scoreDocsInOrder} and returns a new instance of
* {@link ScorerContext} iff the given value differs from the
* {@link #scoreDocsInOrder}. Otherwise, this method has no effect and
* returns this instance.
*/
public ScorerContext scoreDocsInOrder(boolean scoreDocsInOrder) {
if (this.scoreDocsInOrder == scoreDocsInOrder) {
return this;
}
return new ScorerContext(scoreDocsInOrder, topScorer);
}
/**
* Creates and returns a copy of this context with the given value for
* {@link #topScorer} and returns a new instance of
* {@link ScorerContext} iff the given value differs from the
* {@link #topScorer}. Otherwise, this method has no effect and
* returns this instance.
*/
public ScorerContext topScorer(boolean topScorer) {
if (this.topScorer == topScorer) {
return this;
}
return new ScorerContext(scoreDocsInOrder, topScorer);
}
}
}

View File

@ -31,6 +31,7 @@ import org.apache.lucene.search.Weight;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Similarity;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.ToStringUtils;
/**
@ -240,19 +241,19 @@ public class CustomScoreQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
// Pass true for "scoresDocsInOrder", because we
// require in-order scoring, even if caller does not,
// since we call advance on the valSrcScorers. Pass
// false for "topScorer" because we will not invoke
// score(Collector) on these scorers:
Scorer subQueryScorer = subQueryWeight.scorer(context, true, false);
Scorer subQueryScorer = subQueryWeight.scorer(context, ScorerContext.def());
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, true, topScorer);
valSrcScorers[i] = valSrcWeights[i].scorer(context, scorerContext.scoreDocsInOrder(true));
}
return new CustomScorer(similarity, context.reader, this, subQueryScorer, valSrcScorers);
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search.function;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.*;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.ToStringUtils;
@ -99,7 +100,7 @@ public class ValueSourceQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new ValueSourceScorer(similarity, context, this);
}

View File

@ -143,8 +143,7 @@ public class PayloadNearQuery extends SpanNearQuery {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new PayloadNearSpanScorer(query.getSpans(context.reader), this,
similarity, context.reader.norms(query.getField()));
}

View File

@ -74,8 +74,7 @@ public class PayloadTermQuery extends SpanTermQuery {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new PayloadTermSpanScorer((TermSpans) query.getSpans(context.reader),
this, similarity, context.reader.norms(query.getField()));
}

View File

@ -72,7 +72,7 @@ public class SpanWeight extends Weight {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new SpanScorer(query.getSpans(context.reader), this, similarity, context.reader
.norms(query.getField()));
}
@ -111,7 +111,7 @@ public class SpanWeight extends Weight {
fieldExpl.setDescription("fieldWeight("+field+":"+query.toString(field)+
" in "+doc+"), product of:");
Explanation tfExpl = ((SpanScorer)scorer(context, true, false)).explain(doc);
Explanation tfExpl = ((SpanScorer)scorer(context, ScorerContext.def())).explain(doc);
fieldExpl.addDetail(tfExpl);
fieldExpl.addDetail(idfExpl);

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.util.PriorityQueue;
@ -360,7 +361,7 @@ final class JustCompileSearch {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer)
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext)
throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}

View File

@ -16,6 +16,7 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.store.RAMDirectory;
@ -263,7 +264,7 @@ public class QueryUtils {
try {
if (scorer == null) {
Weight w = q.weight(s);
scorer = w.scorer(readerContextArray[leafPtr], true, false);
scorer = w.scorer(readerContextArray[leafPtr], ScorerContext.def());
}
int op = order[(opidx[0]++) % order.length];
@ -308,7 +309,7 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = new IndexSearcher(previousReader);
Weight w = q.weight(indexSearcher);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def());
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 +334,7 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = new IndexSearcher(previousReader);
Weight w = q.weight(indexSearcher);
Scorer scorer = w.scorer((AtomicReaderContext)previousReader.getTopReaderContext() , true, false);
Scorer scorer = w.scorer((AtomicReaderContext)previousReader.getTopReaderContext(), ScorerContext.def());
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);
@ -363,7 +364,7 @@ public class QueryUtils {
long startMS = System.currentTimeMillis();
for (int i=lastDoc[0]+1; i<=doc; i++) {
Weight w = q.weight(s);
Scorer scorer = w.scorer(context[leafPtr], true, false);
Scorer scorer = w.scorer(context[leafPtr], ScorerContext.def());
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();
@ -390,7 +391,7 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = new IndexSearcher(previousReader);
Weight w = q.weight(indexSearcher);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def());
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 +414,7 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = new IndexSearcher(previousReader);
Weight w = q.weight(indexSearcher);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def());
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);

View File

@ -27,6 +27,7 @@ import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.store.Directory;
import java.text.DecimalFormat;
@ -168,7 +169,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
QueryUtils.check(random, dq, s);
assertTrue(s.getTopReaderContext().isAtomic);
final Weight dw = dq.weight(s);
final Scorer ds = dw.scorer((AtomicReaderContext)s.getTopReaderContext(), true, false);
final Scorer ds = dw.scorer((AtomicReaderContext)s.getTopReaderContext(), ScorerContext.def());
final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
if (skipOk) {
fail("firsttime skipTo found a match? ... "
@ -183,7 +184,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
assertTrue(s.getTopReaderContext().isAtomic);
QueryUtils.check(random, dq, s);
final Weight dw = dq.weight(s);
final Scorer ds = dw.scorer((AtomicReaderContext)s.getTopReaderContext(), true, false);
final Scorer ds = dw.scorer((AtomicReaderContext)s.getTopReaderContext(), ScorerContext.def());
assertTrue("firsttime skipTo found no match",
ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS);
assertEquals("found wrong docid", "d4", r.document(ds.docID()).get("id"));

View File

@ -29,6 +29,7 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.store.Directory;
public class TestTermScorer extends LuceneTestCase {
@ -72,7 +73,7 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = termQuery.weight(indexSearcher);
assertTrue(indexSearcher.getTopReaderContext().isAtomic);
Scorer ts = weight.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, true);
Scorer ts = weight.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def().scoreDocsInOrder(true).topScorer(true));
// we have 2 documents with the term all in them, one document for all the
// other values
final List<TestHit> docs = new ArrayList<TestHit>();
@ -133,7 +134,7 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = termQuery.weight(indexSearcher);
assertTrue(indexSearcher.getTopReaderContext().isAtomic);
Scorer ts = weight.scorer((AtomicReaderContext) indexSearcher.getTopReaderContext(), true, true);
Scorer ts = weight.scorer((AtomicReaderContext) indexSearcher.getTopReaderContext(), ScorerContext.def().scoreDocsInOrder(true).topScorer(true));
assertTrue("next did not return a doc",
ts.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue("score is not correct", ts.score() == 1.6931472f);
@ -152,7 +153,7 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = termQuery.weight(indexSearcher);
assertTrue(indexSearcher.getTopReaderContext().isAtomic);
Scorer ts = weight.scorer((AtomicReaderContext) indexSearcher.getTopReaderContext(), true, true);
Scorer ts = weight.scorer((AtomicReaderContext) indexSearcher.getTopReaderContext(), ScorerContext.def().scoreDocsInOrder(true).topScorer(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);

View File

@ -31,6 +31,7 @@ import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
@ -170,7 +171,7 @@ public class TestNearSpansOrdered extends LuceneTestCase {
SpanNearQuery q = makeQuery();
Weight w = q.weight(searcher);
assertTrue(searcher.getTopReaderContext().isAtomic);
Scorer s = w.scorer((AtomicReaderContext) searcher.getTopReaderContext(), true, false);
Scorer s = w.scorer((AtomicReaderContext) searcher.getTopReaderContext(), ScorerContext.def());
assertEquals(1, s.advance(1));
}
/**

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.DefaultSimilarity;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.store.Directory;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.index.IndexWriter;
@ -421,7 +422,7 @@ public class TestSpans extends LuceneTestCase {
}
};
Scorer spanScorer = snq.weight(searcher).scorer(new AtomicReaderContext(new SlowMultiReaderWrapper(searcher.getIndexReader())), true, false);
Scorer spanScorer = snq.weight(searcher).scorer(new AtomicReaderContext(new SlowMultiReaderWrapper(searcher.getIndexReader())), ScorerContext.def());
assertTrue("first doc", spanScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertEquals("first doc number", spanScorer.docID(), 11);

View File

@ -370,13 +370,13 @@ class SpatialDistanceQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new SpatialScorer(getSimilarity(searcher), context, this);
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
return ((SpatialScorer)scorer(context, true, true)).explain(doc);
return ((SpatialScorer)scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true))).explain(doc);
}
}

View File

@ -90,7 +90,7 @@ public class SolrConstantScoreQuery extends ConstantScoreQuery {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new ConstantScorer(similarity, context, this);
}

View File

@ -91,8 +91,8 @@ public class BoostedQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
Scorer subQueryScorer = qWeight.scorer(context, true, false);
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
Scorer subQueryScorer = qWeight.scorer(context, ScorerContext.def());
if(subQueryScorer == null) {
return null;
}

View File

@ -94,13 +94,13 @@ public class FunctionQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new AllScorer(getSimilarity(searcher), context, this);
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
return ((AllScorer)scorer(context, true, true)).explain(doc);
return ((AllScorer)scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true))).explain(doc);
}
}

View File

@ -20,6 +20,7 @@ package org.apache.solr.search.function;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.*;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.ReaderUtil;
import org.apache.solr.common.SolrException;
@ -111,7 +112,7 @@ class QueryDocValues extends DocValues {
public float floatVal(int doc) {
try {
if (doc < lastDocRequested) {
scorer = weight.scorer(readerContext, true, false);
scorer = weight.scorer(readerContext, ScorerContext.def());
if (scorer==null) return defVal;
scorerDoc = -1;
}