LUCENE-6184: Make BooleanScorer only score windows that contain matches.

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1653020 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrien Grand 2015-01-19 14:49:25 +00:00
parent e61cd9654b
commit cb3ec682d4
18 changed files with 252 additions and 95 deletions

View File

@ -38,6 +38,9 @@ Optimizations
mode uses a higher deflate level for more compact storage.
(Robert Muir)
* LUCENE-6184: Make BooleanScorer only score windows that contain
matches. (Adrien Grand)
======================= Lucene 5.0.0 =======================
New Features

View File

@ -22,6 +22,7 @@ import java.util.Arrays;
import java.util.Collection;
import org.apache.lucene.search.BooleanQuery.BooleanWeight;
import org.apache.lucene.util.PriorityQueue;
/**
* BulkSorer that is used for pure disjunctions: no MUST clauses and
@ -40,17 +41,24 @@ final class BooleanScorer extends BulkScorer {
int freq;
}
static class BulkScorerAndDoc {
final BulkScorer scorer;
int next;
BulkScorerAndDoc(BulkScorer scorer) {
this.scorer = scorer;
this.next = 0;
}
}
final Bucket[] buckets = new Bucket[SIZE];
// This is basically an inlined FixedBitSet... seems to help with bound checks
final long[] matching = new long[SET_SIZE];
final float[] coordFactors;
final BulkScorer[] optionalScorers;
final PriorityQueue<BulkScorerAndDoc> optionalScorers;
final FakeScorer fakeScorer = new FakeScorer();
boolean hasMatches;
int max = 0;
final class OrCollector implements LeafCollector {
Scorer scorer;
@ -61,7 +69,6 @@ final class BooleanScorer extends BulkScorer {
@Override
public void collect(int doc) throws IOException {
hasMatches = true;
final int i = doc & MASK;
final int idx = i >>> 6;
matching[idx] |= 1L << i;
@ -77,7 +84,15 @@ final class BooleanScorer extends BulkScorer {
for (int i = 0; i < buckets.length; i++) {
buckets[i] = new Bucket();
}
this.optionalScorers = optionalScorers.toArray(new BulkScorer[0]);
this.optionalScorers = new PriorityQueue<BulkScorerAndDoc>(optionalScorers.size()) {
@Override
protected boolean lessThan(BulkScorerAndDoc a, BulkScorerAndDoc b) {
return a.next < b.next;
}
};
for (BulkScorer scorer : optionalScorers) {
this.optionalScorers.add(new BulkScorerAndDoc(scorer));
}
coordFactors = new float[optionalScorers.size() + 1];
for (int i = 0; i < coordFactors.length; i++) {
@ -86,6 +101,7 @@ final class BooleanScorer extends BulkScorer {
}
private void scoreDocument(LeafCollector collector, int base, int i) throws IOException {
final FakeScorer fakeScorer = this.fakeScorer;
final Bucket bucket = buckets[i];
fakeScorer.freq = bucket.freq;
fakeScorer.score = (float) bucket.score * coordFactors[bucket.freq];
@ -109,38 +125,32 @@ final class BooleanScorer extends BulkScorer {
}
}
private boolean collectMatches() throws IOException {
boolean more = false;
for (BulkScorer scorer : optionalScorers) {
more |= scorer.score(orCollector, max);
}
return more;
}
private BulkScorerAndDoc scoreWindow(LeafCollector collector, int base, int min, int max,
PriorityQueue<BulkScorerAndDoc> optionalScorers, BulkScorerAndDoc top) throws IOException {
assert top.next < max;
do {
top.next = top.scorer.score(orCollector, min, max);
top = optionalScorers.updateTop();
} while (top.next < max);
private boolean scoreWindow(LeafCollector collector, int base, int max) throws IOException {
this.max = Math.min(base + SIZE, max);
hasMatches = false;
boolean more = collectMatches();
if (hasMatches) {
scoreMatches(collector, base);
Arrays.fill(matching, 0L);
}
return more;
scoreMatches(collector, base);
Arrays.fill(matching, 0L);
return top;
}
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
public int score(LeafCollector collector, int min, int max) throws IOException {
fakeScorer.doc = -1;
collector.setScorer(fakeScorer);
final PriorityQueue<BulkScorerAndDoc> optionalScorers = this.optionalScorers;
for (int docBase = this.max & ~MASK; docBase < max; docBase += SIZE) {
if (scoreWindow(collector, docBase, max) == false) {
return false;
}
BulkScorerAndDoc top = optionalScorers.top();
for (int windowMin = Math.max(min, top.next); windowMin < max; windowMin = top.next) {
final int windowBase = windowMin & ~MASK; // find the window that windowMin belongs to
final int windowMax = Math.min(max, windowBase + SIZE);
top = scoreWindow(collector, windowBase, windowMin, windowMax, optionalScorers, top);
assert top.next >= windowMax;
}
return true;
return top.next;
}
}

View File

@ -32,15 +32,46 @@ public abstract class BulkScorer {
* @param collector The collector to which all matching documents are passed.
*/
public void score(LeafCollector collector) throws IOException {
score(collector, Integer.MAX_VALUE);
final int next = score(collector, 0, DocIdSetIterator.NO_MORE_DOCS);
assert next == DocIdSetIterator.NO_MORE_DOCS;
}
/**
* 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.
* Collects matching documents in a range and return an estimation of the
* next matching document which is on or after {@code max}.
* <p>The return value must be:</p><ul>
* <li>&gt;= {@code max},</li>
* <li>{@link DocIdSetIterator#NO_MORE_DOCS} if there are no more matches,</li>
* <li>&lt;= the first matching document that is &gt;= {@code max} otherwise.</li>
* </ul>
* <p>{@code min} is the minimum document to be considered for matching. All
* documents strictly before this value must be ignored.</p>
* <p>Although {@code max} would be a legal return value for this method, higher
* values might help callers skip more efficiently over non-matching portions
* of the docID space.</p>
* <p>For instance, a {@link Scorer}-based implementation could look like
* below:</p>
* <pre class="prettyprint">
* private final Scorer scorer; // set via constructor
*
* public int score(LeafCollector collector, int min, int max) throws IOException {
* collector.setScorer(scorer);
* int doc = scorer.docID();
* if (doc &lt; min) {
* doc = scorer.advance(min);
* }
* while (doc &lt; max) {
* collector.collect(doc);
* doc = scorer.nextDoc();
* }
* return doc;
* }
* </pre>
*
* @param collector The collector to which all matching documents are passed.
* @param min Score starting at, including, this document
* @param max Score up to, but not including, this doc
* @return an under-estimation of the next matching doc after max
*/
public abstract boolean score(LeafCollector collector, int max) throws IOException;
public abstract int score(LeafCollector collector, int min, int max) throws IOException;
}

View File

@ -207,8 +207,8 @@ public class ConstantScoreQuery extends Query {
}
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
return bulkScorer.score(wrapCollector(collector), max);
public int score(LeafCollector collector, int min, int max) throws IOException {
return bulkScorer.score(wrapCollector(collector), min, max);
}
private LeafCollector wrapCollector(LeafCollector collector) {

View File

@ -220,12 +220,12 @@ public class FilteredQuery extends Query {
}
@Override
public boolean score(LeafCollector collector, int maxDoc) throws IOException {
public int score(LeafCollector collector, int min, 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();
if (scorer.docID() < min) {
scorer.advance(min);
}
while (true) {
final int scorerDoc = scorer.docID();
@ -239,7 +239,7 @@ public class FilteredQuery extends Query {
}
}
return scorer.docID() != Scorer.NO_MORE_DOCS;
return scorer.docID();
}
}

View File

@ -56,7 +56,7 @@ public class MatchAllDocsQuery extends Query {
while(liveDocs != null && doc < maxDoc && !liveDocs.get(doc)) {
doc++;
}
if (doc == maxDoc) {
if (doc >= maxDoc) { // can be > maxDoc when called from advance()
doc = NO_MORE_DOCS;
}
return doc;

View File

@ -136,7 +136,7 @@ public abstract class Weight {
}
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
public int score(LeafCollector collector, int min, 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,
@ -144,13 +144,13 @@ public abstract class Weight {
// Collector doing something "interesting" in
// setScorer will be forced to use BS2 anyways:
collector.setScorer(scorer);
if (max == DocIdSetIterator.NO_MORE_DOCS) {
if (scorer.docID() == -1 && min == 0 && max == DocIdSetIterator.NO_MORE_DOCS) {
scoreAll(collector, scorer);
return false;
return DocIdSetIterator.NO_MORE_DOCS;
} else {
int doc = scorer.docID();
if (doc < 0) {
doc = scorer.nextDoc();
if (doc < min) {
doc = scorer.advance(min);
}
return scoreRange(collector, scorer, doc, max);
}
@ -160,12 +160,12 @@ public abstract class Weight {
* separate this from {@link #scoreAll} to help out
* hotspot.
* See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
static boolean scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
static int scoreRange(LeafCollector collector, Scorer scorer, int currentDoc, int end) throws IOException {
while (currentDoc < end) {
collector.collect(currentDoc);
currentDoc = scorer.nextDoc();
}
return currentDoc != DocIdSetIterator.NO_MORE_DOCS;
return currentDoc;
}
/** Specialized method to bulk-score all hits; we
@ -173,8 +173,7 @@ public abstract class Weight {
* hotspot.
* See <a href="https://issues.apache.org/jira/browse/LUCENE-5487">LUCENE-5487</a> */
static void scoreAll(LeafCollector collector, Scorer scorer) throws IOException {
int doc;
while ((doc = scorer.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
for (int doc = scorer.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = scorer.nextDoc()) {
collector.collect(doc);
}
}

View File

@ -508,7 +508,7 @@ on the built-in available scoring models and extending or changing Similarity.
abstract method:
<ol>
<li>
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int) score(LeafCollector,int)} &mdash;
{@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.LeafCollector,int,int) score(LeafCollector,int,int)} &mdash;
Score all documents up to but not including the specified max document.
</li>
</ol>

View File

@ -731,7 +731,7 @@ public class TestBooleanCoord extends LuceneTestCase {
assertEquals(expected, scorer.score(), 0.0001f);
seen.set(true);
}
}, 1);
}, 0, 1);
assertTrue(seen.get());
// test the explanation

View File

@ -16,6 +16,11 @@ package org.apache.lucene.search;
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.analysis.MockAnalyzer;
@ -29,6 +34,8 @@ import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
public class TestBooleanOr extends LuceneTestCase {
private static String FIELD_T = "T";
@ -196,13 +203,62 @@ public class TestBooleanOr extends LuceneTestCase {
};
while (end.intValue() < docCount) {
final int min = end.intValue();
final int inc = TestUtil.nextInt(random(), 1, 1000);
end.getAndAdd(inc);
scorer.score(c, end.intValue());
final int max = end.addAndGet(inc);
scorer.score(c, min, max);
}
assertEquals(docCount, hits.cardinality());
r.close();
dir.close();
}
private static BulkScorer scorer(int... matches) {
return new BulkScorer() {
final FakeScorer scorer = new FakeScorer();
int i = 0;
@Override
public int score(LeafCollector collector, int min, int max) throws IOException {
collector.setScorer(scorer);
while (i < matches.length && matches[i] < min) {
i += 1;
}
while (i < matches.length && matches[i] < max) {
scorer.doc = matches[i];
collector.collect(scorer.doc);
i += 1;
}
if (i == matches.length) {
return DocIdSetIterator.NO_MORE_DOCS;
}
return RandomInts.randomIntBetween(random(), max, matches[i]);
}
};
}
// Make sure that BooleanScorer keeps working even if the sub clauses return
// next matching docs which are less than the actual next match
public void testSubScorerNextIsNotMatch() throws IOException {
final List<BulkScorer> optionalScorers = Arrays.asList(
scorer(100000, 1000001, 9999999),
scorer(4000, 1000051),
scorer(5000, 100000, 9999998, 9999999)
);
Collections.shuffle(optionalScorers, random());
BooleanScorer scorer = new BooleanScorer(null, true, 0, optionalScorers);
final List<Integer> matches = new ArrayList<>();
scorer.score(new LeafCollector() {
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
public void collect(int doc) throws IOException {
matches.add(doc);
}
});
assertEquals(Arrays.asList(4000, 5000, 100000, 1000001, 1000051, 9999998, 9999999), matches);
}
}

View File

@ -18,18 +18,13 @@ package org.apache.lucene.search;
*/
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;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
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;
@ -106,10 +101,11 @@ public class TestBooleanScorer extends LuceneTestCase {
return new BulkScorer() {
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
public int score(LeafCollector collector, int min, int max) throws IOException {
assert min == 0;
collector.setScorer(new FakeScorer());
collector.collect(0);
return false;
return DocIdSetIterator.NO_MORE_DOCS;
}
};
}

View File

@ -64,7 +64,10 @@ class DrillSidewaysScorer extends BulkScorer {
}
@Override
public boolean score(LeafCollector collector, int maxDoc) throws IOException {
public int score(LeafCollector collector, int min, int maxDoc) throws IOException {
if (min != 0) {
throw new IllegalArgumentException("min must be 0, got " + min);
}
if (maxDoc != Integer.MAX_VALUE) {
throw new IllegalArgumentException("maxDoc must be Integer.MAX_VALUE");
}
@ -150,7 +153,7 @@ class DrillSidewaysScorer extends BulkScorer {
doUnionScoring(collector, disis, sidewaysCollectors);
}
return false;
return Integer.MAX_VALUE;
}
/** Used when base query is highly constraining vs the

View File

@ -280,20 +280,8 @@ public class AssertingLeafReader extends FilterLeafReader {
private int doc;
public AssertingDocsEnum(DocsEnum in) {
this(in, true);
}
public AssertingDocsEnum(DocsEnum in, boolean failOnUnsupportedDocID) {
super(in);
try {
int docid = in.docID();
assert docid == -1 : in.getClass() + ": invalid initial doc id: " + docid;
} catch (UnsupportedOperationException e) {
if (failOnUnsupportedDocID) {
throw e;
}
}
doc = -1;
this.doc = in.docID();
}
@Override

View File

@ -22,22 +22,27 @@ import java.util.Random;
import org.apache.lucene.index.DocsEnum;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
/** Wraps a Scorer with additional checks */
final class AssertingBulkScorer extends BulkScorer {
public static BulkScorer wrap(Random random, BulkScorer other) {
public static BulkScorer wrap(Random random, BulkScorer other, int maxDoc) {
if (other == null || other instanceof AssertingBulkScorer) {
return other;
}
return new AssertingBulkScorer(random, other);
return new AssertingBulkScorer(random, other, maxDoc);
}
final Random random;
final BulkScorer in;
final int maxDoc;
int max = 0;
private AssertingBulkScorer(Random random, BulkScorer in) {
private AssertingBulkScorer(Random random, BulkScorer in, int maxDoc) {
this.random = random;
this.in = in;
this.maxDoc = maxDoc;
}
public BulkScorer getIn() {
@ -46,11 +51,12 @@ final class AssertingBulkScorer extends BulkScorer {
@Override
public void score(LeafCollector collector) throws IOException {
collector = new AssertingLeafCollector(random, collector, DocsEnum.NO_MORE_DOCS);
assert max == 0;
collector = new AssertingLeafCollector(random, collector, 0, DocsEnum.NO_MORE_DOCS);
if (random.nextBoolean()) {
try {
final boolean remaining = in.score(collector, DocsEnum.NO_MORE_DOCS);
assert !remaining;
final int next = score(collector, 0, DocsEnum.NO_MORE_DOCS);
assert next == DocIdSetIterator.NO_MORE_DOCS;
} catch (UnsupportedOperationException e) {
in.score(collector);
}
@ -60,9 +66,19 @@ final class AssertingBulkScorer extends BulkScorer {
}
@Override
public boolean score(LeafCollector collector, int max) throws IOException {
collector = new AssertingLeafCollector(random, collector, max);
return in.score(collector, max);
public int score(LeafCollector collector, int min, final int max) throws IOException {
assert min >= this.max: "Scoring backward: min=" + min + " while previous max was max=" + this.max;
assert min < max : "max must be greater than min, got min=" + min + ", and max=" + max;
this.max = max;
collector = new AssertingLeafCollector(random, collector, min, max);
final int next = in.score(collector, min, max);
assert next >= max;
if (max >= maxDoc || next >= maxDoc) {
assert next == DocIdSetIterator.NO_MORE_DOCS;
return DocIdSetIterator.NO_MORE_DOCS;
} else {
return RandomInts.randomIntBetween(random, max, next);
}
}
@Override

View File

@ -47,7 +47,7 @@ class AssertingCollector extends FilterCollector {
public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
final LeafCollector in = super.getLeafCollector(context);
final int docBase = context.docBase;
return new AssertingLeafCollector(random, in, DocIdSetIterator.NO_MORE_DOCS) {
return new AssertingLeafCollector(random, in, 0, DocIdSetIterator.NO_MORE_DOCS) {
@Override
public void collect(int doc) throws IOException {
// check that documents are scored in order globally,

View File

@ -25,14 +25,16 @@ import java.util.Random;
class AssertingLeafCollector extends FilterLeafCollector {
private final Random random;
private final int min;
private final int max;
private Scorer scorer;
private int lastCollected = -1;
AssertingLeafCollector(Random random, LeafCollector collector, int max) {
AssertingLeafCollector(Random random, LeafCollector collector, int min, int max) {
super(collector);
this.random = random;
this.min = min;
this.max = max;
}
@ -45,6 +47,7 @@ class AssertingLeafCollector extends FilterLeafCollector {
@Override
public void collect(int doc) throws IOException {
assert doc > lastCollected : "Out of order : " + lastCollected + " " + doc;
assert doc >= min : "Out of range: " + doc + " < " + min;
assert doc < max : "Out of range: " + doc + " >= " + max;
assert scorer.docID() == doc : "Collected: " + doc + " but scorer: " + scorer.docID();
in.collect(doc);

View File

@ -59,9 +59,8 @@ class AssertingWeight extends Weight {
@Override
public Scorer scorer(LeafReaderContext 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 Scorer inScorer = in.scorer(context, acceptDocs);
assert inScorer == null || inScorer.docID() == -1;
return AssertingScorer.wrap(new Random(random.nextLong()), inScorer);
}
@ -72,6 +71,6 @@ class AssertingWeight extends Weight {
return null;
}
return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer);
return AssertingBulkScorer.wrap(new Random(random.nextLong()), inScorer, context.reader().maxDoc());
}
}

View File

@ -111,6 +111,7 @@ public class QueryUtils {
if (s!=null) {
checkFirstSkipTo(q1,s);
checkSkipTo(q1,s);
checkBulkScorerSkipTo(random, q1, s);
if (wrap) {
check(random, q1, wrapUnderlyingReader(random, s, -1), false);
check(random, q1, wrapUnderlyingReader(random, s, 0), false);
@ -419,4 +420,56 @@ public class QueryUtils {
}
}
}
/** Check that the scorer and bulk scorer advance consistently. */
public static void checkBulkScorerSkipTo(Random r, Query query, IndexSearcher searcher) throws IOException {
Weight weight = searcher.createNormalizedWeight(query);
for (LeafReaderContext context : searcher.getIndexReader().leaves()) {
final Scorer scorer = weight.scorer(context, context.reader().getLiveDocs());
final BulkScorer bulkScorer = weight.bulkScorer(context, context.reader().getLiveDocs());
if (scorer == null && bulkScorer == null) {
continue;
}
int upTo = 0;
while (true) {
final int min = upTo + r.nextInt(5);
final int max = min + 1 + r.nextInt(r.nextBoolean() ? 10 : 5000);
if (scorer.docID() < min) {
scorer.advance(min);
}
final int next = bulkScorer.score(new LeafCollector() {
Scorer scorer2;
@Override
public void setScorer(Scorer scorer) throws IOException {
this.scorer2 = scorer;
}
@Override
public void collect(int doc) throws IOException {
assert doc >= min;
assert doc < max;
Assert.assertEquals(scorer.docID(), doc);
Assert.assertEquals(scorer.score(), scorer2.score(), 0.01f);
scorer.nextDoc();
}
}, min, max);
assert max <= next;
assert next <= scorer.docID();
upTo = max;
if (scorer.docID() == DocIdSetIterator.NO_MORE_DOCS) {
bulkScorer.score(new LeafCollector() {
@Override
public void setScorer(Scorer scorer) throws IOException {}
@Override
public void collect(int doc) throws IOException {
// no more matches
assert false;
}
}, upTo, DocIdSetIterator.NO_MORE_DOCS);
break;
}
}
}
}
}