LUCENE-5487: separate Weight.bulkScorer and Weight.scorer

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1576487 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2014-03-11 20:04:53 +00:00
commit 40fd13c0ad
60 changed files with 1270 additions and 859 deletions

View File

@ -120,6 +120,12 @@ API Changes
a boolean that indicates if a new merge was found in the caller thread before
the scheduler was called. (Simon Willnauer)
* LUCENE-5487: Separated bulk scorer (new Weight.bulkScorer method) from
normal scoring (Weight.scorer) for those queries that can do bulk
scoring more efficiently, e.g. BooleanQuery in some cases. This
also simplified the Weight.scorer API by removing the two confusing
booleans. (Robert Muir, Uwe Schindler, Mike McCandless)
Optimizations
* LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads

View File

@ -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 BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder,
Bits acceptDocs) throws IOException {
if (scoreDocsInOrder || minNrShouldMatch > 1) {
// TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
// but the same is even true of pure conjunctions...
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
}
List<BulkScorer> prohibited = new ArrayList<BulkScorer>();
List<BulkScorer> optional = new ArrayList<BulkScorer>();
Iterator<BooleanClause> cIter = clauses.iterator();
for (Weight w : weights) {
BooleanClause c = cIter.next();
BulkScorer subScorer = w.bulkScorer(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.bulkScorer(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;
@ -373,9 +394,14 @@ public class BooleanQuery extends Query implements Iterable<BooleanClause> {
@Override
public boolean scoresDocsOutOfOrder() {
if (minNrShouldMatch > 1) {
// BS2 (in-order) will be used by scorer()
return false;
}
for (BooleanClause c : clauses) {
if (c.isRequired()) {
return false; // BS2 (in-order) will be used by scorer()
// BS2 (in-order) will be used by scorer()
return false;
}
}

View File

@ -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 BulkScorer {
private static final class BooleanScorerCollector extends Collector {
private BucketTable bucketTable;
@ -108,38 +109,6 @@ final class BooleanScorer extends Scorer {
}
// An internal class which is used in score(Collector, int) for setting the
// current score. This is required since Collector exposes a setScorer method
// and implementations that need the score will call scorer.score().
// Therefore the only methods that are implemented are score() and doc().
private static final class BucketScorer extends Scorer {
double score;
int doc = NO_MORE_DOCS;
int freq;
public BucketScorer(Weight weight) { super(weight); }
@Override
public int advance(int target) { return NO_MORE_DOCS; }
@Override
public int docID() { return doc; }
@Override
public int freq() { return freq; }
@Override
public int nextDoc() { return NO_MORE_DOCS; }
@Override
public float score() { return (float)score; }
@Override
public long cost() { return 1; }
}
static final class Bucket {
int doc = -1; // tells if bucket is valid
double score; // incremental score
@ -175,19 +144,21 @@ final class BooleanScorer extends Scorer {
}
static final class SubScorer {
public Scorer scorer;
public BulkScorer 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(BulkScorer 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 +177,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<BulkScorer> optionalScorers, List<BulkScorer> prohibitedScorers, int maxCoord) throws IOException {
this.minNrShouldMatch = minNrShouldMatch;
this.weight = weight;
for (BulkScorer 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 (BulkScorer scorer : prohibitedScorers) {
scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
}
coordFactors = new float[optionalScorers.size() + 1];
@ -234,17 +199,15 @@ 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);
FakeScorer fs = new FakeScorer();
// The internal loop will set the score and doc before calling collect.
collector.setScorer(bs);
collector.setScorer(fs);
do {
bucketTable.first = null;
@ -263,7 +226,7 @@ final class BooleanScorer extends Scorer {
// that should work)... but in theory an outside
// app could pass a different max so we must check
// it:
if (current.doc >= max){
if (current.doc >= max) {
tmp = current;
current = current.next;
tmp.next = bucketTable.first;
@ -272,9 +235,9 @@ final class BooleanScorer extends Scorer {
}
if (current.coord >= minNrShouldMatch) {
bs.score = current.score * coordFactors[current.coord];
bs.doc = current.doc;
bs.freq = current.coord;
fs.score = (float) (current.score * coordFactors[current.coord]);
fs.doc = current.doc;
fs.freq = current.coord;
collector.collect(current.doc);
}
}
@ -292,9 +255,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,42 +266,7 @@ 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);
}
@Override
public String toString() {
StringBuilder buffer = new StringBuilder();
@ -350,9 +278,4 @@ final class BooleanScorer extends Scorer {
buffer.append(")");
return buffer.toString();
}
@Override
public Collection<ChildScorer> getChildren() {
throw new UnsupportedOperationException();
}
}

View File

@ -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;

View File

@ -0,0 +1,46 @@
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;
/** This class is used to score a range of documents at
* once, and is returned by {@link Weight#bulkScorer}. Only
* queries that have a more optimized means of scoring
* across a range of documents need to override this.
* Otherwise, a default implementation is wrapped around
* the {@link Scorer} returned by {@link Weight#scorer}. */
public abstract class BulkScorer {
/** Scores and collects all matching documents.
* @param collector The collector to which all matching documents are passed.
*/
public void score(Collector collector) throws IOException {
score(collector, Integer.MAX_VALUE);
}
/**
* Collects matching documents in a range.
*
* @param collector The collector to which all matching documents are passed.
* @param max Score up to, but not including, this doc
* @return true if more matching documents may remain.
*/
public abstract boolean score(Collector collector, int max) throws IOException;
}

View File

@ -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 BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
final DocIdSetIterator disi;
if (filter != null) {
assert query == null;
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
} else {
assert query != null && innerWeight != null;
BulkScorer bulkScorer = innerWeight.bulkScorer(context, scoreDocsInOrder, acceptDocs);
if (bulkScorer == null) {
return null;
}
return new ConstantBulkScorer(bulkScorer, 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 BulkScorer} so that if the CSQ
* wraps a query with its own optimized top-level
* scorer (e.g. BooleanScorer) we can use that
* top-level scorer. */
protected class ConstantBulkScorer extends BulkScorer {
final BulkScorer bulkScorer;
final Weight weight;
final float theScore;
public ConstantBulkScorer(BulkScorer bulkScorer, Weight weight, float theScore) {
this.bulkScorer = bulkScorer;
this.weight = weight;
this.theScore = theScore;
}
@Override
public boolean score(Collector collector, int max) throws IOException {
return bulkScorer.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();
}
}
}

View File

@ -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);

View File

@ -0,0 +1,72 @@
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.util.Collection;
/** Used by {@link BulkScorer}s that need to pass a {@link
* Scorer} to {@link Collector#setScorer}. */
final class FakeScorer extends Scorer {
float score;
int doc = -1;
int freq = 1;
public FakeScorer() {
super(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() {
return freq;
}
@Override
public int nextDoc() {
throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
}
@Override
public float score() {
return score;
}
@Override
public long cost() {
return 1;
}
@Override
public Weight getWeight() {
throw new UnsupportedOperationException();
}
@Override
public Collection<ChildScorer> getChildren() {
throw new UnsupportedOperationException();
}
}

View File

@ -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,16 +124,30 @@ 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, Bits acceptDocs) throws IOException {
assert filter != null;
final DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
if (filterDocIdSet == null) {
// this means the filter does not accept any documents.
return null;
}
return strategy.filteredScorer(context, scoreDocsInOrder, topScorer, weight, filterDocIdSet);
return strategy.filteredScorer(context, weight, filterDocIdSet);
}
// return a filtering top scorer
@Override
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
assert filter != null;
DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
if (filterDocIdSet == null) {
// this means the filter does not accept any documents.
return null;
}
return strategy.filteredBulkScorer(context, weight, scoreDocsInOrder, filterDocIdSet);
}
};
}
@ -147,37 +161,20 @@ public class FilteredQuery extends Query {
private static final class QueryFirstScorer extends Scorer {
private final Scorer scorer;
private int scorerDoc = -1;
private Bits filterbits;
private final Bits filterBits;
protected QueryFirstScorer(Weight weight, Bits filterBits, Scorer other) {
super(weight);
this.scorer = other;
this.filterbits = filterBits;
this.filterBits = filterBits;
}
// 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,
// so we can use the wrapped scorer directly:
collector.setScorer(scorer);
for (;;) {
final int scorerDoc = scorer.nextDoc();
if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
if (filterbits.get(scorerDoc)) {
collector.collect(scorerDoc);
}
}
}
@Override
public int nextDoc() throws IOException {
int doc;
for(;;) {
doc = scorer.nextDoc();
if (doc == Scorer.NO_MORE_DOCS || filterbits.get(doc)) {
if (doc == Scorer.NO_MORE_DOCS || filterBits.get(doc)) {
return scorerDoc = doc;
}
}
@ -186,7 +183,7 @@ public class FilteredQuery extends Query {
@Override
public int advance(int target) throws IOException {
int doc = scorer.advance(target);
if (doc != Scorer.NO_MORE_DOCS && !filterbits.get(doc)) {
if (doc != Scorer.NO_MORE_DOCS && !filterBits.get(doc)) {
return scorerDoc = nextDoc();
} else {
return scorerDoc = doc;
@ -216,6 +213,40 @@ public class FilteredQuery extends Query {
return scorer.cost();
}
}
private static class QueryFirstBulkScorer extends BulkScorer {
private final Scorer scorer;
private final Bits filterBits;
public QueryFirstBulkScorer(Scorer scorer, Bits filterBits) {
this.scorer = scorer;
this.filterBits = filterBits;
}
@Override
public boolean score(Collector collector, int maxDoc) throws IOException {
// the normalization trick already applies the boost of this query,
// so we can use the wrapped scorer directly:
collector.setScorer(scorer);
if (scorer.docID() == -1) {
scorer.nextDoc();
}
while (true) {
final int scorerDoc = scorer.docID();
if (scorerDoc < maxDoc) {
if (filterBits.get(scorerDoc)) {
collector.collect(scorerDoc);
}
scorer.nextDoc();
} else {
break;
}
}
return scorer.docID() != Scorer.NO_MORE_DOCS;
}
}
/**
* A Scorer that uses a "leap-frog" approach (also called "zig-zag join"). The scorer and the filter
@ -236,32 +267,7 @@ public class FilteredQuery extends Query {
this.secondary = secondary;
this.scorer = scorer;
}
// 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,
// so we can use the wrapped scorer directly:
collector.setScorer(scorer);
int primDoc = primaryNext();
int secDoc = secondary.advance(primDoc);
for (;;) {
if (primDoc == secDoc) {
// Check if scorer has exhausted, only before collecting.
if (primDoc == DocIdSetIterator.NO_MORE_DOCS) {
break;
}
collector.collect(primDoc);
primDoc = primary.nextDoc();
secDoc = secondary.advance(primDoc);
} else if (secDoc > primDoc) {
primDoc = primary.advance(secDoc);
} else {
secDoc = secondary.advance(primDoc);
}
}
}
private final int advanceToNextCommonDoc() throws IOException {
for (;;) {
if (secondaryDoc < primaryDoc) {
@ -472,18 +478,6 @@ public class FilteredQuery extends Query {
*
* @param context
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
* @param scoreDocsInOrder
* specifies whether in-order scoring of documents is required. Note
* that if set to false (i.e., out-of-order scoring is required),
* this method can return whatever scoring mode it supports, as every
* in-order scorer is also an out-of-order one. However, an
* out-of-order scorer may not support {@link Scorer#nextDoc()}
* and/or {@link Scorer#advance(int)}, therefore it is recommended to
* request an in-order scorer if use of these methods is required.
* @param topScorer
* if true, {@link Scorer#score(Collector)} will be called; if false,
* {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
* be called.
* @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
* @param docIdSet the filter {@link DocIdSet} to apply
* @return a filtered scorer
@ -491,8 +485,30 @@ 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;
/**
* Returns a filtered {@link BulkScorer} based on this
* strategy. This is an optional method: the default
* implementation just calls {@link #filteredScorer} and
* wraps that into a BulkScorer.
*
* @param context
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
* @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
* @param docIdSet the filter {@link DocIdSet} to apply
* @return a filtered top scorer
*/
public BulkScorer filteredBulkScorer(AtomicReaderContext context,
Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
Scorer scorer = filteredScorer(context, weight, docIdSet);
if (scorer == null) {
return null;
}
// This impl always scores docs in order, so we can
// ignore scoreDocsInOrder:
return new Weight.DefaultBulkScorer(scorer);
}
}
/**
@ -506,7 +522,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 +539,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,25 +575,27 @@ 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 (scorer == null) {
return null;
}
if (scorerFirst) {
return (scorer == null) ? null : new LeapFrogScorer(weight, scorer, filterIter, scorer);
return new LeapFrogScorer(weight, scorer, filterIter, scorer);
} else {
return (scorer == null) ? null : new LeapFrogScorer(weight, filterIter, scorer, scorer);
return new LeapFrogScorer(weight, filterIter, scorer, scorer);
}
}
}
/**
@ -596,16 +614,33 @@ 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);
// Filter does not provide random-access Bits; we
// must fallback to leapfrog:
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);
}
@Override
public BulkScorer filteredBulkScorer(final AtomicReaderContext context,
Weight weight,
boolean scoreDocsInOrder, // ignored (we always top-score in order)
DocIdSet docIdSet) throws IOException {
Bits filterAcceptDocs = docIdSet.bits();
if (filterAcceptDocs == null) {
// Filter does not provide random-access Bits; we
// must fallback to leapfrog:
return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, scoreDocsInOrder, docIdSet);
}
final Scorer scorer = weight.scorer(context, null);
return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs);
}
}
}

View File

@ -607,7 +607,7 @@ public class IndexSearcher {
// continue with the following leaf
continue;
}
Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs());
BulkScorer scorer = weight.bulkScorer(ctx, !collector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
if (scorer != null) {
try {
scorer.score(collector);
@ -768,45 +768,6 @@ public class IndexSearcher {
this.doMaxScore = doMaxScore;
}
private final class FakeScorer extends Scorer {
float score;
int doc;
public FakeScorer() {
super(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 score;
}
@Override
public long cost() {
return 1;
}
}
private final FakeScorer fakeScorer = new FakeScorer();
@Override

View File

@ -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);
}

View File

@ -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) {

View File

@ -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) {

View File

@ -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; }

View File

@ -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);

View File

@ -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

View File

@ -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) {

View File

@ -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;
@ -35,8 +36,7 @@ import org.apache.lucene.util.Bits;
* {@link AtomicReader} dependent state should reside in the {@link Scorer}.
* <p>
* Since {@link Weight} creates {@link Scorer} instances for a given
* {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext,
* boolean, boolean, Bits)})
* {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, Bits)})
* callers must maintain the relationship between the searcher's top-level
* {@link IndexReaderContext} and the context used to create a {@link Scorer}.
* <p>
@ -51,7 +51,7 @@ import org.apache.lucene.util.Bits;
* <li>The query normalization factor is passed to {@link #normalize(float, float)}. At
* this point the weighting is complete.
* <li>A <code>Scorer</code> is constructed by
* {@link #scorer(AtomicReaderContext, boolean, boolean, Bits)}.
* {@link #scorer(AtomicReaderContext, Bits)}.
* </ol>
*
* @since 2.9
@ -91,18 +91,6 @@ public abstract class Weight {
*
* @param context
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
* @param scoreDocsInOrder
* specifies whether in-order scoring of documents is required. Note
* that if set to false (i.e., out-of-order scoring is required),
* this method can return whatever scoring mode it supports, as every
* in-order scorer is also an out-of-order one. However, an
* out-of-order scorer may not support {@link Scorer#nextDoc()}
* and/or {@link Scorer#advance(int)}, therefore it is recommended to
* request an in-order scorer if use of these methods is required.
* @param topScorer
* if true, {@link Scorer#score(Collector)} will be called; if false,
* {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
* be called.
* @param acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
@ -110,19 +98,89 @@ 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;
/**
* Optional method, to return a {@link BulkScorer} to
* score the query and send hits to a {@link Collector}.
* Only queries that have a different top-level approach
* need to override this; the default implementation
* pulls a normal {@link Scorer} and iterates and
* collects the resulting hits.
*
* @param context
* the {@link AtomicReaderContext} for which to return the {@link Scorer}.
* @param scoreDocsInOrder
* specifies whether in-order scoring of documents is required. Note
* that if set to false (i.e., out-of-order scoring is required),
* this method can return whatever scoring mode it supports, as every
* in-order scorer is also an out-of-order one. However, an
* out-of-order scorer may not support {@link Scorer#nextDoc()}
* and/or {@link Scorer#advance(int)}, therefore it is recommended to
* request an in-order scorer if use of these
* methods is required.
* @param acceptDocs
* Bits that represent the allowable docs to match (typically deleted docs
* but possibly filtering other documents)
*
* @return a {@link BulkScorer} which scores documents and
* passes them to a collector.
* @throws IOException if there is a low-level I/O error
*/
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
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 DefaultBulkScorer(scorer);
}
/** Just wraps a Scorer and performs top scoring using it. */
static class DefaultBulkScorer extends BulkScorer {
private final Scorer scorer;
public DefaultBulkScorer(Scorer scorer) {
assert scorer != null;
this.scorer = scorer;
}
@Override
public boolean score(Collector collector, int max) throws IOException {
// TODO: this may be sort of weird, when we are
// embedded in a BooleanScorer, because we are
// called for every chunk of 2048 documents. But,
// then, scorer is a FakeScorer in that case, so any
// Collector doing something "interesting" in
// setScorer will be forced to use BS2 anyways:
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
* method is used in conjunction with {@link Collector}'s
* {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
* {@link #scorer(AtomicReaderContext, boolean, boolean, Bits)} to
* {@link #bulkScorer(AtomicReaderContext, boolean, Bits)} to
* create a matching {@link Scorer} instance for a given {@link Collector}, or
* vice versa.
* <p>
* <b>NOTE:</b> the default implementation returns <code>false</code>, i.e.
* the <code>Scorer</code> scores documents in-order.
*/
public boolean scoresDocsOutOfOrder() { return false; }
public boolean scoresDocsOutOfOrder() {
return false;
}
}

View File

@ -53,8 +53,8 @@ org.apache.lucene.search.IndexSearcher#search(Query,Filter,int)}.
<p>
Once a Query has been created and submitted to the {@link org.apache.lucene.search.IndexSearcher IndexSearcher}, the scoring
process begins. After some infrastructure setup, control finally passes to the {@link org.apache.lucene.search.Weight Weight}
implementation and its {@link org.apache.lucene.search.Scorer Scorer} instances. See the <a href="#algorithm">Algorithm</a>
section for more notes on the process.
implementation and its {@link org.apache.lucene.search.Scorer Scorer} or {@link org.apache.lucene.search.BulkScorer BulkScore}
instances. See the <a href="#algorithm">Algorithm</a> section for more notes on the process.
</p>
<!-- FILL IN MORE HERE -->
<!-- TODO: this page over-links the same things too many times -->
@ -370,6 +370,12 @@ on the built-in available scoring models and extending or changing Similarity.
{@link org.apache.lucene.search.Scorer Scorer} &mdash; An abstract class containing common
functionality for scoring. Provides both scoring and
explanation capabilities. This is created per-segment.</li>
<li>
{@link org.apache.lucene.search.BulkScorer BulkScorer} &mdash; An abstract class that scores
a range of documents. A default implementation simply iterates through the hits from
{@link org.apache.lucene.search.Scorer Scorer}, but some queries such as
{@link org.apache.lucene.search.BooleanQuery BooleanQuery} have more efficient
implementations.</li>
</ol>
Details on each of these classes, and their children, can be found in the subsections below.
</p>
@ -430,12 +436,18 @@ on the built-in available scoring models and extending or changing Similarity.
that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will just defer to the Similarity's implementation:
{@link org.apache.lucene.search.similarities.Similarity.SimWeight#normalize SimWeight#normalize(float,float)}.</li>
<li>
{@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, boolean, boolean, org.apache.lucene.util.Bits)
scorer(AtomicReaderContext context, boolean scoresDocsInOrder, boolean topScorer, Bits acceptDocs)} &mdash;
{@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, org.apache.lucene.util.Bits)
scorer(AtomicReaderContext context, Bits acceptDocs)} &mdash;
Construct a new {@link org.apache.lucene.search.Scorer Scorer} for this Weight. See <a href="#scorerClass">The Scorer Class</a>
below for help defining a Scorer. As the name implies, the Scorer is responsible for doing the actual scoring of documents
given the Query.
</li>
<li>
{@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.AtomicReaderContext, boolean, org.apache.lucene.util.Bits)
scorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} &mdash;
Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
</li>
<li>
{@link org.apache.lucene.search.Weight#explain(org.apache.lucene.index.AtomicReaderContext, int)
explain(AtomicReaderContext context, int doc)} &mdash; Provide a means for explaining why a given document was
@ -489,6 +501,18 @@ on the built-in available scoring models and extending or changing Similarity.
</li>
</ol>
</p>
<a name="bulkScorerClass"></a>
<h4>The BulkScorer Class</h4>
<p>The
{@link org.apache.lucene.search.BulkScorer BulkScorer} scores a range of documents. There is only one
abstract method:
<ol>
<li>
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector,int) score(Collector,int)} &mdash;
Score all documents up to but not including the specified max document.
</li>
</ol>
</p>
<h4>Why would I want to add my own Query?</h4>
<p>In a nutshell, you want to add your own custom Query implementation when you think that Lucene's
@ -539,7 +563,7 @@ on the built-in available scoring models and extending or changing Similarity.
<p>If a Filter is being used, some initial setup is done to determine which docs to include.
Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each
{@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling
{@link org.apache.lucene.search.Scorer#score(org.apache.lucene.search.Collector) Scorer.score()}.
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector) BulkScorer.score(Collector)}.
</p>
<p>At last, we are actually going to score some documents. The score method takes in the Collector
(most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here

View File

@ -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) {

View File

@ -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) {

View File

@ -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) {

View File

@ -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);
}

View File

@ -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);
BulkScorer scorer = w.bulkScorer(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());

View File

@ -17,6 +17,7 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
@ -29,6 +30,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
@ -43,7 +45,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.NamedThreadFactory;
import org.apache.lucene.util.TestUtil;
import org.apache.lucene.util.TestUtil;
public class TestBooleanQuery extends LuceneTestCase {
@ -234,8 +235,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 +252,7 @@ public class TestBooleanQuery extends LuceneTestCase {
for(int iter2=0;iter2<10;iter2++) {
weight = s.createNormalizedWeight(q);
scorer = weight.scorer(s.leafContexts.get(0),
true, false, null);
scorer = weight.scorer(s.leafContexts.get(0), null);
if (VERBOSE) {
System.out.println(" iter2=" + iter2);
@ -327,4 +326,30 @@ public class TestBooleanQuery extends LuceneTestCase {
directory.close();
}
// LUCENE-5487
public void testInOrderWithMinShouldMatch() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
Document doc = new Document();
doc.add(newTextField("field", "some text here", Field.Store.NO));
w.addDocument(doc);
IndexReader r = w.getReader();
w.close();
IndexSearcher s = new IndexSearcher(r) {
@Override
protected void search(List<AtomicReaderContext> leaves, Weight weight, Collector collector) throws IOException {
assertEquals(-1, collector.getClass().getSimpleName().indexOf("OutOfOrder"));
super.search(leaves, weight, collector);
}
};
BooleanQuery bq = new BooleanQuery();
bq.add(new TermQuery(new Term("field", "some")), BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term("field", "text")), BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term("field", "here")), BooleanClause.Occur.SHOULD);
bq.setMinimumNumberShouldMatch(2);
s.search(bq, 10);
r.close();
dir.close();
}
}

View File

@ -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;
@ -30,10 +32,10 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanQuery.BooleanWeight;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
public class TestBooleanScorer extends LuceneTestCase
{
public class TestBooleanScorer extends LuceneTestCase {
private static final String FIELD = "category";
public void testMethod() throws Exception {
@ -78,27 +80,23 @@ 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) {
BulkScorer[] scorers = new BulkScorer[] {new BulkScorer() {
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;
FakeScorer fs = new FakeScorer();
fs.doc = doc;
fs.score = 1.0f;
c.setScorer(fs);
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.<BulkScorer>emptyList(), scorers.length);
final List<Integer> hits = new ArrayList<Integer>();
bs.score(new Collector() {
@ -157,7 +155,7 @@ public class TestBooleanScorer extends LuceneTestCase
public void setScorer(Scorer scorer) {
// Make sure we got BooleanScorer:
final Class<?> clazz = scorer instanceof AssertingScorer ? ((AssertingScorer) scorer).getIn().getClass() : scorer.getClass();
assertEquals("Scorer is implemented by wrong class", BooleanScorer.class.getName() + "$BucketScorer", clazz.getName());
assertEquals("Scorer is implemented by wrong class", FakeScorer.class.getName(), clazz.getName());
}
@Override
@ -180,4 +178,80 @@ public class TestBooleanScorer extends LuceneTestCase
r.close();
d.close();
}
/** Throws UOE if Weight.scorer is called */
private static class CrazyMustUseBulkScorerQuery extends Query {
@Override
public String toString(String field) {
return "MustUseBulkScorerQuery";
}
@Override
public Weight createWeight(IndexSearcher searcher) throws IOException {
return new Weight() {
@Override
public Explanation explain(AtomicReaderContext context, int doc) {
throw new UnsupportedOperationException();
}
@Override
public Query getQuery() {
return CrazyMustUseBulkScorerQuery.this;
}
@Override
public float getValueForNormalization() {
return 1.0f;
}
@Override
public void normalize(float norm, float topLevelBoost) {
}
@Override
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) {
throw new UnsupportedOperationException();
}
@Override
public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) {
return new BulkScorer() {
@Override
public boolean score(Collector collector, int max) throws IOException {
collector.setScorer(new FakeScorer());
collector.collect(0);
return false;
}
};
}
};
}
}
/** Make sure BooleanScorer can embed another
* BooleanScorer. */
public void testEmbeddedBooleanScorer() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), dir);
Document doc = new Document();
doc.add(newTextField("field", "doctors are people who prescribe medicines of which they know little, to cure diseases of which they know less, in human beings of whom they know nothing", Field.Store.NO));
w.addDocument(doc);
IndexReader r = w.getReader();
w.close();
IndexSearcher s = newSearcher(r);
BooleanQuery q1 = new BooleanQuery();
q1.add(new TermQuery(new Term("field", "little")), BooleanClause.Occur.SHOULD);
q1.add(new TermQuery(new Term("field", "diseases")), BooleanClause.Occur.SHOULD);
BooleanQuery q2 = new BooleanQuery();
q2.add(q1, BooleanClause.Occur.SHOULD);
q2.add(new CrazyMustUseBulkScorerQuery(), BooleanClause.Occur.SHOULD);
assertEquals(1, s.search(q2, 10).totalHits);
r.close();
dir.close();
}
}

View File

@ -121,7 +121,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
checkHits(searcher, csq2, csq2.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), ConstantScoreQuery.ConstantScorer.class.getName());
// for the combined BQ, the scorer should always be BooleanScorer's BucketScorer, because our scorer supports out-of order collection!
final String bucketScorerClass = BooleanScorer.class.getName() + "$BucketScorer";
final String bucketScorerClass = FakeScorer.class.getName();
checkHits(searcher, bq, csq1.getBoost() + csq2.getBoost(), bucketScorerClass, null);
checkHits(searcher, csqbq, csqbq.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), bucketScorerClass);
} finally {
@ -158,7 +158,7 @@ public class TestConstantScoreQuery extends LuceneTestCase {
}
// LUCENE-5307
// don't reuse the scorer of filters since they have been created with topScorer=false
// don't reuse the scorer of filters since they have been created with bulkScorer=false
public void testQueryWrapperFilter() throws IOException {
Directory d = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(random(), d);

View File

@ -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"));

View File

@ -385,10 +385,10 @@ public class TestFilteredQuery extends LuceneTestCase {
private static FilteredQuery.FilterStrategy randomFilterStrategy(Random random, final boolean useRandomAccess) {
if (useRandomAccess) {
return new FilteredQuery.RandomAccessFilterStrategy() {
return new FilteredQuery.RandomAccessFilterStrategy() {
@Override
protected boolean useRandomAccess(Bits bits, int firstFilterDoc) {
return useRandomAccess;
return true;
}
};
}

View File

@ -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);
}
}

View File

@ -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());
BulkScorer ts = weight.bulkScorer(context, true, context.reader().getLiveDocs());
// we have 2 documents with the term all in them, one document for all the
// other values
final List<TestHit> docs = new ArrayList<TestHit>();
@ -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);

View File

@ -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));
}

View File

@ -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);
}

View File

@ -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.BulkScorer;
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 BulkScorer bulkScorer(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;
@ -192,7 +198,7 @@ class DrillSidewaysQuery extends Query {
return null;
}
return new DrillSidewaysScorer(this, context,
return new DrillSidewaysScorer(context,
baseScorer,
drillDownCollector, dims,
scoreSubDocsAtOnce);

View File

@ -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.BulkScorer;
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 BulkScorer {
//private static boolean DEBUG = false;
@ -50,9 +52,8 @@ class DrillSidewaysScorer extends Scorer {
private int collectDocID = -1;
private float collectScore;
DrillSidewaysScorer(Weight w, AtomicReaderContext context, Scorer baseScorer, Collector drillDownCollector,
DrillSidewaysScorer(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,53 @@ 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"));
}
@Override
public Weight getWeight() {
throw new UnsupportedOperationException();
}
}
static class DocsAndCost implements Comparable<DocsAndCost> {

View File

@ -18,12 +18,13 @@ package org.apache.lucene.facet.taxonomy;
*/
import java.io.IOException;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.lucene.facet.FacetsCollector;
import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
import org.apache.lucene.facet.FacetsCollector;
import org.apache.lucene.facet.FacetsConfig;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.queries.function.FunctionValues;
@ -31,6 +32,7 @@ import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
import org.apache.lucene.util.IntsRef;
/** Aggregates sum of values from {@link
@ -70,6 +72,16 @@ public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {
@Override public int nextDoc() throws IOException { throw new UnsupportedOperationException(); }
@Override public int advance(int target) throws IOException { throw new UnsupportedOperationException(); }
@Override public long cost() { return 0; }
@Override
public Weight getWeight() {
throw new UnsupportedOperationException();
}
@Override
public Collection<ChildScorer> getChildren() {
throw new UnsupportedOperationException();
}
}
private final void sumValues(List<MatchingDocs> matchingDocs, boolean keepScores, ValueSource valueSource) throws IOException {

View File

@ -18,14 +18,15 @@ package org.apache.lucene.search.grouping;
*/
import java.io.IOException;
import java.util.Collection;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.search.*;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.PriorityQueue;
import java.io.IOException;
// TODO: this sentence is too long for the class summary.
/** BlockGroupingCollector performs grouping with a
* single pass collector, as long as you are grouping by a
@ -90,7 +91,7 @@ public class BlockGroupingCollector extends Collector {
int doc;
public FakeScorer() {
super((Weight) null);
super(null);
}
@Override
@ -122,6 +123,16 @@ public class BlockGroupingCollector extends Collector {
public long cost() {
return 1;
}
@Override
public Weight getWeight() {
throw new UnsupportedOperationException();
}
@Override
public Collection<ChildScorer> getChildren() {
throw new UnsupportedOperationException();
}
}
private static final class OneGroup {

View File

@ -0,0 +1,75 @@
package org.apache.lucene.search.join;
/*
* 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.util.Collection;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Weight;
/** Passed to {@link Collector#setScorer} during join collection. */
final class FakeScorer extends Scorer {
float score;
int doc = -1;
int freq = 1;
public FakeScorer() {
super(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 score;
}
@Override
public long cost() {
return 1;
}
@Override
public Weight getWeight() {
throw new UnsupportedOperationException();
}
@Override
public Collection<ChildScorer> getChildren() {
throw new UnsupportedOperationException();
}
}

View File

@ -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.BulkScorer;
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) bulkScorer(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,41 @@ 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 BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
if (scoreDocsInOrder) {
return super.bulkScorer(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);
}
}
}
};
}
// 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 BulkScorer {
final BytesRef spare = new BytesRef();
final Bits acceptDocs;
@ -203,7 +220,6 @@ class TermsIncludingScoreQuery extends Query {
int doc;
SVInnerScorer(Weight weight, Bits acceptDocs, TermsEnum termsEnum, long cost) {
super(weight);
this.acceptDocs = acceptDocs;
this.termsEnum = termsEnum;
this.cost = cost;
@ -211,25 +227,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 +269,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 +284,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());
}
}

View File

@ -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

View File

@ -325,46 +325,6 @@ public class ToParentBlockJoinCollector extends Collector {
}
}
private final static class FakeScorer extends Scorer {
float score;
int doc;
public FakeScorer() {
super((Weight) null);
}
@Override
public float score() {
return score;
}
@Override
public int freq() {
return 1; // TODO: does anything else make sense?... duplicate of grouping's FakeScorer btw?
}
@Override
public int docID() {
return doc;
}
@Override
public int advance(int target) {
throw new UnsupportedOperationException();
}
@Override
public int nextDoc() {
throw new UnsupportedOperationException();
}
@Override
public long cost() {
return 1;
}
}
private OneGroup[] sortedGroups;
private void sortQueue() {

View File

@ -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);
}

View File

@ -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());

View File

@ -17,6 +17,19 @@ package org.apache.lucene.search.join;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.document.Document;
@ -34,6 +47,8 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Explanation;
@ -54,19 +69,6 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
public class TestJoinUtil extends LuceneTestCase {
public void testSimple() throws Exception {
@ -151,6 +153,104 @@ public class TestJoinUtil extends LuceneTestCase {
dir.close();
}
/** LUCENE-5487: verify a join query inside a SHOULD BQ
* will still use the join query's optimized BulkScorers */
public void testInsideBooleanQuery() throws Exception {
final String idField = "id";
final String toField = "productId";
Directory dir = newDirectory();
RandomIndexWriter w = new RandomIndexWriter(
random(),
dir,
newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy()));
// 0
Document doc = new Document();
doc.add(new TextField("description", "random text", Field.Store.NO));
doc.add(new TextField("name", "name1", Field.Store.NO));
doc.add(new TextField(idField, "7", Field.Store.NO));
w.addDocument(doc);
// 1
doc = new Document();
doc.add(new TextField("price", "10.0", Field.Store.NO));
doc.add(new TextField(idField, "2", Field.Store.NO));
doc.add(new TextField(toField, "7", Field.Store.NO));
w.addDocument(doc);
// 2
doc = new Document();
doc.add(new TextField("price", "20.0", Field.Store.NO));
doc.add(new TextField(idField, "3", Field.Store.NO));
doc.add(new TextField(toField, "7", Field.Store.NO));
w.addDocument(doc);
// 3
doc = new Document();
doc.add(new TextField("description", "more random text", Field.Store.NO));
doc.add(new TextField("name", "name2", Field.Store.NO));
doc.add(new TextField(idField, "0", Field.Store.NO));
w.addDocument(doc);
w.commit();
// 4
doc = new Document();
doc.add(new TextField("price", "10.0", Field.Store.NO));
doc.add(new TextField(idField, "5", Field.Store.NO));
doc.add(new TextField(toField, "0", Field.Store.NO));
w.addDocument(doc);
// 5
doc = new Document();
doc.add(new TextField("price", "20.0", Field.Store.NO));
doc.add(new TextField(idField, "6", Field.Store.NO));
doc.add(new TextField(toField, "0", Field.Store.NO));
w.addDocument(doc);
w.forceMerge(1);
IndexSearcher indexSearcher = new IndexSearcher(w.getReader());
w.close();
// Search for product
Query joinQuery =
JoinUtil.createJoinQuery(idField, false, toField, new TermQuery(new Term("description", "random")), indexSearcher, ScoreMode.Avg);
BooleanQuery bq = new BooleanQuery();
bq.add(joinQuery, BooleanClause.Occur.SHOULD);
bq.add(new TermQuery(new Term("id", "3")), BooleanClause.Occur.SHOULD);
indexSearcher.search(bq, new Collector() {
boolean sawFive;
@Override
public void setNextReader(AtomicReaderContext context) {
}
@Override
public void collect(int docID) {
// Hairy / evil (depends on how BooleanScorer
// stores temporarily collected docIDs by
// appending to head of linked list):
if (docID == 5) {
sawFive = true;
} else if (docID == 1) {
assertFalse("optimized bulkScorer was not used for join query embedded in boolean query!", sawFive);
}
}
@Override
public void setScorer(Scorer scorer) {
}
@Override
public boolean acceptsDocsOutOfOrder() {
return true;
}
});
indexSearcher.getIndexReader().close();
dir.close();
}
public void testSimpleWithScoring() throws Exception {
final String idField = "id";
final String toField = "movieId";

View File

@ -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);
}

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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;

View File

@ -0,0 +1,110 @@
package org.apache.lucene.search;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.lang.ref.WeakReference;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Random;
import java.util.WeakHashMap;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.util.VirtualMethod;
/** A crazy {@link BulkScorer} that wraps a {@link Scorer}
* but shuffles the order of the collected documents. */
public class AssertingBulkOutOfOrderScorer extends BulkScorer {
final Random random;
final Scorer scorer;
public AssertingBulkOutOfOrderScorer(Random random, Scorer scorer) {
this.random = random;
this.scorer = scorer;
}
private void shuffle(int[] docIDs, float[] scores, int[] freqs, int size) {
for (int i = size - 1; i > 0; --i) {
final int other = random.nextInt(i + 1);
final int tmpDoc = docIDs[i];
docIDs[i] = docIDs[other];
docIDs[other] = tmpDoc;
final float tmpScore = scores[i];
scores[i] = scores[other];
scores[other] = tmpScore;
final int tmpFreq = freqs[i];
freqs[i] = freqs[other];
freqs[other] = tmpFreq;
}
}
private static void flush(int[] docIDs, float[] scores, int[] freqs, int size,
FakeScorer scorer, Collector collector) throws IOException {
for (int i = 0; i < size; ++i) {
scorer.doc = docIDs[i];
scorer.freq = freqs[i];
scorer.score = scores[i];
collector.collect(scorer.doc);
}
}
@Override
public boolean score(Collector collector, int max) throws IOException {
if (scorer.docID() == -1) {
scorer.nextDoc();
}
FakeScorer fake = new FakeScorer();
collector.setScorer(fake);
final int bufferSize = 1 + random.nextInt(100);
final int[] docIDs = new int[bufferSize];
final float[] scores = new float[bufferSize];
final int[] freqs = new int[bufferSize];
int buffered = 0;
int doc = scorer.docID();
while (doc < max) {
docIDs[buffered] = doc;
scores[buffered] = scorer.score();
freqs[buffered] = scorer.freq();
if (++buffered == bufferSize) {
shuffle(docIDs, scores, freqs, buffered);
flush(docIDs, scores, freqs, buffered, fake, collector);
buffered = 0;
}
doc = scorer.nextDoc();
}
shuffle(docIDs, scores, freqs, buffered);
flush(docIDs, scores, freqs, buffered, fake, collector);
return doc != Scorer.NO_MORE_DOCS;
}
@Override
public String toString() {
return "AssertingBulkOutOfOrderScorer(" + scorer + ")";
}
}

View File

@ -0,0 +1,84 @@
package org.apache.lucene.search;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.lang.ref.WeakReference;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Random;
import java.util.WeakHashMap;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.util.VirtualMethod;
/** Wraps a Scorer with additional checks */
public class AssertingBulkScorer extends BulkScorer {
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", Collector.class);
private static final VirtualMethod<BulkScorer> SCORE_COLLECTOR_RANGE = new VirtualMethod<BulkScorer>(BulkScorer.class, "score", Collector.class, int.class);
public static BulkScorer wrap(Random random, BulkScorer other) {
if (other == null || other instanceof AssertingBulkScorer) {
return other;
}
return new AssertingBulkScorer(random, other);
}
public static boolean shouldWrap(BulkScorer inScorer) {
return SCORE_COLLECTOR.isOverriddenAsOf(inScorer.getClass()) || SCORE_COLLECTOR_RANGE.isOverriddenAsOf(inScorer.getClass());
}
final Random random;
final BulkScorer in;
private AssertingBulkScorer(Random random, BulkScorer in) {
this.random = random;
this.in = in;
}
public BulkScorer getIn() {
return in;
}
@Override
public void score(Collector collector) throws IOException {
if (random.nextBoolean()) {
try {
final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS);
assert !remaining;
} catch (UnsupportedOperationException e) {
in.score(collector);
}
} else {
in.score(collector);
}
}
@Override
public boolean score(Collector collector, int max) throws IOException {
return in.score(collector, max);
}
@Override
public String toString() {
return "AssertingBulkScorer(" + in + ")";
}
}

View File

@ -90,6 +90,7 @@ public class AssertingIndexSearcher extends IndexSearcher {
@Override
protected void search(List<AtomicReaderContext> leaves, Weight weight, Collector collector) throws IOException {
// TODO: shouldn't we AssertingCollector.wrap(collector) here?
super.search(leaves, AssertingWeight.wrap(random, weight), collector);
}

View File

@ -26,37 +26,24 @@ import java.util.Random;
import java.util.WeakHashMap;
import org.apache.lucene.index.AssertingAtomicReader;
import org.apache.lucene.index.DocsEnum;
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);
// 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 +55,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 +64,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,143 +95,6 @@ public class AssertingScorer extends Scorer {
return score;
}
private final static class FakeScorer extends Scorer {
float score;
int doc;
int freq;
final long cost;
public FakeScorer(Scorer other) {
super((Weight) null);
this.cost = other.cost();
}
@Override
public float score() {
return score;
}
@Override
public int freq() {
return freq;
}
@Override
public int docID() {
return doc;
}
@Override
public int advance(int target) {
throw new UnsupportedOperationException();
}
@Override
public int nextDoc() {
throw new UnsupportedOperationException();
}
@Override
public long cost() {
return cost;
}
}
private void shuffle(int[] docIDs, float[] scores, int[] freqs, int size) {
for (int i = size - 1; i > 0; --i) {
final int other = random.nextInt(i + 1);
final int tmpDoc = docIDs[i];
docIDs[i] = docIDs[other];
docIDs[other] = tmpDoc;
final float tmpScore = scores[i];
scores[i] = scores[other];
scores[other] = tmpScore;
final int tmpFreq = freqs[i];
freqs[i] = freqs[other];
freqs[other] = tmpFreq;
}
}
private static void flush(int[] docIDs, float[] scores, int[] freqs, int size,
FakeScorer scorer, Collector collector) throws IOException {
for (int i = 0; i < size; ++i) {
scorer.doc = docIDs[i];
scorer.freq = freqs[i];
scorer.score = scores[i];
collector.collect(scorer.doc);
}
}
private void scoreInRandomOrder(Collector collector) throws IOException {
assert docID() == -1; // not started
FakeScorer fake = new FakeScorer(this);
collector.setScorer(fake);
final int bufferSize = 1 + random.nextInt(100);
final int[] docIDs = new int[bufferSize];
final float[] scores = new float[bufferSize];
final int[] freqs = new int[bufferSize];
int buffered = 0;
int doc;
while ((doc = nextDoc()) != NO_MORE_DOCS) {
docIDs[buffered] = doc;
scores[buffered] = score();
freqs[buffered] = freq();
if (++buffered == bufferSize) {
shuffle(docIDs, scores, freqs, buffered);
flush(docIDs, scores, freqs, buffered, fake, collector);
buffered = 0;
}
}
shuffle(docIDs, scores, freqs, buffered);
flush(docIDs, scores, freqs, buffered, fake, collector);
}
@Override
public void score(Collector collector) throws IOException {
assert topScorer != TopScorer.NO;
if (SCORE_COLLECTOR.isOverriddenAsOf(this.in.getClass())) {
if (random.nextBoolean()) {
try {
final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS, in.nextDoc());
assert !remaining;
} catch (UnsupportedOperationException e) {
in.score(collector);
}
} else {
in.score(collector);
}
} else {
// score(Collector) has not been overridden, use the super method in
// order to benefit from all assertions
if (collector.acceptsDocsOutOfOrder() && random.nextBoolean()) {
scoreInRandomOrder(collector);
} else {
super.score(collector);
}
}
}
@Override
public boolean score(Collector collector, int max, int firstDocID) throws IOException {
assert topScorer != TopScorer.NO;
if (SCORE_COLLECTOR_RANGE.isOverriddenAsOf(this.in.getClass())) {
return in.score(collector, max, firstDocID);
} else {
// score(Collector,int,int) has not been overridden, use the super
// method in order to benefit from all assertions
return super.score(collector, max, firstDocID);
}
}
@Override
public Collection<ChildScorer> getChildren() {
// We cannot hide that we hold a single child, else
@ -275,13 +117,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);
}

View File

@ -60,19 +60,46 @@ 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 BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
// if the caller asks for in-order scoring or if the weight does not support
// out-of order scoring then collection will have to happen in-order.
BulkScorer inScorer = in.bulkScorer(context, scoreDocsInOrder, acceptDocs);
if (inScorer == null) {
return null;
}
if (AssertingBulkScorer.shouldWrap(inScorer)) {
// The incoming scorer already has a specialized
// implementation for BulkScorer, so we should use it:
return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
} else if (scoreDocsInOrder == false && random.nextBoolean()) {
// The caller claims it can handle out-of-order
// docs; let's confirm that by pulling docs and
// randomly shuffling them before collection:
//Scorer scorer = in.scorer(context, acceptDocs);
Scorer scorer = scorer(context, acceptDocs);
// Scorer should not be null if bulkScorer wasn't:
assert scorer != null;
return new AssertingBulkOutOfOrderScorer(new Random(random.nextLong()), scorer);
} else {
// Let super wrap this.scorer instead, so we use
// AssertingScorer:
return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
}
}
@Override
public boolean scoresDocsOutOfOrder() {
return scoresDocsOutOfOrder;
}
}

View File

@ -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);

View File

@ -17,6 +17,21 @@
package org.apache.solr.handler.component;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.net.URL;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import org.apache.commons.lang.StringUtils;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.ReaderUtil;
@ -28,6 +43,7 @@ import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.Weight;
import org.apache.lucene.search.grouping.GroupDocs;
import org.apache.lucene.search.grouping.SearchGroup;
import org.apache.lucene.search.grouping.TopGroups;
@ -83,22 +99,6 @@ import org.apache.solr.search.grouping.endresulttransformer.MainEndResultTransfo
import org.apache.solr.search.grouping.endresulttransformer.SimpleEndResultTransformer;
import org.apache.solr.util.SolrPluginUtils;
import org.apache.commons.lang.StringUtils;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
import java.net.URL;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
/**
* TODO!
*
@ -1227,5 +1227,15 @@ public class QueryComponent extends SearchComponent
public long cost() {
return 1;
}
@Override
public Weight getWeight() {
throw new UnsupportedOperationException();
}
@Override
public Collection<ChildScorer> getChildren() {
throw new UnsupportedOperationException();
}
}
}

View File

@ -333,14 +333,13 @@ class SpatialDistanceQuery extends ExtendedQueryBase implements PostFilter {
}
@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 SpatialScorer(context, acceptDocs, this, queryWeight);
}
@Override
public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
return ((SpatialScorer)scorer(context, true, true, context.reader().getLiveDocs())).explain(doc);
return ((SpatialScorer)scorer(context, context.reader().getLiveDocs())).explain(doc);
}
}

View File

@ -232,8 +232,7 @@ class JoinQuery 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 {
if (filter == null) {
boolean debug = rb != null && rb.isDebug();
long start = debug ? System.currentTimeMillis() : 0;
@ -502,7 +501,7 @@ class JoinQuery 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());
boolean exists = scorer.advance(doc) == doc;
ComplexExplanation result = new ComplexExplanation();

View File

@ -119,8 +119,7 @@ public class SolrConstantScoreQuery extends ConstantScoreQuery implements Extend
}
@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 ConstantScorer(context, this, queryWeight, acceptDocs);
}

View File

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

View File

@ -86,8 +86,8 @@ public class IgnoreAcceptDocsQuery extends Query {
}
@Override
public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Bits acceptDocs) throws IOException {
return w.scorer(context, scoreDocsInOrder, topScorer, null);
public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
return w.scorer(context, null);
}
}