LUCENE-3474: pass Bits to weight.scorer instead of weights pulling their own from the reader

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1177879 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2011-10-01 01:03:35 +00:00
parent 981a297488
commit b4ff06f88e
45 changed files with 205 additions and 216 deletions

View File

@ -41,6 +41,7 @@ import org.apache.lucene.search.spans.SpanOrQuery;
import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.Spans;
import org.apache.lucene.util.Bits;
/**
* Class used to extract {@link WeightedSpanTerm}s from a {@link Query} based on whether
@ -248,11 +249,12 @@ public class WeightedSpanTermExtractor {
for (final String field : fieldNames) {
AtomicReaderContext context = getLeafContextForField(field);
Bits acceptDocs = context.reader.getLiveDocs();
final Spans spans;
if (mustRewriteQuery) {
spans = queries.get(field).getSpans(context);
spans = queries.get(field).getSpans(context, acceptDocs);
} else {
spans = spanQuery.getSpans(context);
spans = spanQuery.getSpans(context, acceptDocs);
}

View File

@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.ConjunctionTermScorer.DocsAndFreqs;
@ -238,7 +239,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, ScorerContext.def().scoreDocsInOrder(true).topScorer(true)) == null) {
if (w.scorer(context, true, true, context.reader.getLiveDocs()) == null) {
if (c.isRequired()) {
fail = true;
Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@ -300,11 +301,12 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext)
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs)
throws IOException {
if (termConjunction) {
// specialized scorer for term conjunctions
return createConjunctionTermScorer(context);
return createConjunctionTermScorer(context, acceptDocs);
}
List<Scorer> required = new ArrayList<Scorer>();
List<Scorer> prohibited = new ArrayList<Scorer>();
@ -312,7 +314,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, ScorerContext.def());
Scorer subScorer = w.scorer(context, true, false, acceptDocs);
if (subScorer == null) {
if (c.isRequired()) {
return null;
@ -327,7 +329,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
// Check if we can return a BooleanScorer
if (!scorerContext.scoreDocsInOrder && scorerContext.topScorer && required.size() == 0 && prohibited.size() < 32) {
if (!scoreDocsInOrder && topScorer && required.size() == 0 && prohibited.size() < 32) {
return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
}
@ -345,7 +347,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
return new BooleanScorer2(this, disableCoord, minNrShouldMatch, required, prohibited, optional, maxCoord);
}
private Scorer createConjunctionTermScorer(AtomicReaderContext context)
private Scorer createConjunctionTermScorer(AtomicReaderContext context, Bits acceptDocs)
throws IOException {
final DocsAndFreqs[] docsAndFreqs = new DocsAndFreqs[weights.size()];
for (int i = 0; i < docsAndFreqs.length; i++) {
@ -356,7 +358,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
}
final ExactDocScorer docScorer = weight.createDocScorer(context);
docsAndFreqs[i] = new DocsAndFreqs(termsEnum.docs(
context.reader.getLiveDocs(), null), termsEnum.docFreq(), docScorer);
acceptDocs, null), termsEnum.docFreq(), docScorer);
}
return new ConjunctionTermScorer(this, disableCoord ? 1.0f : coord(
docsAndFreqs.length, docsAndFreqs.length), docsAndFreqs);

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.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -126,7 +127,8 @@ public class ConstantScoreQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
final DocIdSetIterator disi;
if (filter != null) {
assert query == null;
@ -137,7 +139,7 @@ public class ConstantScoreQuery extends Query {
disi = dis.iterator();
} else {
assert query != null && innerWeight != null;
disi = innerWeight.scorer(context, scorerContext);
disi = innerWeight.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
}
if (disi == null) {
@ -153,7 +155,7 @@ public class ConstantScoreQuery extends Query {
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
final Scorer cs = scorer(context, ScorerContext.def());
final Scorer cs = scorer(context, true, false, context.reader.getLiveDocs());
final boolean exists = (cs != null && cs.advance(doc) == doc);
final ComplexExplanation result = new ComplexExplanation();

View File

@ -25,6 +25,7 @@ import java.util.Set;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Bits;
/**
* A query that generates the union of documents produced by its subqueries, and that scores each document with the maximum
@ -135,11 +136,13 @@ public class DisjunctionMaxQuery extends Query implements Iterable<Query> {
/** Create the scorer used to score our associated DisjunctionMaxQuery */
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
Scorer[] scorers = new Scorer[weights.size()];
int idx = 0;
for (Weight w : weights) {
Scorer subScorer = w.scorer(context, ScorerContext.def());
// we will advance() subscorers
Scorer subScorer = w.scorer(context, true, false, acceptDocs);
if (subScorer != null && subScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
scorers[idx++] = subScorer;
}

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.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -105,9 +106,11 @@ extends Query {
// return a filtering scorer
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scoreContext)
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs)
throws IOException {
final Scorer scorer = weight.scorer(context, ScorerContext.def());
// we will advance() the subscorer
final Scorer scorer = weight.scorer(context, true, false, acceptDocs);
if (scorer == null) {
return null;
}

View File

@ -37,7 +37,6 @@ import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
import org.apache.lucene.search.similarities.SimilarityProvider;
import org.apache.lucene.store.Directory;
@ -569,13 +568,11 @@ public class IndexSearcher implements Closeable {
// 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 < leaves.length; i++) { // search each subreader
collector.setNextReader(leaves[i]);
scorerContext = scorerContext.scoreDocsInOrder(!collector.acceptsDocsOutOfOrder());
Scorer scorer = weight.scorer(leaves[i], scorerContext);
Scorer scorer = weight.scorer(leaves[i], !collector.acceptsDocsOutOfOrder(), true, leaves[i].reader.getLiveDocs());
if (scorer != null) {
scorer.score(collector);
}
@ -593,7 +590,8 @@ public class IndexSearcher implements Closeable {
assert filter != null;
Scorer scorer = weight.scorer(context, ScorerContext.def());
// we are gonna advance() this scorer, so we set inorder=true/toplevel=false
Scorer scorer = weight.scorer(context, true, false, context.reader.getLiveDocs());
if (scorer == null) {
return;
}

View File

@ -38,9 +38,9 @@ public class MatchAllDocsQuery extends Query {
private final int maxDoc;
private final Bits liveDocs;
MatchAllScorer(IndexReader reader, Weight w, float score) throws IOException {
MatchAllScorer(IndexReader reader, Bits liveDocs, Weight w, float score) throws IOException {
super(w);
liveDocs = reader.getLiveDocs();
this.liveDocs = liveDocs;
this.score = score;
maxDoc = reader.maxDoc();
}
@ -104,8 +104,9 @@ public class MatchAllDocsQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new MatchAllScorer(context.reader, this, queryWeight);
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new MatchAllScorer(context.reader, acceptDocs, this, queryWeight);
}
@Override

View File

@ -164,10 +164,11 @@ public class MultiPhraseQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
assert !termArrays.isEmpty();
final IndexReader reader = context.reader;
final Bits liveDocs = reader.getLiveDocs();
final Bits liveDocs = acceptDocs;
PhraseQuery.PostingsAndFreq[] postingsFreqs = new PhraseQuery.PostingsAndFreq[termArrays.size()];
@ -227,7 +228,7 @@ public class MultiPhraseQuery extends Query {
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, ScorerContext.def());
Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -212,10 +212,11 @@ public class PhraseQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
assert !terms.isEmpty();
final IndexReader reader = context.reader;
final Bits liveDocs = reader.getLiveDocs();
final Bits liveDocs = acceptDocs;
PostingsAndFreq[] postingsFreqs = new PostingsAndFreq[terms.size()];
for (int i = 0; i < terms.size(); i++) {
final Term t = terms.get(i);
@ -267,7 +268,7 @@ public class PhraseQuery extends Query {
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, ScorerContext.def());
Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -20,7 +20,6 @@ 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
@ -56,7 +55,7 @@ public class QueryWrapperFilter extends Filter {
return new DocIdSet() {
@Override
public DocIdSetIterator iterator() throws IOException {
return weight.scorer(privateContext, ScorerContext.def());
return weight.scorer(privateContext, true, false, privateContext.reader.getLiveDocs());
}
@Override
public boolean isCacheable() { return false; }

View File

@ -61,7 +61,7 @@ public class SpanQueryFilter extends SpanFilter {
public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
final FixedBitSet bits = new FixedBitSet(context.reader.maxDoc());
Spans spans = query.getSpans(context);
Spans spans = query.getSpans(context, context.reader.getLiveDocs());
List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
int currentDoc = -1;
SpanFilterResult.PositionInfo currentInfo = null;

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.search.similarities.Similarity.ExactDocScorer;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TermContext;
import org.apache.lucene.util.ReaderUtil;
@ -73,14 +74,15 @@ public class TermQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) 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;
}
// TODO should we reuse the DocsEnum here?
final DocsEnum docs = termsEnum.docs(context.reader.getLiveDocs(), null);
final DocsEnum docs = termsEnum.docs(acceptDocs, null);
assert docs != null;
return new TermScorer(this, docs, createDocScorer(context));
}

View File

@ -23,6 +23,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.search.similarities.SimilarityProvider;
import org.apache.lucene.util.Bits;
/**
* Expert: Calculate query weights and build query scorers.
@ -34,7 +35,8 @@ import org.apache.lucene.search.similarities.SimilarityProvider;
* {@link IndexReader} dependent state should reside in the {@link Scorer}.
* <p>
* Since {@link Weight} creates {@link Scorer} instances for a given
* {@link AtomicReaderContext} ({@link #scorer(IndexReader.AtomicReaderContext, ScorerContext)})
* {@link AtomicReaderContext} ({@link #scorer(IndexReader.AtomicReaderContext,
* boolean, boolean, Bits)})
* callers must maintain the relationship between the searcher's top-level
* {@link ReaderContext} and the context used to create a {@link Scorer}.
* <p>
@ -49,7 +51,7 @@ import org.apache.lucene.search.similarities.SimilarityProvider;
* <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(IndexReader.AtomicReaderContext, ScorerContext)}.
* {@link #scorer(IndexReader.AtomicReaderContext, boolean, boolean, Bits)}.
* </ol>
*
* @since 2.9
@ -89,17 +91,33 @@ public abstract class Weight {
*
* @param context
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
* @param scorerContext the {@link ScorerContext} holding the scores context variables
* @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 acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
*
* @return a {@link Scorer} which scores documents in/out-of order.
* @throws IOException
*/
public abstract Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException;
public abstract Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException;
/**
* 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(IndexReader.AtomicReaderContext, ScorerContext)} to
* {@link #scorer(IndexReader.AtomicReaderContext, boolean, boolean, Bits)} to
* create a matching {@link Scorer} instance for a given {@link Collector}, or
* vice versa.
* <p>
@ -107,83 +125,4 @@ public abstract class Weight {
* the <code>Scorer</code> scores documents in-order.
*/
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

@ -33,6 +33,7 @@ import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanScorer;
import org.apache.lucene.search.spans.SpanWeight;
import org.apache.lucene.search.spans.Spans;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ToStringUtils;
@ -147,14 +148,15 @@ public class PayloadNearQuery extends SpanNearQuery {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new PayloadNearSpanScorer(query.getSpans(context), this,
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs), this,
similarity, similarity.sloppyDocScorer(stats, query.getField(), context));
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, ScorerContext.def());
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, true, false, context.reader.getLiveDocs());
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -176,7 +176,7 @@ public class PayloadSpanUtil {
throws IOException {
final AtomicReaderContext[] leaves = ReaderUtil.leaves(context);
for (AtomicReaderContext atomicReaderContext : leaves) {
final Spans spans = query.getSpans(atomicReaderContext);
final Spans spans = query.getSpans(atomicReaderContext, atomicReaderContext.reader.getLiveDocs());
while (spans.next() == true) {
if (spans.isPayloadAvailable()) {
Collection<byte[]> payload = spans.getPayload();

View File

@ -25,7 +25,6 @@ import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.ComplexExplanation;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.search.payloads.PayloadNearQuery.PayloadNearSpanScorer;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.Similarity;
@ -34,6 +33,7 @@ import org.apache.lucene.search.spans.TermSpans;
import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.search.spans.SpanWeight;
import org.apache.lucene.search.spans.SpanScorer;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
@ -79,8 +79,9 @@ public class PayloadTermQuery extends SpanTermQuery {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new PayloadTermSpanScorer((TermSpans) query.getSpans(context),
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs),
this, similarity.sloppyDocScorer(stats, query.getField(), context));
}
@ -174,7 +175,7 @@ public class PayloadTermQuery extends SpanTermQuery {
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, ScorerContext.def());
PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, true, false, context.reader.getLiveDocs());
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
/**
@ -92,8 +93,8 @@ public class FieldMaskingSpanQuery extends SpanQuery {
// ...this is done to be more consistent with things like SpanFirstQuery
@Override
public Spans getSpans(AtomicReaderContext context) throws IOException {
return maskedQuery.getSpans(context);
public Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return maskedQuery.getSpans(context, acceptDocs);
}
@Override

View File

@ -19,6 +19,7 @@ package org.apache.lucene.search.spans;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import java.io.IOException;
import java.util.ArrayList;
@ -77,11 +78,11 @@ public class NearSpansOrdered extends Spans {
private SpanNearQuery query;
private boolean collectPayloads = true;
public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context) throws IOException {
this(spanNearQuery, context, true);
public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, Bits acceptDocs) throws IOException {
this(spanNearQuery, context, acceptDocs, true);
}
public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, boolean collectPayloads)
public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, Bits acceptDocs, boolean collectPayloads)
throws IOException {
if (spanNearQuery.getClauses().length < 2) {
throw new IllegalArgumentException("Less than 2 clauses: "
@ -94,7 +95,7 @@ public class NearSpansOrdered extends Spans {
matchPayload = new LinkedList<byte[]>();
subSpansByDoc = new Spans[clauses.length];
for (int i = 0; i < clauses.length; i++) {
subSpans[i] = clauses[i].getSpans(context);
subSpans[i] = clauses[i].getSpans(context, acceptDocs);
subSpansByDoc[i] = subSpans[i]; // used in toSameDoc()
}
query = spanNearQuery; // kept for toString() only.

View File

@ -18,6 +18,7 @@ package org.apache.lucene.search.spans;
*/
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
import java.io.IOException;
@ -131,7 +132,7 @@ public class NearSpansUnordered extends Spans {
}
public NearSpansUnordered(SpanNearQuery query, AtomicReaderContext context)
public NearSpansUnordered(SpanNearQuery query, AtomicReaderContext context, Bits acceptDocs)
throws IOException {
this.query = query;
this.slop = query.getSlop();
@ -141,7 +142,7 @@ public class NearSpansUnordered extends Spans {
subSpans = new Spans[clauses.length];
for (int i = 0; i < clauses.length; i++) {
SpansCell cell =
new SpansCell(clauses[i].getSpans(context), i);
new SpansCell(clauses[i].getSpans(context, acceptDocs), i);
ordered.add(cell);
subSpans[i] = cell.spans;
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.TopTermsRewrite;
import org.apache.lucene.search.ScoringRewrite;
import org.apache.lucene.search.BooleanClause.Occur; // javadocs only
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.TermContext;
/**
@ -89,7 +90,7 @@ public class SpanMultiTermQueryWrapper<Q extends MultiTermQuery> extends SpanQue
}
@Override
public Spans getSpans(AtomicReaderContext context) throws IOException {
public Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
throw new UnsupportedOperationException("Query should have been rewritten");
}

View File

@ -30,6 +30,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
/** Matches spans which are near one another. One can specify <i>slop</i>, the
@ -117,16 +118,16 @@ public class SpanNearQuery extends SpanQuery implements Cloneable {
}
@Override
public Spans getSpans(final AtomicReaderContext context) throws IOException {
public Spans getSpans(final AtomicReaderContext context, Bits acceptDocs) throws IOException {
if (clauses.size() == 0) // optimize 0-clause case
return new SpanOrQuery(getClauses()).getSpans(context);
return new SpanOrQuery(getClauses()).getSpans(context, acceptDocs);
if (clauses.size() == 1) // optimize 1-clause case
return clauses.get(0).getSpans(context);
return clauses.get(0).getSpans(context, acceptDocs);
return inOrder
? (Spans) new NearSpansOrdered(this, context, collectPayloads)
: (Spans) new NearSpansUnordered(this, context);
? (Spans) new NearSpansOrdered(this, context, acceptDocs, collectPayloads)
: (Spans) new NearSpansUnordered(this, context, acceptDocs);
}
@Override

View File

@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -75,12 +76,12 @@ public class SpanNotQuery extends SpanQuery implements Cloneable {
}
@Override
public Spans getSpans(final AtomicReaderContext context) throws IOException {
public Spans getSpans(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
return new Spans() {
private Spans includeSpans = include.getSpans(context);
private Spans includeSpans = include.getSpans(context, acceptDocs);
private boolean moreInclude = true;
private Spans excludeSpans = exclude.getSpans(context);
private Spans excludeSpans = exclude.getSpans(context, acceptDocs);
private boolean moreExclude = excludeSpans.next();
@Override

View File

@ -28,6 +28,7 @@ import java.util.Set;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.ToStringUtils;
import org.apache.lucene.search.Query;
@ -163,9 +164,9 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
}
@Override
public Spans getSpans(final AtomicReaderContext context) throws IOException {
public Spans getSpans(final AtomicReaderContext context, final Bits acceptDocs) throws IOException {
if (clauses.size() == 1) // optimize 1-clause case
return (clauses.get(0)).getSpans(context);
return (clauses.get(0)).getSpans(context, acceptDocs);
return new Spans() {
private SpanQueue queue = null;
@ -174,7 +175,7 @@ public class SpanOrQuery extends SpanQuery implements Cloneable {
queue = new SpanQueue(clauses.size());
Iterator<SpanQuery> i = clauses.iterator();
while (i.hasNext()) {
Spans spans = i.next().getSpans(context);
Spans spans = i.next().getSpans(context, acceptDocs);
if ( ((target == -1) && spans.next())
|| ((target != -1) && spans.skipTo(target))) {
queue.add(spans);

View File

@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Query;
import org.apache.lucene.util.Bits;
import java.io.IOException;
import java.util.ArrayList;
@ -81,8 +82,8 @@ public abstract class SpanPositionCheckQuery extends SpanQuery implements Clonea
protected abstract AcceptStatus acceptPosition(Spans spans) throws IOException;
@Override
public Spans getSpans(final AtomicReaderContext context) throws IOException {
return new PositionCheckSpan(context);
public Spans getSpans(final AtomicReaderContext context, Bits acceptDocs) throws IOException {
return new PositionCheckSpan(context, acceptDocs);
}
@ -106,8 +107,8 @@ public abstract class SpanPositionCheckQuery extends SpanQuery implements Clonea
protected class PositionCheckSpan extends Spans {
private Spans spans;
public PositionCheckSpan(AtomicReaderContext context) throws IOException {
spans = match.getSpans(context);
public PositionCheckSpan(AtomicReaderContext context, Bits acceptDocs) throws IOException {
spans = match.getSpans(context, acceptDocs);
}
@Override

View File

@ -23,12 +23,13 @@ import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.Bits;
/** Base class for span-based queries. */
public abstract class SpanQuery extends Query {
/** Expert: Returns the matches for this query in an index. Used internally
* to search for spans. */
public abstract Spans getSpans(AtomicReaderContext context) throws IOException;
public abstract Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException;
/** Returns the name of the field matched by this query.*/
public abstract String getField();

View File

@ -21,6 +21,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -81,9 +82,9 @@ public class SpanTermQuery extends SpanQuery {
}
@Override
public Spans getSpans(final AtomicReaderContext context) throws IOException {
public Spans getSpans(final AtomicReaderContext context, Bits acceptDocs) throws IOException {
final IndexReader reader = context.reader;
final DocsAndPositionsEnum postings = reader.termPositionsEnum(reader.getLiveDocs(),
final DocsAndPositionsEnum postings = reader.termPositionsEnum(acceptDocs,
term.field(),
term.bytes());

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.*;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.search.similarities.Similarity.SloppyDocScorer;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.TermContext;
import java.io.IOException;
@ -67,13 +68,14 @@ public class SpanWeight extends Weight {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new SpanScorer(query.getSpans(context), this, similarity.sloppyDocScorer(stats, query.getField(), context));
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new SpanScorer(query.getSpans(context, acceptDocs), this, similarity.sloppyDocScorer(stats, query.getField(), context));
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, ScorerContext.def());
Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
if (scorer != null) {
int newDoc = scorer.advance(doc);
if (newDoc == doc) {

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.index.IndexReader.ReaderContext;
import org.apache.lucene.util.Bits;
/**
* Helper class that adds some extra checks to ensure correct
@ -67,8 +68,9 @@ public class AssertingIndexSearcher extends IndexSearcher {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return w.scorer(context, scorerContext);
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return w.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
}
@Override

View File

@ -29,10 +29,10 @@ 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;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util._TestUtil;
@ -256,7 +256,8 @@ public class QueryUtils {
try {
if (scorer == null) {
Weight w = s.createNormalizedWeight(q);
scorer = w.scorer(readerContextArray[leafPtr], ScorerContext.def());
AtomicReaderContext context = readerContextArray[leafPtr];
scorer = w.scorer(context, true, false, context.reader.getLiveDocs());
}
int op = order[(opidx[0]++) % order.length];
@ -301,7 +302,8 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
Weight w = indexSearcher.createNormalizedWeight(q);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def());
AtomicReaderContext ctx = (AtomicReaderContext)indexSearcher.getTopReaderContext();
Scorer scorer = w.scorer(ctx, true, false, ctx.reader.getLiveDocs());
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);
@ -327,7 +329,8 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader, false);
Weight w = indexSearcher.createNormalizedWeight(q);
Scorer scorer = w.scorer((AtomicReaderContext)previousReader.getTopReaderContext(), ScorerContext.def());
AtomicReaderContext ctx = (AtomicReaderContext)previousReader.getTopReaderContext();
Scorer scorer = w.scorer(ctx, true, false, ctx.reader.getLiveDocs());
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);
@ -347,6 +350,7 @@ public class QueryUtils {
s.search(q,new Collector() {
private Scorer scorer;
private int leafPtr;
private Bits liveDocs;
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer = scorer;
@ -358,7 +362,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[leafPtr], ScorerContext.def());
Scorer scorer = w.scorer(context[leafPtr], true, false, liveDocs);
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();
@ -385,7 +389,7 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
Weight w = indexSearcher.createNormalizedWeight(q);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def());
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, previousReader.getLiveDocs());
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);
@ -396,6 +400,7 @@ public class QueryUtils {
lastReader[0] = context.reader;
lastDoc[0] = -1;
liveDocs = context.reader.getLiveDocs();
}
@Override
public boolean acceptsDocsOutOfOrder() {
@ -409,7 +414,7 @@ public class QueryUtils {
final IndexReader previousReader = lastReader[0];
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
Weight w = indexSearcher.createNormalizedWeight(q);
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def());
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, previousReader.getLiveDocs());
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

@ -25,6 +25,7 @@ import org.apache.lucene.search.similarities.SimilarityProvider;
import org.apache.lucene.search.similarities.Similarity.ExactDocScorer;
import org.apache.lucene.search.similarities.Similarity.SloppyDocScorer;
import org.apache.lucene.search.similarities.Similarity.Stats;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.TermContext;
import org.apache.lucene.index.FieldInvertState;
@ -364,7 +365,8 @@ final class JustCompileSearch {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext)
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs)
throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}

View File

@ -28,7 +28,6 @@ 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.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
import org.apache.lucene.search.similarities.Similarity;
@ -178,7 +177,8 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
QueryUtils.check(random, dq, s);
assertTrue(s.getTopReaderContext().isAtomic);
final Weight dw = s.createNormalizedWeight(dq);
final Scorer ds = dw.scorer((AtomicReaderContext)s.getTopReaderContext(), ScorerContext.def());
AtomicReaderContext context = (AtomicReaderContext)s.getTopReaderContext();
final Scorer ds = dw.scorer(context, true, false, context.reader.getLiveDocs());
final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
if (skipOk) {
fail("firsttime skipTo found a match? ... "
@ -193,7 +193,8 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
assertTrue(s.getTopReaderContext().isAtomic);
QueryUtils.check(random, dq, s);
final Weight dw = s.createNormalizedWeight(dq);
final Scorer ds = dw.scorer((AtomicReaderContext)s.getTopReaderContext(), ScorerContext.def());
AtomicReaderContext context = (AtomicReaderContext)s.getTopReaderContext();
final Scorer ds = dw.scorer(context, true, false, context.reader.getLiveDocs());
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,7 +29,6 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SlowMultiReaderWrapper;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
@ -79,7 +78,8 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
assertTrue(indexSearcher.getTopReaderContext().isAtomic);
Scorer ts = weight.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), ScorerContext.def().scoreDocsInOrder(true).topScorer(true));
AtomicReaderContext context = (AtomicReaderContext)indexSearcher.getTopReaderContext();
Scorer ts = weight.scorer(context, true, 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>();
@ -140,7 +140,8 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
assertTrue(indexSearcher.getTopReaderContext().isAtomic);
Scorer ts = weight.scorer((AtomicReaderContext) indexSearcher.getTopReaderContext(), ScorerContext.def().scoreDocsInOrder(true).topScorer(true));
AtomicReaderContext context = (AtomicReaderContext) indexSearcher.getTopReaderContext();
Scorer ts = weight.scorer(context, true, true, context.reader.getLiveDocs());
assertTrue("next did not return a doc",
ts.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
assertTrue("score is not correct", ts.score() == 1.6931472f);
@ -158,8 +159,8 @@ public class TestTermScorer extends LuceneTestCase {
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
assertTrue(indexSearcher.getTopReaderContext().isAtomic);
Scorer ts = weight.scorer((AtomicReaderContext) indexSearcher.getTopReaderContext(), ScorerContext.def().scoreDocsInOrder(true).topScorer(true));
AtomicReaderContext context = (AtomicReaderContext) indexSearcher.getTopReaderContext();
Scorer ts = weight.scorer(context, true, true, context.reader.getLiveDocs());
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

@ -23,6 +23,7 @@ import java.util.Collection;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.Bits;
/**
* Holds all implementations of classes in the o.a.l.s.spans package as a
@ -82,7 +83,7 @@ final class JustCompileSearchSpans {
}
@Override
public Spans getSpans(AtomicReaderContext context) throws IOException {
public Spans getSpans(AtomicReaderContext context, Bits acceptDocs) throws IOException {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}

View File

@ -49,7 +49,7 @@ public class MultiSpansWrapper extends Spans { // can't be package private due t
public static Spans wrap(ReaderContext topLevelReaderContext, SpanQuery query) throws IOException {
AtomicReaderContext[] leaves = ReaderUtil.leaves(topLevelReaderContext);
if(leaves.length == 1) {
return query.getSpans(leaves[0]);
return query.getSpans(leaves[0], leaves[0].reader.getLiveDocs());
}
return new MultiSpansWrapper(leaves, query);
}
@ -60,14 +60,14 @@ public class MultiSpansWrapper extends Spans { // can't be package private due t
return false;
}
if (current == null) {
current = query.getSpans(leaves[leafOrd]);
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader.getLiveDocs());
}
while(true) {
if (current.next()) {
return true;
}
if (++leafOrd < leaves.length) {
current = query.getSpans(leaves[leafOrd]);
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader.getLiveDocs());
} else {
current = null;
break;
@ -85,17 +85,17 @@ public class MultiSpansWrapper extends Spans { // can't be package private due t
int subIndex = ReaderUtil.subIndex(target, leaves);
assert subIndex >= leafOrd;
if (subIndex != leafOrd) {
current = query.getSpans(leaves[subIndex]);
current = query.getSpans(leaves[subIndex], leaves[subIndex].reader.getLiveDocs());
leafOrd = subIndex;
} else if (current == null) {
current = query.getSpans(leaves[leafOrd]);
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader.getLiveDocs());
}
while (true) {
if (current.skipTo(target - leaves[leafOrd].docBase)) {
return true;
}
if (++leafOrd < leaves.length) {
current = query.getSpans(leaves[leafOrd]);
current = query.getSpans(leaves[leafOrd], leaves[leafOrd].reader.getLiveDocs());
} else {
current = null;
break;

View File

@ -30,7 +30,6 @@ 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;
import org.apache.lucene.util.ReaderUtil;
@ -170,7 +169,7 @@ public class TestNearSpansOrdered extends LuceneTestCase {
Weight w = searcher.createNormalizedWeight(q);
ReaderContext topReaderContext = searcher.getTopReaderContext();
AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
Scorer s = w.scorer(leaves[0], ScorerContext.def());
Scorer s = w.scorer(leaves[0], true, false, leaves[0].reader.getLiveDocs());
assertEquals(1, s.advance(1));
}

View File

@ -23,7 +23,6 @@ import org.apache.lucene.search.CheckHits;
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.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
import org.apache.lucene.search.similarities.Similarity;
@ -435,7 +434,7 @@ public class TestSpans extends LuceneTestCase {
slop,
ordered);
spanScorer = searcher.createNormalizedWeight(snq).scorer(leaves[i], ScorerContext.def());
spanScorer = searcher.createNormalizedWeight(snq).scorer(leaves[i], true, false, leaves[i].reader.getLiveDocs());
} finally {
searcher.setSimilarityProvider(oldSim);
}

View File

@ -38,6 +38,7 @@ import org.apache.lucene.search.Scorer.ChildScorer;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.grouping.TopGroups;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.FixedBitSet;
/**
@ -146,9 +147,10 @@ public class BlockJoinQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext readerContext, ScorerContext context) throws IOException {
public Scorer scorer(AtomicReaderContext readerContext, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
// Pass scoreDocsInOrder true, topScorer false to our sub:
final Scorer childScorer = childWeight.scorer(readerContext, ScorerContext.def().scoreDocsInOrder(true).topScorer(false));
final Scorer childScorer = childWeight.scorer(readerContext, true, false, acceptDocs);
if (childScorer == null) {
// No matches

View File

@ -30,6 +30,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
/**
@ -224,19 +225,20 @@ public class CustomScoreQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) 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, ScorerContext.def());
Scorer subQueryScorer = subQueryWeight.scorer(context, true, false, acceptDocs);
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, scorerContext.scoreDocsInOrder(true));
valSrcScorers[i] = valSrcWeights[i].scorer(context, true, topScorer, acceptDocs);
}
return new CustomScorer(CustomScoreQuery.this.getCustomScoreProvider(context), this, getBoost(), subQueryScorer, valSrcScorers);
}

View File

@ -20,6 +20,7 @@ package org.apache.lucene.queries.function;
import org.apache.lucene.search.*;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
@ -91,8 +92,10 @@ public class BoostedQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
Scorer subQueryScorer = qWeight.scorer(context, ScorerContext.def());
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
// we are gonna advance() the subscorer
Scorer subQueryScorer = qWeight.scorer(context, true, false, acceptDocs);
if(subQueryScorer == null) {
return null;
}

View File

@ -89,13 +89,14 @@ public class FunctionQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new AllScorer(context, this, queryWeight);
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new AllScorer(context, acceptDocs, this, queryWeight);
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
return ((AllScorer)scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true))).explain(doc);
return ((AllScorer)scorer(context, true, true, context.reader.getLiveDocs())).explain(doc);
}
}
@ -106,18 +107,15 @@ public class FunctionQuery extends Query {
final float qWeight;
int doc=-1;
final DocValues vals;
final boolean hasDeletions;
final Bits liveDocs;
public AllScorer(AtomicReaderContext context, FunctionWeight w, float qWeight) throws IOException {
public AllScorer(AtomicReaderContext context, Bits acceptDocs, FunctionWeight w, float qWeight) throws IOException {
super(w);
this.weight = w;
this.qWeight = qWeight;
this.reader = context.reader;
this.maxDoc = reader.maxDoc();
this.hasDeletions = reader.hasDeletions();
this.liveDocs = MultiFields.getLiveDocs(reader);
assert !hasDeletions || liveDocs != null;
this.liveDocs = acceptDocs;
vals = func.getValues(weight.context, context);
}
@ -137,7 +135,7 @@ public class FunctionQuery extends Query {
if (doc>=maxDoc) {
return doc=NO_MORE_DOCS;
}
if (hasDeletions && !liveDocs.get(doc)) continue;
if (liveDocs != null && !liveDocs.get(doc)) continue;
return doc;
}
}

View File

@ -22,7 +22,7 @@ import org.apache.lucene.queries.function.DocValues;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.FloatDocValues;
import org.apache.lucene.search.*;
import org.apache.lucene.search.Weight.ScorerContext;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.mutable.MutableValue;
import org.apache.lucene.util.mutable.MutableValueFloat;
@ -77,6 +77,7 @@ public class QueryValueSource extends ValueSource {
class QueryDocValues extends FloatDocValues {
final AtomicReaderContext readerContext;
final Bits acceptDocs;
final Weight weight;
final float defVal;
final Map fcontext;
@ -95,6 +96,7 @@ class QueryDocValues extends FloatDocValues {
super(vs);
this.readerContext = readerContext;
this.acceptDocs = readerContext.reader.getLiveDocs();
this.defVal = vs.defVal;
this.q = vs.q;
this.fcontext = fcontext;
@ -121,7 +123,7 @@ class QueryDocValues extends FloatDocValues {
try {
if (doc < lastDocRequested) {
if (noMatches) return defVal;
scorer = weight.scorer(readerContext, ScorerContext.def());
scorer = weight.scorer(readerContext, true, false, acceptDocs);
if (scorer==null) {
noMatches = true;
return defVal;
@ -152,7 +154,7 @@ class QueryDocValues extends FloatDocValues {
try {
if (doc < lastDocRequested) {
if (noMatches) return false;
scorer = weight.scorer(readerContext, ScorerContext.def());
scorer = weight.scorer(readerContext, true, false, acceptDocs);
scorerDoc = -1;
if (scorer==null) {
noMatches = true;
@ -210,7 +212,7 @@ class QueryDocValues extends FloatDocValues {
mval.exists = false;
return;
}
scorer = weight.scorer(readerContext, ScorerContext.def());
scorer = weight.scorer(readerContext, true, false, acceptDocs);
scorerDoc = -1;
if (scorer==null) {
noMatches = true;

View File

@ -366,13 +366,14 @@ class SpatialDistanceQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
return new SpatialScorer(context, this, queryWeight);
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new SpatialScorer(context, acceptDocs, this, queryWeight);
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
return ((SpatialScorer)scorer(context, ScorerContext.def().scoreDocsInOrder(true).topScorer(true))).explain(doc);
return ((SpatialScorer)scorer(context, true, true, context.reader.getLiveDocs())).explain(doc);
}
}
@ -400,13 +401,13 @@ class SpatialDistanceQuery extends Query {
int lastDistDoc;
double lastDist;
public SpatialScorer(AtomicReaderContext readerContext, SpatialWeight w, float qWeight) throws IOException {
public SpatialScorer(AtomicReaderContext readerContext, Bits acceptDocs, SpatialWeight w, float qWeight) throws IOException {
super(w);
this.weight = w;
this.qWeight = qWeight;
this.reader = readerContext.reader;
this.maxDoc = reader.maxDoc();
this.liveDocs = reader.getLiveDocs();
this.liveDocs = acceptDocs;
latVals = latSource.getValues(weight.latContext, readerContext);
lonVals = lonSource.getValues(weight.lonContext, readerContext);

View File

@ -187,7 +187,8 @@ class JoinQuery extends Query {
@Override
public Scorer scorer(IndexReader.AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(IndexReader.AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
if (filter == null) {
boolean debug = rb != null && rb.isDebug();
long start = debug ? System.currentTimeMillis() : 0;
@ -483,7 +484,7 @@ class JoinQuery extends Query {
@Override
public Explanation explain(IndexReader.AtomicReaderContext context, int doc) throws IOException {
Scorer scorer = scorer(context, null);
Scorer scorer = scorer(context, true, false, context.reader.getLiveDocs());
boolean exists = scorer.advance(doc) == doc;
ComplexExplanation result = new ComplexExplanation();

View File

@ -2,6 +2,7 @@ package org.apache.solr.search;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.*;
import org.apache.lucene.util.Bits;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
import org.apache.solr.common.SolrException;
@ -118,7 +119,8 @@ public class SolrConstantScoreQuery extends ConstantScoreQuery implements Extend
}
@Override
public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
boolean topScorer, Bits acceptDocs) throws IOException {
return new ConstantScorer(context, this, queryWeight);
}

View File

@ -2080,7 +2080,7 @@ class FilterImpl extends Filter {
iterators.add(iter);
}
for (Weight w : weights) {
Scorer scorer = w.scorer(context, Weight.ScorerContext.def());
Scorer scorer = w.scorer(context, true, false, context.reader.getLiveDocs());
if (scorer == null) return null;
iterators.add(scorer);
}