mirror of https://github.com/apache/lucene.git
LUCENE-5487: commit current patch
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene5487@1573830 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b3318f4c36
commit
a6c2e65534
|
@ -242,7 +242,7 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
|
||||
Weight w = wIter.next();
|
||||
BooleanClause c = cIter.next();
|
||||
if (w.scorer(context, true, true, context.reader().getLiveDocs()) == null) {
|
||||
if (w.scorer(context, context.reader().getLiveDocs()) == null) {
|
||||
if (c.isRequired()) {
|
||||
fail = true;
|
||||
Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
|
||||
|
@ -305,8 +305,43 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs)
|
||||
public TopScorer topScorer(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);
|
||||
}
|
||||
|
||||
List<TopScorer> prohibited = new ArrayList<TopScorer>();
|
||||
List<TopScorer> optional = new ArrayList<TopScorer>();
|
||||
Iterator<BooleanClause> cIter = clauses.iterator();
|
||||
for (Weight w : weights) {
|
||||
BooleanClause c = cIter.next();
|
||||
TopScorer subScorer = w.topScorer(context, false, acceptDocs);
|
||||
if (subScorer == null) {
|
||||
if (c.isRequired()) {
|
||||
return null;
|
||||
}
|
||||
} else if (c.isRequired()) {
|
||||
// TODO: there are some cases where BooleanScorer
|
||||
// would handle conjunctions faster than
|
||||
// BooleanScorer2...
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
} else if (c.isProhibited()) {
|
||||
prohibited.add(subScorer);
|
||||
} else {
|
||||
optional.add(subScorer);
|
||||
}
|
||||
}
|
||||
|
||||
// Check if we can and should return a BooleanScorer
|
||||
return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs)
|
||||
throws IOException {
|
||||
List<Scorer> required = new ArrayList<Scorer>();
|
||||
List<Scorer> prohibited = new ArrayList<Scorer>();
|
||||
|
@ -314,7 +349,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, acceptDocs);
|
||||
Scorer subScorer = w.scorer(context, acceptDocs);
|
||||
if (subScorer == null) {
|
||||
if (c.isRequired()) {
|
||||
return null;
|
||||
|
@ -328,20 +363,6 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
|
|||
}
|
||||
}
|
||||
|
||||
// NOTE: we could also use BooleanScorer, if we knew
|
||||
// this BooleanQuery was embedded in another
|
||||
// BooleanQuery that was also using BooleanScorer (ie,
|
||||
// BooleanScorer can nest). But this is hard to
|
||||
// detect and we never do so today... (ie, we only
|
||||
// return BooleanScorer for topScorer):
|
||||
|
||||
// Check if we can and should return a BooleanScorer
|
||||
// TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
|
||||
// but the same is even true of pure conjunctions...
|
||||
if (!scoreDocsInOrder && topScorer && required.size() == 0 && minNrShouldMatch <= 1) {
|
||||
return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
|
||||
}
|
||||
|
||||
if (required.size() == 0 && optional.size() == 0) {
|
||||
// no required and optional clauses.
|
||||
return null;
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.Collection;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.search.BooleanQuery.BooleanWeight;
|
||||
|
||||
/* Description from Doug Cutting (excerpted from
|
||||
|
@ -58,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 Scorer {
|
||||
final class BooleanScorer extends TopScorer {
|
||||
|
||||
private static final class BooleanScorerCollector extends Collector {
|
||||
private BucketTable bucketTable;
|
||||
|
@ -121,7 +122,7 @@ final class BooleanScorer extends Scorer {
|
|||
public BucketScorer(Weight weight) { super(weight); }
|
||||
|
||||
@Override
|
||||
public int advance(int target) { return NO_MORE_DOCS; }
|
||||
public int advance(int target) { return DocsEnum.NO_MORE_DOCS; }
|
||||
|
||||
@Override
|
||||
public int docID() { return doc; }
|
||||
|
@ -130,7 +131,7 @@ final class BooleanScorer extends Scorer {
|
|||
public int freq() { return freq; }
|
||||
|
||||
@Override
|
||||
public int nextDoc() { return NO_MORE_DOCS; }
|
||||
public int nextDoc() { return DocsEnum.NO_MORE_DOCS; }
|
||||
|
||||
@Override
|
||||
public float score() { return (float)score; }
|
||||
|
@ -175,19 +176,21 @@ final class BooleanScorer extends Scorer {
|
|||
}
|
||||
|
||||
static final class SubScorer {
|
||||
public Scorer scorer;
|
||||
public TopScorer scorer;
|
||||
// TODO: re-enable this if BQ ever sends us required clauses
|
||||
//public boolean required = false;
|
||||
public boolean prohibited;
|
||||
public Collector collector;
|
||||
public SubScorer next;
|
||||
public boolean more;
|
||||
|
||||
public SubScorer(Scorer scorer, boolean required, boolean prohibited,
|
||||
public SubScorer(TopScorer scorer, boolean required, boolean prohibited,
|
||||
Collector collector, SubScorer next) {
|
||||
if (required) {
|
||||
throw new IllegalArgumentException("this scorer cannot handle required=true");
|
||||
}
|
||||
this.scorer = scorer;
|
||||
this.more = true;
|
||||
// TODO: re-enable this if BQ ever sends us required clauses
|
||||
//this.required = required;
|
||||
this.prohibited = prohibited;
|
||||
|
@ -206,26 +209,20 @@ final class BooleanScorer extends Scorer {
|
|||
private Bucket current;
|
||||
// Any time a prohibited clause matches we set bit 0:
|
||||
private static final int PROHIBITED_MASK = 1;
|
||||
|
||||
BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
|
||||
List<Scorer> optionalScorers, List<Scorer> prohibitedScorers, int maxCoord) throws IOException {
|
||||
super(weight);
|
||||
this.minNrShouldMatch = minNrShouldMatch;
|
||||
|
||||
if (optionalScorers != null && optionalScorers.size() > 0) {
|
||||
for (Scorer scorer : optionalScorers) {
|
||||
if (scorer.nextDoc() != NO_MORE_DOCS) {
|
||||
scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers);
|
||||
}
|
||||
}
|
||||
private final Weight weight;
|
||||
|
||||
BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
|
||||
List<TopScorer> optionalScorers, List<TopScorer> prohibitedScorers, int maxCoord) throws IOException {
|
||||
this.minNrShouldMatch = minNrShouldMatch;
|
||||
this.weight = weight;
|
||||
|
||||
for (TopScorer scorer : optionalScorers) {
|
||||
scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers);
|
||||
}
|
||||
|
||||
if (prohibitedScorers != null && prohibitedScorers.size() > 0) {
|
||||
for (Scorer scorer : prohibitedScorers) {
|
||||
if (scorer.nextDoc() != NO_MORE_DOCS) {
|
||||
scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
|
||||
}
|
||||
}
|
||||
for (TopScorer scorer : prohibitedScorers) {
|
||||
scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
|
||||
}
|
||||
|
||||
coordFactors = new float[optionalScorers.size() + 1];
|
||||
|
@ -234,11 +231,9 @@ final class BooleanScorer extends Scorer {
|
|||
}
|
||||
}
|
||||
|
||||
// firstDocID is ignored since nextDoc() initializes 'current'
|
||||
@Override
|
||||
public boolean score(Collector collector, int max, int firstDocID) throws IOException {
|
||||
// Make sure it's only BooleanScorer that calls us:
|
||||
assert firstDocID == -1;
|
||||
public boolean score(Collector collector, int max) throws IOException {
|
||||
|
||||
boolean more;
|
||||
Bucket tmp;
|
||||
BucketScorer bs = new BucketScorer(weight);
|
||||
|
@ -292,9 +287,9 @@ final class BooleanScorer extends Scorer {
|
|||
more = false;
|
||||
end += BucketTable.SIZE;
|
||||
for (SubScorer sub = scorers; sub != null; sub = sub.next) {
|
||||
int subScorerDocID = sub.scorer.docID();
|
||||
if (subScorerDocID != NO_MORE_DOCS) {
|
||||
more |= sub.scorer.score(sub.collector, end, subScorerDocID);
|
||||
if (sub.more) {
|
||||
sub.more = sub.scorer.score(sub.collector, end);
|
||||
more |= sub.more;
|
||||
}
|
||||
}
|
||||
current = bucketTable.first;
|
||||
|
@ -303,40 +298,10 @@ final class BooleanScorer extends Scorer {
|
|||
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
score(collector, Integer.MAX_VALUE, -1);
|
||||
score(collector, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -350,9 +315,4 @@ final class BooleanScorer extends Scorer {
|
|||
buffer.append(")");
|
||||
return buffer.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -279,28 +279,6 @@ class BooleanScorer2 extends Scorer {
|
|||
: new MinShouldMatchSumScorer(weight, prohibitedScorers)));
|
||||
}
|
||||
|
||||
/** Scores and collects all matching documents.
|
||||
* @param collector The collector to which all matching documents are passed through.
|
||||
*/
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
collector.setScorer(this);
|
||||
while ((doc = countingSumScorer.nextDoc()) != NO_MORE_DOCS) {
|
||||
collector.collect(doc);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max, int firstDocID) throws IOException {
|
||||
doc = firstDocID;
|
||||
collector.setScorer(this);
|
||||
while (doc < max) {
|
||||
collector.collect(doc);
|
||||
doc = countingSumScorer.nextDoc();
|
||||
}
|
||||
return doc != NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
|
|
|
@ -134,8 +134,23 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, final Bits acceptDocs) throws IOException {
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
final DocIdSetIterator disi;
|
||||
if (filter != null) {
|
||||
assert query == null;
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
} else {
|
||||
assert query != null && innerWeight != null;
|
||||
TopScorer topScorer = innerWeight.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
if (topScorer == null) {
|
||||
return null;
|
||||
}
|
||||
return new ConstantTopScorer(topScorer, this, queryWeight);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
final DocIdSetIterator disi;
|
||||
if (filter != null) {
|
||||
assert query == null;
|
||||
|
@ -146,7 +161,7 @@ public class ConstantScoreQuery extends Query {
|
|||
disi = dis.iterator();
|
||||
} else {
|
||||
assert query != null && innerWeight != null;
|
||||
disi = innerWeight.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
|
||||
disi = innerWeight.scorer(context, acceptDocs);
|
||||
}
|
||||
|
||||
if (disi == null) {
|
||||
|
@ -154,7 +169,7 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
return new ConstantScorer(disi, this, queryWeight);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean scoresDocsOutOfOrder() {
|
||||
return (innerWeight != null) ? innerWeight.scoresDocsOutOfOrder() : false;
|
||||
|
@ -162,7 +177,7 @@ public class ConstantScoreQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
final Scorer cs = scorer(context, true, false, context.reader().getLiveDocs());
|
||||
final Scorer cs = scorer(context, context.reader().getLiveDocs());
|
||||
final boolean exists = (cs != null && cs.advance(doc) == doc);
|
||||
|
||||
final ComplexExplanation result = new ComplexExplanation();
|
||||
|
@ -181,6 +196,52 @@ public class ConstantScoreQuery extends Query {
|
|||
}
|
||||
}
|
||||
|
||||
/** We return this as our {@link TopScorer} 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;
|
||||
final Weight weight;
|
||||
final float theScore;
|
||||
|
||||
public ConstantTopScorer(TopScorer topScorer, Weight weight, float theScore) {
|
||||
this.topScorer = topScorer;
|
||||
this.weight = weight;
|
||||
this.theScore = theScore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max) throws IOException {
|
||||
return topScorer.score(wrapCollector(collector), max);
|
||||
}
|
||||
|
||||
private Collector wrapCollector(final Collector collector) {
|
||||
return new Collector() {
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
// we must wrap again here, but using the scorer passed in as parameter:
|
||||
collector.setScorer(new ConstantScorer(scorer, weight, theScore));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
collector.collect(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNextReader(AtomicReaderContext context) throws IOException {
|
||||
collector.setNextReader(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return collector.acceptsDocsOutOfOrder();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
protected class ConstantScorer extends Scorer {
|
||||
final DocIdSetIterator docIdSetIterator;
|
||||
final float theScore;
|
||||
|
@ -222,57 +283,13 @@ public class ConstantScoreQuery extends Query {
|
|||
return docIdSetIterator.cost();
|
||||
}
|
||||
|
||||
private Collector wrapCollector(final Collector collector) {
|
||||
return new Collector() {
|
||||
@Override
|
||||
public void setScorer(Scorer scorer) throws IOException {
|
||||
// we must wrap again here, but using the scorer passed in as parameter:
|
||||
collector.setScorer(new ConstantScorer(scorer, ConstantScorer.this.weight, ConstantScorer.this.theScore));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void collect(int doc) throws IOException {
|
||||
collector.collect(doc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setNextReader(AtomicReaderContext context) throws IOException {
|
||||
collector.setNextReader(context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean acceptsDocsOutOfOrder() {
|
||||
return collector.acceptsDocsOutOfOrder();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
// this optimization allows out of order scoring as top scorer!
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
if (query != null) {
|
||||
((Scorer) docIdSetIterator).score(wrapCollector(collector));
|
||||
} else {
|
||||
super.score(collector);
|
||||
}
|
||||
}
|
||||
|
||||
// this optimization allows out of order scoring as top scorer,
|
||||
@Override
|
||||
public boolean score(Collector collector, int max, int firstDocID) throws IOException {
|
||||
if (query != null) {
|
||||
return ((Scorer) docIdSetIterator).score(wrapCollector(collector), max, firstDocID);
|
||||
} else {
|
||||
return super.score(collector, max, firstDocID);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
if (query != null)
|
||||
if (query != null) {
|
||||
return Collections.singletonList(new ChildScorer((Scorer) docIdSetIterator, "constant"));
|
||||
else
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -153,12 +153,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, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
List<Scorer> scorers = new ArrayList<Scorer>();
|
||||
for (Weight w : weights) {
|
||||
// we will advance() subscorers
|
||||
Scorer subScorer = w.scorer(context, true, false, acceptDocs);
|
||||
Scorer subScorer = w.scorer(context, acceptDocs);
|
||||
if (subScorer != null) {
|
||||
scorers.add(subScorer);
|
||||
|
||||
|
|
|
@ -93,12 +93,12 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void normalize (float norm, float topLevelBoost) {
|
||||
public void normalize(float norm, float topLevelBoost) {
|
||||
weight.normalize(norm, topLevelBoost * getBoost()); // incorporate boost
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain (AtomicReaderContext ir, int i) throws IOException {
|
||||
public Explanation explain(AtomicReaderContext ir, int i) throws IOException {
|
||||
Explanation inner = weight.explain (ir, i);
|
||||
Filter f = FilteredQuery.this.filter;
|
||||
DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader().getLiveDocs());
|
||||
|
@ -124,7 +124,7 @@ public class FilteredQuery extends Query {
|
|||
|
||||
// return a filtering scorer
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, final Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, final Bits acceptDocs) throws IOException {
|
||||
assert filter != null;
|
||||
|
||||
final DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
|
||||
|
@ -133,7 +133,7 @@ public class FilteredQuery extends Query {
|
|||
return null;
|
||||
}
|
||||
|
||||
return strategy.filteredScorer(context, scoreDocsInOrder, topScorer, weight, filterDocIdSet);
|
||||
return strategy.filteredScorer(context, weight, filterDocIdSet);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
@ -154,8 +154,10 @@ public class FilteredQuery extends Query {
|
|||
this.scorer = other;
|
||||
this.filterbits = filterBits;
|
||||
}
|
||||
|
||||
|
||||
// nocommit move to custom TopScorer
|
||||
// optimization: we are topScorer and collect directly
|
||||
/*
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
// the normalization trick already applies the boost of this query,
|
||||
|
@ -171,6 +173,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
|
@ -236,8 +239,10 @@ public class FilteredQuery extends Query {
|
|||
this.secondary = secondary;
|
||||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
|
||||
// nocommit move to TopScorer
|
||||
// optimization: we are topScorer and collect directly using short-circuited algo
|
||||
/*
|
||||
@Override
|
||||
public final void score(Collector collector) throws IOException {
|
||||
// the normalization trick already applies the boost of this query,
|
||||
|
@ -261,6 +266,7 @@ public class FilteredQuery extends Query {
|
|||
}
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
private final int advanceToNextCommonDoc() throws IOException {
|
||||
for (;;) {
|
||||
|
@ -491,8 +497,7 @@ public class FilteredQuery extends Query {
|
|||
* @throws IOException if an {@link IOException} occurs
|
||||
*/
|
||||
public abstract Scorer filteredScorer(AtomicReaderContext context,
|
||||
boolean scoreDocsInOrder, boolean topScorer, Weight weight,
|
||||
DocIdSet docIdSet) throws IOException;
|
||||
Weight weight, DocIdSet docIdSet) throws IOException;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -506,7 +511,7 @@ public class FilteredQuery extends Query {
|
|||
public static class RandomAccessFilterStrategy extends FilterStrategy {
|
||||
|
||||
@Override
|
||||
public Scorer filteredScorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Weight weight, DocIdSet docIdSet) throws IOException {
|
||||
public Scorer filteredScorer(AtomicReaderContext context, Weight weight, DocIdSet docIdSet) throws IOException {
|
||||
final DocIdSetIterator filterIter = docIdSet.iterator();
|
||||
if (filterIter == null) {
|
||||
// this means the filter does not accept any documents.
|
||||
|
@ -523,12 +528,12 @@ public class FilteredQuery extends Query {
|
|||
final boolean useRandomAccess = filterAcceptDocs != null && useRandomAccess(filterAcceptDocs, firstFilterDoc);
|
||||
if (useRandomAccess) {
|
||||
// if we are using random access, we return the inner scorer, just with other acceptDocs
|
||||
return weight.scorer(context, scoreDocsInOrder, topScorer, filterAcceptDocs);
|
||||
return weight.scorer(context, filterAcceptDocs);
|
||||
} else {
|
||||
assert firstFilterDoc > -1;
|
||||
// we are gonna advance() this scorer, so we set inorder=true/toplevel=false
|
||||
// we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
|
||||
final Scorer scorer = weight.scorer(context, true, false, null);
|
||||
final Scorer scorer = weight.scorer(context, null);
|
||||
// TODO once we have way to figure out if we use RA or LeapFrog we can remove this scorer
|
||||
return (scorer == null) ? null : new PrimaryAdvancedLeapFrogScorer(weight, firstFilterDoc, filterIter, scorer);
|
||||
}
|
||||
|
@ -559,18 +564,17 @@ public class FilteredQuery extends Query {
|
|||
private LeapFrogFilterStrategy(boolean scorerFirst) {
|
||||
this.scorerFirst = scorerFirst;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Scorer filteredScorer(AtomicReaderContext context,
|
||||
boolean scoreDocsInOrder, boolean topScorer, Weight weight,
|
||||
DocIdSet docIdSet) throws IOException {
|
||||
Weight weight, DocIdSet docIdSet) throws IOException {
|
||||
final DocIdSetIterator filterIter = docIdSet.iterator();
|
||||
if (filterIter == null) {
|
||||
// this means the filter does not accept any documents.
|
||||
return null;
|
||||
}
|
||||
// we are gonna advance() this scorer, so we set inorder=true/toplevel=false
|
||||
// we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
|
||||
final Scorer scorer = weight.scorer(context, true, false, null);
|
||||
final Scorer scorer = weight.scorer(context, null);
|
||||
if (scorerFirst) {
|
||||
return (scorer == null) ? null : new LeapFrogScorer(weight, scorer, filterIter, scorer);
|
||||
} else {
|
||||
|
@ -596,13 +600,13 @@ public class FilteredQuery extends Query {
|
|||
private static final class QueryFirstFilterStrategy extends FilterStrategy {
|
||||
@Override
|
||||
public Scorer filteredScorer(final AtomicReaderContext context,
|
||||
boolean scoreDocsInOrder, boolean topScorer, Weight weight,
|
||||
Weight weight,
|
||||
DocIdSet docIdSet) throws IOException {
|
||||
Bits filterAcceptDocs = docIdSet.bits();
|
||||
if (filterAcceptDocs == null) {
|
||||
return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, scoreDocsInOrder, topScorer, weight, docIdSet);
|
||||
return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet);
|
||||
}
|
||||
final Scorer scorer = weight.scorer(context, true, false, null);
|
||||
final Scorer scorer = weight.scorer(context, null);
|
||||
return scorer == null ? null : new QueryFirstScorer(weight,
|
||||
filterAcceptDocs, scorer);
|
||||
}
|
||||
|
|
|
@ -607,7 +607,7 @@ public class IndexSearcher {
|
|||
// continue with the following leaf
|
||||
continue;
|
||||
}
|
||||
Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs());
|
||||
TopScorer scorer = weight.topScorer(ctx, !collector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
try {
|
||||
scorer.score(collector);
|
||||
|
|
|
@ -114,8 +114,7 @@ public class MatchAllDocsQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
|
||||
}
|
||||
|
||||
|
|
|
@ -179,8 +179,7 @@ public class MultiPhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
assert !termArrays.isEmpty();
|
||||
final AtomicReader reader = context.reader();
|
||||
final Bits liveDocs = acceptDocs;
|
||||
|
@ -263,7 +262,7 @@ public class MultiPhraseQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
|
||||
Scorer scorer = scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
if (newDoc == doc) {
|
||||
|
|
|
@ -245,8 +245,7 @@ public class PhraseQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
assert !terms.isEmpty();
|
||||
final AtomicReader reader = context.reader();
|
||||
final Bits liveDocs = acceptDocs;
|
||||
|
@ -305,7 +304,7 @@ public class PhraseQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
|
||||
Scorer scorer = scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
if (newDoc == doc) {
|
||||
|
|
|
@ -56,7 +56,7 @@ public class QueryWrapperFilter extends Filter {
|
|||
return new DocIdSet() {
|
||||
@Override
|
||||
public DocIdSetIterator iterator() throws IOException {
|
||||
return weight.scorer(privateContext, true, false, acceptDocs);
|
||||
return weight.scorer(privateContext, acceptDocs);
|
||||
}
|
||||
@Override
|
||||
public boolean isCacheable() { return false; }
|
||||
|
|
|
@ -44,11 +44,6 @@ public class ScoreCachingWrappingScorer extends Scorer {
|
|||
this.scorer = scorer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max, int firstDocID) throws IOException {
|
||||
return scorer.score(collector, max, firstDocID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
int doc = scorer.docID();
|
||||
|
@ -75,11 +70,6 @@ public class ScoreCachingWrappingScorer extends Scorer {
|
|||
return scorer.nextDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
scorer.score(collector);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
return scorer.advance(target);
|
||||
|
|
|
@ -54,42 +54,6 @@ public abstract class Scorer extends DocsEnum {
|
|||
this.weight = weight;
|
||||
}
|
||||
|
||||
/** Scores and collects all matching documents.
|
||||
* @param collector The collector to which all matching documents are passed.
|
||||
*/
|
||||
public void score(Collector collector) throws IOException {
|
||||
assert docID() == -1; // not started
|
||||
collector.setScorer(this);
|
||||
int doc;
|
||||
while ((doc = nextDoc()) != NO_MORE_DOCS) {
|
||||
collector.collect(doc);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*
|
||||
* @param collector
|
||||
* The collector to which all matching documents are passed.
|
||||
* @param max
|
||||
* Do not score documents past this.
|
||||
* @param firstDocID
|
||||
* The first document ID (ensures {@link #nextDoc()} is called before
|
||||
* this method.
|
||||
* @return true if more matching documents may remain.
|
||||
*/
|
||||
public boolean score(Collector collector, int max, int firstDocID) throws IOException {
|
||||
assert docID() == firstDocID;
|
||||
collector.setScorer(this);
|
||||
int doc;
|
||||
for (doc = firstDocID; doc < max; doc = nextDoc()) {
|
||||
collector.collect(doc);
|
||||
}
|
||||
return doc != NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
/** Returns the score of the current document matching the query.
|
||||
* Initially invalid, until {@link #nextDoc()} or {@link #advance(int)}
|
||||
* is called the first time, or when called from within
|
||||
|
|
|
@ -75,8 +75,7 @@ public class TermQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, 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) {
|
||||
|
@ -111,7 +110,7 @@ public class TermQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
|
||||
Scorer scorer = scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
if (newDoc == doc) {
|
||||
|
|
|
@ -0,0 +1,43 @@
|
|||
package org.apache.lucene.search;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public abstract class TopScorer {
|
||||
|
||||
/** Scores and collects all matching documents.
|
||||
* @param collector The collector to which all matching documents are passed.
|
||||
*/
|
||||
public void score(Collector collector) throws IOException {
|
||||
score(collector, Integer.MAX_VALUE);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*
|
||||
* @param collector
|
||||
* The collector to which all matching documents are passed.
|
||||
* @param max
|
||||
* Do not score documents past this.
|
||||
* @return true if more matching documents may remain.
|
||||
*/
|
||||
public abstract boolean score(Collector collector, int max) throws IOException;
|
||||
}
|
|
@ -21,6 +21,7 @@ import java.io.IOException;
|
|||
|
||||
import org.apache.lucene.index.AtomicReader; // javadocs
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexReaderContext; // javadocs
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
@ -110,8 +111,37 @@ public abstract class Weight {
|
|||
* @return a {@link Scorer} which scores documents in/out-of order.
|
||||
* @throws IOException if there is a low-level I/O error
|
||||
*/
|
||||
public abstract Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException;
|
||||
public abstract Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException;
|
||||
|
||||
// nocommit jdocs
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
final Scorer scorer = scorer(context, acceptDocs);
|
||||
if (scorer == null) {
|
||||
// No docs match
|
||||
return null;
|
||||
}
|
||||
|
||||
// This impl always scores docs in order, so we can
|
||||
// ignore scoreDocsInOrder:
|
||||
return new TopScorer() {
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max) throws IOException {
|
||||
// nocommit weird to do this here? we do it many,
|
||||
// many times from BS1 inside one segment?
|
||||
collector.setScorer(scorer);
|
||||
if (scorer.docID() == -1) {
|
||||
scorer.nextDoc();
|
||||
}
|
||||
int doc;
|
||||
for (doc = scorer.docID(); doc < max; doc = scorer.nextDoc()) {
|
||||
collector.collect(doc);
|
||||
}
|
||||
return doc != DocsEnum.NO_MORE_DOCS;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true iff this implementation scores docs only out of order. This
|
||||
|
|
|
@ -148,15 +148,14 @@ public class PayloadNearQuery extends SpanNearQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this,
|
||||
similarity, similarity.simScorer(stats, context));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
|
||||
PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
if (newDoc == doc) {
|
||||
|
|
|
@ -79,8 +79,7 @@ public class PayloadTermQuery extends SpanTermQuery {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts),
|
||||
this, similarity.simScorer(stats, context));
|
||||
}
|
||||
|
@ -177,7 +176,7 @@ public class PayloadTermQuery extends SpanTermQuery {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
|
||||
PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
if (newDoc == doc) {
|
||||
|
|
|
@ -81,8 +81,7 @@ public class SpanWeight extends Weight {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
if (stats == null) {
|
||||
return null;
|
||||
} else {
|
||||
|
@ -92,7 +91,7 @@ public class SpanWeight extends Weight {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
SpanScorer scorer = (SpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
|
||||
SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null) {
|
||||
int newDoc = scorer.advance(doc);
|
||||
if (newDoc == doc) {
|
||||
|
|
|
@ -17,8 +17,6 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
|
@ -231,11 +229,6 @@ final class JustCompileSearch {
|
|||
super(weight);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean score(Collector collector, int max, int firstDocID) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
|
@ -351,8 +344,7 @@ final class JustCompileSearch {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) {
|
||||
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
|
||||
}
|
||||
|
||||
|
|
|
@ -183,7 +183,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
Weight w = s.createNormalizedWeight(bq);
|
||||
|
||||
assertEquals(1, s.getIndexReader().leaves().size());
|
||||
Scorer scorer = w.scorer(s.getIndexReader().leaves().get(0), false, true, null);
|
||||
TopScorer scorer = w.topScorer(s.getIndexReader().leaves().get(0), false, null);
|
||||
|
||||
final FixedBitSet hits = new FixedBitSet(docCount);
|
||||
final AtomicInteger end = new AtomicInteger();
|
||||
|
@ -211,7 +211,7 @@ public class TestBooleanOr extends LuceneTestCase {
|
|||
while (end.intValue() < docCount) {
|
||||
final int inc = TestUtil.nextInt(random(), 1, 1000);
|
||||
end.getAndAdd(inc);
|
||||
scorer.score(c, end.intValue(), -1);
|
||||
scorer.score(c, end.intValue());
|
||||
}
|
||||
|
||||
assertEquals(docCount, hits.cardinality());
|
||||
|
|
|
@ -234,8 +234,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
|
||||
Weight weight = s.createNormalizedWeight(q);
|
||||
|
||||
Scorer scorer = weight.scorer(s.leafContexts.get(0),
|
||||
true, false, null);
|
||||
Scorer scorer = weight.scorer(s.leafContexts.get(0), null);
|
||||
|
||||
// First pass: just use .nextDoc() to gather all hits
|
||||
final List<ScoreDoc> hits = new ArrayList<ScoreDoc>();
|
||||
|
@ -252,8 +251,7 @@ public class TestBooleanQuery extends LuceneTestCase {
|
|||
for(int iter2=0;iter2<10;iter2++) {
|
||||
|
||||
weight = s.createNormalizedWeight(q);
|
||||
scorer = weight.scorer(s.leafContexts.get(0),
|
||||
true, false, null);
|
||||
scorer = weight.scorer(s.leafContexts.get(0), null);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println(" iter2=" + iter2);
|
||||
|
|
|
@ -17,8 +17,10 @@ package org.apache.lucene.search;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
|
@ -78,27 +80,50 @@ public class TestBooleanScorer extends LuceneTestCase
|
|||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(ir);
|
||||
BooleanWeight weight = (BooleanWeight) new BooleanQuery().createWeight(searcher);
|
||||
Scorer[] scorers = new Scorer[] {new Scorer(weight) {
|
||||
TopScorer[] scorers = new TopScorer[] {new TopScorer() {
|
||||
private int doc = -1;
|
||||
@Override public float score() { return 0; }
|
||||
@Override public int freq() { return 0; }
|
||||
@Override public int docID() { return doc; }
|
||||
|
||||
@Override public int nextDoc() {
|
||||
return doc = doc == -1 ? 3000 : NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override public int advance(int target) {
|
||||
return doc = target <= 3000 ? 3000 : NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 1;
|
||||
public boolean score(Collector c, int maxDoc) throws IOException {
|
||||
assert doc == -1;
|
||||
doc = 3000;
|
||||
c.setScorer(new Scorer(null) {
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
throw new UnsupportedOperationException("FakeScorer doesn't support advance(int)");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
throw new UnsupportedOperationException("FakeScorer doesn't support freq()");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return 1.0f;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return 1;
|
||||
}
|
||||
});
|
||||
c.collect(3000);
|
||||
return false;
|
||||
}
|
||||
}};
|
||||
|
||||
BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), null, scorers.length);
|
||||
BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), Collections.<TopScorer>emptyList(), scorers.length);
|
||||
|
||||
final List<Integer> hits = new ArrayList<Integer>();
|
||||
bs.score(new Collector() {
|
||||
|
@ -180,4 +205,7 @@ public class TestBooleanScorer extends LuceneTestCase
|
|||
r.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
// nocommit add test verifying that BQ inside BQ can get BS1
|
||||
// not BS2 like today
|
||||
}
|
||||
|
|
|
@ -180,7 +180,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
assertTrue(s.getTopReaderContext() instanceof AtomicReaderContext);
|
||||
final Weight dw = s.createNormalizedWeight(dq);
|
||||
AtomicReaderContext context = (AtomicReaderContext)s.getTopReaderContext();
|
||||
final Scorer ds = dw.scorer(context, true, false, context.reader().getLiveDocs());
|
||||
final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
|
||||
final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
|
||||
if (skipOk) {
|
||||
fail("firsttime skipTo found a match? ... "
|
||||
|
@ -196,7 +196,7 @@ public class TestDisjunctionMaxQuery extends LuceneTestCase {
|
|||
QueryUtils.check(random(), dq, s);
|
||||
final Weight dw = s.createNormalizedWeight(dq);
|
||||
AtomicReaderContext context = (AtomicReaderContext)s.getTopReaderContext();
|
||||
final Scorer ds = dw.scorer(context, true, false, context.reader().getLiveDocs());
|
||||
final Scorer ds = dw.scorer(context, 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"));
|
||||
|
|
|
@ -126,7 +126,7 @@ public class TestMinShouldMatch2 extends LuceneTestCase {
|
|||
if (slow) {
|
||||
return new SlowMinShouldMatchScorer(weight, reader, searcher);
|
||||
} else {
|
||||
return weight.scorer(reader.getContext(), true, false, null);
|
||||
return weight.scorer(reader.getContext(), null);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -78,7 +78,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof AtomicReaderContext);
|
||||
AtomicReaderContext context = (AtomicReaderContext)indexSearcher.getTopReaderContext();
|
||||
Scorer ts = weight.scorer(context, true, true, context.reader().getLiveDocs());
|
||||
TopScorer ts = weight.topScorer(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>();
|
||||
|
@ -140,7 +140,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof AtomicReaderContext);
|
||||
AtomicReaderContext context = (AtomicReaderContext) indexSearcher.getTopReaderContext();
|
||||
Scorer ts = weight.scorer(context, true, false, context.reader().getLiveDocs());
|
||||
Scorer ts = weight.scorer(context, 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);
|
||||
|
@ -159,7 +159,7 @@ public class TestTermScorer extends LuceneTestCase {
|
|||
Weight weight = indexSearcher.createNormalizedWeight(termQuery);
|
||||
assertTrue(indexSearcher.getTopReaderContext() instanceof AtomicReaderContext);
|
||||
AtomicReaderContext context = (AtomicReaderContext) indexSearcher.getTopReaderContext();
|
||||
Scorer ts = weight.scorer(context, true, false, context.reader().getLiveDocs());
|
||||
Scorer ts = weight.scorer(context, 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);
|
||||
|
|
|
@ -167,7 +167,7 @@ public class TestNearSpansOrdered extends LuceneTestCase {
|
|||
Weight w = searcher.createNormalizedWeight(q);
|
||||
IndexReaderContext topReaderContext = searcher.getTopReaderContext();
|
||||
AtomicReaderContext leave = topReaderContext.leaves().get(0);
|
||||
Scorer s = w.scorer(leave, true, false, leave.reader().getLiveDocs());
|
||||
Scorer s = w.scorer(leave, leave.reader().getLiveDocs());
|
||||
assertEquals(1, s.advance(1));
|
||||
}
|
||||
|
||||
|
|
|
@ -429,7 +429,7 @@ public class TestSpans extends LuceneTestCase {
|
|||
slop,
|
||||
ordered);
|
||||
|
||||
spanScorer = searcher.createNormalizedWeight(snq).scorer(ctx, true, false, ctx.reader().getLiveDocs());
|
||||
spanScorer = searcher.createNormalizedWeight(snq).scorer(ctx, ctx.reader().getLiveDocs());
|
||||
} finally {
|
||||
searcher.setSimilarity(oldSim);
|
||||
}
|
||||
|
|
|
@ -29,6 +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.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
|
@ -117,12 +118,17 @@ class DrillSidewaysQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
// We can only run as a top scorer:
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
// TODO: it could be better if we take acceptDocs
|
||||
// into account instead of baseScorer?
|
||||
Scorer baseScorer = baseWeight.scorer(context, scoreDocsInOrder, false, acceptDocs);
|
||||
Scorer baseScorer = baseWeight.scorer(context, acceptDocs);
|
||||
|
||||
DrillSidewaysScorer.DocsAndCost[] dims = new DrillSidewaysScorer.DocsAndCost[drillDowns.length];
|
||||
int nullCount = 0;
|
||||
|
@ -167,7 +173,7 @@ class DrillSidewaysQuery extends Query {
|
|||
dims[dim].disi = disi;
|
||||
}
|
||||
} else {
|
||||
DocIdSetIterator disi = ((Weight) drillDowns[dim]).scorer(context, true, false, null);
|
||||
DocIdSetIterator disi = ((Weight) drillDowns[dim]).scorer(context, null);
|
||||
if (disi == null) {
|
||||
nullCount++;
|
||||
continue;
|
||||
|
|
|
@ -22,14 +22,16 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
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.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
class DrillSidewaysScorer extends Scorer {
|
||||
class DrillSidewaysScorer extends TopScorer {
|
||||
|
||||
//private static boolean DEBUG = false;
|
||||
|
||||
|
@ -52,7 +54,6 @@ class DrillSidewaysScorer extends Scorer {
|
|||
|
||||
DrillSidewaysScorer(Weight w, AtomicReaderContext context, Scorer baseScorer, Collector drillDownCollector,
|
||||
DocsAndCost[] dims, boolean scoreSubDocsAtOnce) {
|
||||
super(w);
|
||||
this.dims = dims;
|
||||
this.context = context;
|
||||
this.baseScorer = baseScorer;
|
||||
|
@ -61,18 +62,22 @@ class DrillSidewaysScorer extends Scorer {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
public boolean score(Collector collector, int maxDoc) throws IOException {
|
||||
if (maxDoc != Integer.MAX_VALUE) {
|
||||
throw new IllegalArgumentException("maxDoc must be Integer.MAX_VALUE");
|
||||
}
|
||||
//if (DEBUG) {
|
||||
// System.out.println("\nscore: reader=" + context.reader());
|
||||
//}
|
||||
//System.out.println("score r=" + context.reader());
|
||||
collector.setScorer(this);
|
||||
FakeScorer scorer = new FakeScorer();
|
||||
collector.setScorer(scorer);
|
||||
if (drillDownCollector != null) {
|
||||
drillDownCollector.setScorer(this);
|
||||
drillDownCollector.setScorer(scorer);
|
||||
drillDownCollector.setNextReader(context);
|
||||
}
|
||||
for (DocsAndCost dim : dims) {
|
||||
dim.sidewaysCollector.setScorer(this);
|
||||
dim.sidewaysCollector.setScorer(scorer);
|
||||
dim.sidewaysCollector.setNextReader(context);
|
||||
}
|
||||
|
||||
|
@ -140,6 +145,8 @@ class DrillSidewaysScorer extends Scorer {
|
|||
//System.out.println("union");
|
||||
doUnionScoring(collector, disis, sidewaysCollectors);
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/** Used when base query is highly constraining vs the
|
||||
|
@ -154,7 +161,7 @@ class DrillSidewaysScorer extends Scorer {
|
|||
//}
|
||||
int docID = baseScorer.docID();
|
||||
|
||||
nextDoc: while (docID != NO_MORE_DOCS) {
|
||||
nextDoc: while (docID != DocsEnum.NO_MORE_DOCS) {
|
||||
Collector failedCollector = null;
|
||||
for (int i=0;i<disis.length;i++) {
|
||||
// TODO: should we sort this 2nd dimension of
|
||||
|
@ -612,39 +619,48 @@ class DrillSidewaysScorer extends Scorer {
|
|||
sidewaysCollector.collect(collectDocID);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return collectDocID;
|
||||
}
|
||||
private final class FakeScorer extends Scorer {
|
||||
float score;
|
||||
int doc;
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return collectScore;
|
||||
}
|
||||
public FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
throw new UnsupportedOperationException("FakeScorer doesn't support advance(int)");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return 1+dims.length;
|
||||
}
|
||||
@Override
|
||||
public int docID() {
|
||||
return collectDocID;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
@Override
|
||||
public int freq() {
|
||||
return 1+dims.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return collectScore;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return baseScorer.cost();
|
||||
}
|
||||
@Override
|
||||
public long cost() {
|
||||
return baseScorer.cost();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
return Collections.singletonList(new ChildScorer(baseScorer, "MUST"));
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
return Collections.singletonList(new ChildScorer(baseScorer, "MUST"));
|
||||
}
|
||||
}
|
||||
|
||||
static class DocsAndCost implements Comparable<DocsAndCost> {
|
||||
|
|
|
@ -17,6 +17,10 @@ package org.apache.lucene.search.join;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.DocsEnum;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
|
@ -30,16 +34,13 @@ 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.Weight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.BytesRefHash;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
|
||||
class TermsIncludingScoreQuery extends Query {
|
||||
|
||||
final String field;
|
||||
|
@ -131,11 +132,9 @@ class TermsIncludingScoreQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
SVInnerScorer scorer = (SVInnerScorer) scorer(context, false, false, context.reader().getLiveDocs());
|
||||
SVInnerScorer scorer = (SVInnerScorer) topScorer(context, false, null);
|
||||
if (scorer != null) {
|
||||
if (scorer.advanceForExplainOnly(doc) == doc) {
|
||||
return scorer.explain();
|
||||
}
|
||||
return scorer.explain(doc);
|
||||
}
|
||||
return new ComplexExplanation(false, 0.0f, "Not a match");
|
||||
}
|
||||
|
@ -163,7 +162,7 @@ class TermsIncludingScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
Terms terms = context.reader().terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
|
@ -173,23 +172,80 @@ class TermsIncludingScoreQuery extends Query {
|
|||
final long cost = context.reader().maxDoc() * terms.size();
|
||||
|
||||
segmentTermsEnum = terms.iterator(segmentTermsEnum);
|
||||
if (scoreDocsInOrder) {
|
||||
if (multipleValuesPerDocument) {
|
||||
return new MVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
|
||||
} else {
|
||||
return new SVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
|
||||
}
|
||||
} else if (multipleValuesPerDocument) {
|
||||
return new MVInnerScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
|
||||
if (multipleValuesPerDocument) {
|
||||
return new MVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
|
||||
} else {
|
||||
return new SVInnerScorer(this, acceptDocs, segmentTermsEnum, cost);
|
||||
return new SVInOrderScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
|
||||
|
||||
if (scoreDocsInOrder) {
|
||||
return super.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
} else {
|
||||
Terms terms = context.reader().terms(field);
|
||||
if (terms == null) {
|
||||
return null;
|
||||
}
|
||||
// what is the runtime...seems ok?
|
||||
final long cost = context.reader().maxDoc() * terms.size();
|
||||
|
||||
segmentTermsEnum = terms.iterator(segmentTermsEnum);
|
||||
// Optimized impls that take advantage of docs
|
||||
// being allowed to be out of order:
|
||||
if (multipleValuesPerDocument) {
|
||||
return new MVInnerScorer(this, acceptDocs, segmentTermsEnum, context.reader().maxDoc(), cost);
|
||||
} else {
|
||||
return new SVInnerScorer(this, acceptDocs, segmentTermsEnum, cost);
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
final static class FakeScorer extends Scorer {
|
||||
float score;
|
||||
int doc;
|
||||
|
||||
public FakeScorer() {
|
||||
super(null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() {
|
||||
return score;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
|
||||
// This impl assumes that the 'join' values are used uniquely per doc per field. Used for one to many relations.
|
||||
class SVInnerScorer extends Scorer {
|
||||
class SVInnerScorer extends TopScorer {
|
||||
|
||||
final BytesRef spare = new BytesRef();
|
||||
final Bits acceptDocs;
|
||||
|
@ -202,8 +258,9 @@ class TermsIncludingScoreQuery extends Query {
|
|||
int scoreUpto;
|
||||
int doc;
|
||||
|
||||
// nocommit can we embed this into a BQ and get BS1?
|
||||
|
||||
SVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, long cost) {
|
||||
super(weight);
|
||||
this.acceptDocs = acceptDocs;
|
||||
this.termsEnum = termsEnum;
|
||||
this.cost = cost;
|
||||
|
@ -211,25 +268,20 @@ class TermsIncludingScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
collector.setScorer(this);
|
||||
for (int doc = nextDocOutOfOrder(); doc != NO_MORE_DOCS; doc = nextDocOutOfOrder()) {
|
||||
collector.collect(doc);
|
||||
public boolean score(Collector collector, int max) throws IOException {
|
||||
FakeScorer fakeScorer = new FakeScorer();
|
||||
collector.setScorer(fakeScorer);
|
||||
if (doc == -1) {
|
||||
doc = nextDocOutOfOrder();
|
||||
}
|
||||
while(doc < max) {
|
||||
fakeScorer.doc = doc;
|
||||
fakeScorer.score = scores[ords[scoreUpto]];
|
||||
collector.collect(doc);
|
||||
doc = nextDocOutOfOrder();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public float score() throws IOException {
|
||||
return scores[ords[scoreUpto]];
|
||||
}
|
||||
|
||||
Explanation explain() throws IOException {
|
||||
return new ComplexExplanation(true, score(), "Score based on join value " + termsEnum.term().utf8ToString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docID() {
|
||||
return doc;
|
||||
return doc != DocsEnum.NO_MORE_DOCS;
|
||||
}
|
||||
|
||||
int nextDocOutOfOrder() throws IOException {
|
||||
|
@ -258,17 +310,7 @@ class TermsIncludingScoreQuery extends Query {
|
|||
return docsEnum.nextDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
throw new UnsupportedOperationException("nextDoc() isn't supported because doc ids are emitted out of order");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
throw new UnsupportedOperationException("advance() isn't supported because doc ids are emitted out of order");
|
||||
}
|
||||
|
||||
private int advanceForExplainOnly(int target) throws IOException {
|
||||
private Explanation explain(int target) throws IOException {
|
||||
int docId;
|
||||
do {
|
||||
docId = nextDocOutOfOrder();
|
||||
|
@ -283,17 +325,8 @@ class TermsIncludingScoreQuery extends Query {
|
|||
}
|
||||
docsEnum = null; // goto the next ord.
|
||||
} while (docId != DocIdSetIterator.NO_MORE_DOCS);
|
||||
return docId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int freq() {
|
||||
return 1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long cost() {
|
||||
return cost;
|
||||
return new ComplexExplanation(true, scores[ords[scoreUpto]], "Score based on join value " + termsEnum.term().utf8ToString());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -124,11 +124,9 @@ public class ToChildBlockJoinQuery extends Query {
|
|||
// NOTE: acceptDocs applies (and is checked) only in the
|
||||
// child document space
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext readerContext, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext readerContext, Bits acceptDocs) throws IOException {
|
||||
|
||||
// Pass scoreDocsInOrder true, topScorer false to our sub:
|
||||
final Scorer parentScorer = parentWeight.scorer(readerContext, true, false, null);
|
||||
final Scorer parentScorer = parentWeight.scorer(readerContext, null);
|
||||
|
||||
if (parentScorer == null) {
|
||||
// No matches
|
||||
|
|
|
@ -17,6 +17,12 @@ package org.apache.lucene.search.join;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
|
@ -35,12 +41,6 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.FixedBitSet;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Locale;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* This query requires that you index
|
||||
* children and parent docs as a single block, using the
|
||||
|
@ -158,12 +158,9 @@ public class ToParentBlockJoinQuery extends Query {
|
|||
// NOTE: acceptDocs applies (and is checked) only in the
|
||||
// parent document space
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext readerContext, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
|
||||
// Pass scoreDocsInOrder true, topScorer false to our sub and the live docs:
|
||||
final Scorer childScorer = childWeight.scorer(readerContext, true, false, readerContext.reader().getLiveDocs());
|
||||
public Scorer scorer(AtomicReaderContext readerContext, Bits acceptDocs) throws IOException {
|
||||
|
||||
final Scorer childScorer = childWeight.scorer(readerContext, readerContext.reader().getLiveDocs());
|
||||
if (childScorer == null) {
|
||||
// No matches
|
||||
return null;
|
||||
|
@ -195,7 +192,7 @@ public class ToParentBlockJoinQuery extends Query {
|
|||
|
||||
@Override
|
||||
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
|
||||
BlockJoinScorer scorer = (BlockJoinScorer) scorer(context, true, false, context.reader().getLiveDocs());
|
||||
BlockJoinScorer scorer = (BlockJoinScorer) scorer(context, context.reader().getLiveDocs());
|
||||
if (scorer != null && scorer.advance(doc) == doc) {
|
||||
return scorer.explain(context.docBase);
|
||||
}
|
||||
|
|
|
@ -1103,7 +1103,6 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
parentDoc = s.doc(qGroup.groupValue);
|
||||
assertEquals("Lisa", parentDoc.get("name"));
|
||||
|
||||
|
||||
r.close();
|
||||
dir.close();
|
||||
}
|
||||
|
@ -1126,7 +1125,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
|
||||
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
||||
Weight weight = s.createNormalizedWeight(q);
|
||||
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), true, true, null);
|
||||
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
|
||||
assertEquals(1, disi.advance(1));
|
||||
r.close();
|
||||
dir.close();
|
||||
|
@ -1160,7 +1159,7 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
|
||||
ToParentBlockJoinQuery q = new ToParentBlockJoinQuery(tq, parentFilter, ScoreMode.Avg);
|
||||
Weight weight = s.createNormalizedWeight(q);
|
||||
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), true, true, null);
|
||||
DocIdSetIterator disi = weight.scorer(s.getIndexReader().leaves().get(0), null);
|
||||
assertEquals(2, disi.advance(0));
|
||||
r.close();
|
||||
dir.close();
|
||||
|
@ -1200,7 +1199,6 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
ToParentBlockJoinQuery childJoinQuery = new ToParentBlockJoinQuery(childQuery, parentsFilter, ScoreMode.Avg);
|
||||
|
||||
ToParentBlockJoinCollector c = new ToParentBlockJoinCollector(Sort.RELEVANCE, 2, true, true);
|
||||
|
||||
s.search(childJoinQuery, c);
|
||||
|
||||
//Get all child documents within groups
|
||||
|
@ -1312,7 +1310,6 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
GroupDocs<Integer> group = groups.groups[0];
|
||||
StoredDocument doc = r.document(group.groupValue.intValue());
|
||||
assertEquals("0", doc.get("parentID"));
|
||||
System.out.println("group: " + group);
|
||||
|
||||
group = groups.groups[1];
|
||||
doc = r.document(group.groupValue.intValue());
|
||||
|
@ -1378,7 +1375,6 @@ public class TestBlockJoin extends LuceneTestCase {
|
|||
GroupDocs<Integer> group = groups.groups[0];
|
||||
StoredDocument doc = r.document(group.groupValue.intValue());
|
||||
assertEquals("0", doc.get("parentID"));
|
||||
System.out.println("group: " + group);
|
||||
|
||||
group = groups.groups[1];
|
||||
doc = r.document(group.groupValue.intValue());
|
||||
|
|
|
@ -234,20 +234,14 @@ public class CustomScoreQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
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, true, false, acceptDocs);
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
Scorer subQueryScorer = subQueryWeight.scorer(context, 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, true, topScorer, acceptDocs);
|
||||
valSrcScorers[i] = valSrcWeights[i].scorer(context, acceptDocs);
|
||||
}
|
||||
return new CustomScorer(CustomScoreQuery.this.getCustomScoreProvider(context), this, queryWeight, subQueryScorer, valSrcScorers);
|
||||
}
|
||||
|
|
|
@ -97,11 +97,9 @@ public class BoostedQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
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) {
|
||||
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
|
||||
Scorer subQueryScorer = qWeight.scorer(context, acceptDocs);
|
||||
if (subQueryScorer == null) {
|
||||
return null;
|
||||
}
|
||||
return new BoostedQuery.CustomScorer(context, this, getBoost(), subQueryScorer, boostVal);
|
||||
|
|
|
@ -90,14 +90,13 @@ public class FunctionQuery extends Query {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, 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, true, true, context.reader().getLiveDocs())).explain(doc);
|
||||
return ((AllScorer)scorer(context, context.reader().getLiveDocs())).explain(doc);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -123,7 +123,7 @@ class QueryDocValues extends FloatDocValues {
|
|||
try {
|
||||
if (doc < lastDocRequested) {
|
||||
if (noMatches) return defVal;
|
||||
scorer = weight.scorer(readerContext, true, false, acceptDocs);
|
||||
scorer = weight.scorer(readerContext, acceptDocs);
|
||||
if (scorer==null) {
|
||||
noMatches = true;
|
||||
return defVal;
|
||||
|
@ -154,7 +154,7 @@ class QueryDocValues extends FloatDocValues {
|
|||
try {
|
||||
if (doc < lastDocRequested) {
|
||||
if (noMatches) return false;
|
||||
scorer = weight.scorer(readerContext, true, false, acceptDocs);
|
||||
scorer = weight.scorer(readerContext, acceptDocs);
|
||||
scorerDoc = -1;
|
||||
if (scorer==null) {
|
||||
noMatches = true;
|
||||
|
@ -212,7 +212,7 @@ class QueryDocValues extends FloatDocValues {
|
|||
mval.exists = false;
|
||||
return;
|
||||
}
|
||||
scorer = weight.scorer(readerContext, true, false, acceptDocs);
|
||||
scorer = weight.scorer(readerContext, acceptDocs);
|
||||
scorerDoc = -1;
|
||||
if (scorer==null) {
|
||||
noMatches = true;
|
||||
|
|
|
@ -32,31 +32,23 @@ import org.apache.lucene.util.VirtualMethod;
|
|||
/** Wraps a Scorer with additional checks */
|
||||
public class AssertingScorer extends Scorer {
|
||||
|
||||
enum TopScorer {
|
||||
YES, NO, UNKNOWN;
|
||||
}
|
||||
|
||||
private static final VirtualMethod<Scorer> SCORE_COLLECTOR = new VirtualMethod<Scorer>(Scorer.class, "score", Collector.class);
|
||||
private static final VirtualMethod<Scorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<Scorer>(Scorer.class, "score", Collector.class, int.class, int.class);
|
||||
//private static final VirtualMethod<Scorer> SCORE_COLLECTOR = new VirtualMethod<Scorer>(Scorer.class, "score", Collector.class);
|
||||
//private static final VirtualMethod<Scorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<Scorer>(Scorer.class, "score", Collector.class, int.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<Scorer, WeakReference<AssertingScorer>> ASSERTING_INSTANCES = Collections.synchronizedMap(new WeakHashMap<Scorer, WeakReference<AssertingScorer>>());
|
||||
|
||||
private static Scorer wrap(Random random, Scorer other, TopScorer topScorer, boolean inOrder) {
|
||||
public static Scorer wrap(Random random, Scorer other) {
|
||||
if (other == null || other instanceof AssertingScorer) {
|
||||
return other;
|
||||
}
|
||||
final AssertingScorer assertScorer = new AssertingScorer(random, other, topScorer, inOrder);
|
||||
final AssertingScorer assertScorer = new AssertingScorer(random, other);
|
||||
ASSERTING_INSTANCES.put(other, new WeakReference<AssertingScorer>(assertScorer));
|
||||
return assertScorer;
|
||||
}
|
||||
|
||||
static Scorer wrap(Random random, Scorer other, boolean topScorer, boolean inOrder) {
|
||||
return wrap(random, other, topScorer ? TopScorer.YES : TopScorer.NO, inOrder);
|
||||
}
|
||||
|
||||
static Scorer getAssertingScorer(Random random, Scorer other) {
|
||||
if (other == null || other instanceof AssertingScorer) {
|
||||
return other;
|
||||
|
@ -68,7 +60,7 @@ public class AssertingScorer extends Scorer {
|
|||
// scorer1.score(collector) calls
|
||||
// collector.setScorer(scorer2) with scorer1 != scorer2, such as
|
||||
// BooleanScorer. In that case we can't enable all assertions
|
||||
return new AssertingScorer(random, other, TopScorer.UNKNOWN, false);
|
||||
return new AssertingScorer(random, other);
|
||||
} else {
|
||||
return assertingScorer;
|
||||
}
|
||||
|
@ -77,20 +69,12 @@ public class AssertingScorer extends Scorer {
|
|||
final Random random;
|
||||
final Scorer in;
|
||||
final AssertingAtomicReader.AssertingDocsEnum docsEnumIn;
|
||||
final TopScorer topScorer;
|
||||
final boolean inOrder;
|
||||
final boolean canCallNextDoc;
|
||||
|
||||
private AssertingScorer(Random random, Scorer in, TopScorer topScorer, boolean inOrder) {
|
||||
private AssertingScorer(Random random, Scorer in) {
|
||||
super(in.weight);
|
||||
this.random = random;
|
||||
this.in = in;
|
||||
this.topScorer = topScorer;
|
||||
this.inOrder = inOrder;
|
||||
this.docsEnumIn = new AssertingAtomicReader.AssertingDocsEnum(in, topScorer == TopScorer.NO);
|
||||
this.canCallNextDoc = topScorer != TopScorer.YES // not a top scorer
|
||||
|| !SCORE_COLLECTOR_RANGE.isOverriddenAsOf(in.getClass()) // the default impl relies upon nextDoc()
|
||||
|| !SCORE_COLLECTOR.isOverriddenAsOf(in.getClass()); // the default impl relies upon nextDoc()
|
||||
this.docsEnumIn = new AssertingAtomicReader.AssertingDocsEnum(in);
|
||||
}
|
||||
|
||||
public Scorer getIn() {
|
||||
|
@ -116,9 +100,10 @@ public class AssertingScorer extends Scorer {
|
|||
return score;
|
||||
}
|
||||
|
||||
// nocommit make an AssertingTopScorer
|
||||
/*
|
||||
@Override
|
||||
public void score(Collector collector) throws IOException {
|
||||
assert topScorer != TopScorer.NO;
|
||||
if (SCORE_COLLECTOR.isOverriddenAsOf(this.in.getClass())) {
|
||||
if (random.nextBoolean()) {
|
||||
try {
|
||||
|
@ -148,6 +133,7 @@ public class AssertingScorer extends Scorer {
|
|||
return super.score(collector, max, firstDocID);
|
||||
}
|
||||
}
|
||||
*/
|
||||
|
||||
@Override
|
||||
public Collection<ChildScorer> getChildren() {
|
||||
|
@ -171,13 +157,11 @@ public class AssertingScorer extends Scorer {
|
|||
|
||||
@Override
|
||||
public int nextDoc() throws IOException {
|
||||
assert canCallNextDoc : "top scorers should not call nextDoc()";
|
||||
return docsEnumIn.nextDoc();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int advance(int target) throws IOException {
|
||||
assert canCallNextDoc : "top scorers should not call advance(target)";
|
||||
return docsEnumIn.advance(target);
|
||||
}
|
||||
|
||||
|
|
|
@ -58,13 +58,21 @@ class AssertingWeight extends Weight {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
|
||||
boolean topScorer, Bits acceptDocs) throws IOException {
|
||||
public Scorer scorer(AtomicReaderContext context, 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.
|
||||
final boolean inOrder = scoreDocsInOrder || !scoresDocsOutOfOrder();
|
||||
final Scorer inScorer = in.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
|
||||
return AssertingScorer.wrap(new Random(random.nextLong()), inScorer, topScorer, inOrder);
|
||||
final Scorer inScorer = in.scorer(context, acceptDocs);
|
||||
return AssertingScorer.wrap(new Random(random.nextLong()), inScorer);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TopScorer topScorer(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.
|
||||
// nocommit add wrapping:
|
||||
TopScorer inScorer = in.topScorer(context, scoreDocsInOrder, acceptDocs);
|
||||
//return AssertingScorer.wrap(new Random(random.nextLong()), inScorer);
|
||||
return inScorer;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -267,7 +267,7 @@ public class QueryUtils {
|
|||
if (scorer == null) {
|
||||
Weight w = s.createNormalizedWeight(q);
|
||||
AtomicReaderContext context = readerContextArray.get(leafPtr);
|
||||
scorer = w.scorer(context, true, false, context.reader().getLiveDocs());
|
||||
scorer = w.scorer(context, context.reader().getLiveDocs());
|
||||
}
|
||||
|
||||
int op = order[(opidx[0]++) % order.length];
|
||||
|
@ -314,7 +314,7 @@ public class QueryUtils {
|
|||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q);
|
||||
AtomicReaderContext ctx = (AtomicReaderContext)indexSearcher.getTopReaderContext();
|
||||
Scorer scorer = w.scorer(ctx, true, false, ctx.reader().getLiveDocs());
|
||||
Scorer scorer = w.scorer(ctx, 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);
|
||||
|
@ -341,7 +341,7 @@ public class QueryUtils {
|
|||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q);
|
||||
AtomicReaderContext ctx = previousReader.getContext();
|
||||
Scorer scorer = w.scorer(ctx, true, false, ctx.reader().getLiveDocs());
|
||||
Scorer scorer = w.scorer(ctx, 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);
|
||||
|
@ -372,7 +372,7 @@ public class QueryUtils {
|
|||
long startMS = System.currentTimeMillis();
|
||||
for (int i=lastDoc[0]+1; i<=doc; i++) {
|
||||
Weight w = s.createNormalizedWeight(q);
|
||||
Scorer scorer = w.scorer(context.get(leafPtr), true, false, liveDocs);
|
||||
Scorer scorer = w.scorer(context.get(leafPtr), 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();
|
||||
|
@ -400,7 +400,7 @@ public class QueryUtils {
|
|||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
||||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q);
|
||||
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, previousReader.getLiveDocs());
|
||||
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), 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);
|
||||
|
@ -425,7 +425,7 @@ public class QueryUtils {
|
|||
IndexSearcher indexSearcher = LuceneTestCase.newSearcher(previousReader);
|
||||
indexSearcher.setSimilarity(s.getSimilarity());
|
||||
Weight w = indexSearcher.createNormalizedWeight(q);
|
||||
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), true, false, previousReader.getLiveDocs());
|
||||
Scorer scorer = w.scorer((AtomicReaderContext)indexSearcher.getTopReaderContext(), 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);
|
||||
|
|
Loading…
Reference in New Issue