From 915992b0887a9188dee5a784c3e393708c089fbf Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Mon, 26 Oct 2015 13:10:00 +0000 Subject: [PATCH] LUCENE-6850: Optimize BooleanScorer for sparse clauses. git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1710591 13f79535-47bb-0310-9956-ffa450edef68 --- lucene/CHANGES.txt | 2 + .../apache/lucene/search/BooleanScorer.java | 85 +++++++++++----- .../lucene/search/BooleanTopLevelScorers.java | 36 ++++++- .../apache/lucene/search/BooleanWeight.java | 13 ++- .../lucene/search/TestBooleanScorer.java | 96 +++++++++++++++++++ 5 files changed, 206 insertions(+), 26 deletions(-) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index 65e095bd3c7..0f84469f44f 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -156,6 +156,8 @@ Optimizations strings larger than 64kb by an amount equal to string's utf8 size. (Dawid Weiss, Robert Muir, shalin) +* LUCENE-6850: Optimize BooleanScorer for sparse clauses. (Adrien Grand) + Bug Fixes * LUCENE-6817: ComplexPhraseQueryParser.ComplexPhraseQuery does not display diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java index 7a164141a00..3bc49994e44 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -83,11 +83,11 @@ final class BooleanScorer extends BulkScorer { } void advance(int min) throws IOException { - score(null, min, min); + score(orCollector, null, min, min); } - void score(Bits acceptDocs, int min, int max) throws IOException { - next = scorer.score(orCollector, acceptDocs, min, max); + void score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { + next = scorer.score(collector, acceptDocs, min, max); } } @@ -179,6 +179,9 @@ final class BooleanScorer extends BulkScorer { if (minShouldMatch < 1 || minShouldMatch > scorers.size()) { throw new IllegalArgumentException("minShouldMatch should be within 1..num_scorers. Got " + minShouldMatch); } + if (scorers.size() <= 1) { + throw new IllegalArgumentException("This scorer can only be used with two scorers or more, got " + scorers.size()); + } for (int i = 0; i < buckets.length; i++) { buckets[i] = new Bucket(); } @@ -237,12 +240,12 @@ final class BooleanScorer extends BulkScorer { } } - private void scoreWindow(LeafCollector collector, Bits acceptDocs, int base, int min, int max, - BulkScorerAndDoc[] scorers, int numScorers) throws IOException { + private void scoreWindowIntoBitSetAndReplay(LeafCollector collector, Bits acceptDocs, + int base, int min, int max, BulkScorerAndDoc[] scorers, int numScorers) throws IOException { for (int i = 0; i < numScorers; ++i) { final BulkScorerAndDoc scorer = scorers[i]; assert scorer.next < max; - scorer.score(acceptDocs, min, max); + scorer.score(orCollector, acceptDocs, min, max); } scoreMatches(collector, base); @@ -270,14 +273,7 @@ final class BooleanScorer extends BulkScorer { return headTop; } - private void scoreWindow(LeafCollector collector, Bits acceptDocs, int windowBase, int windowMin, int windowMax) throws IOException { - // Fill 'leads' with all scorers from 'head' that are in the right window - leads[0] = head.pop(); - int maxFreq = 1; - while (head.size() > 0 && head.top().next < windowMax) { - leads[maxFreq++] = head.pop(); - } - + private void scoreWindowMultipleScorers(LeafCollector collector, Bits acceptDocs, int windowBase, int windowMin, int windowMax, int maxFreq) throws IOException { while (maxFreq < minShouldMatch && maxFreq + tail.size() >= minShouldMatch) { // a match is still possible final BulkScorerAndDoc candidate = tail.pop(); @@ -296,7 +292,7 @@ final class BooleanScorer extends BulkScorer { } tail.clear(); - scoreWindow(collector, acceptDocs, windowBase, windowMin, windowMax, leads, maxFreq); + scoreWindowIntoBitSetAndReplay(collector, acceptDocs, windowBase, windowMin, windowMax, leads, maxFreq); } // Push back scorers into head and tail @@ -308,21 +304,64 @@ final class BooleanScorer extends BulkScorer { } } + private void scoreWindowSingleScorer(BulkScorerAndDoc bulkScorer, LeafCollector collector, + Bits acceptDocs, int windowMin, int windowMax, int max) throws IOException { + assert tail.size() == 0; + final int nextWindowBase = head.top().next & ~MASK; + final int end = Math.max(windowMax, Math.min(max, nextWindowBase)); + + bulkScorer.score(collector, acceptDocs, windowMin, end); + + // reset the scorer that should be used for the general case + collector.setScorer(fakeScorer); + } + + private BulkScorerAndDoc scoreWindow(BulkScorerAndDoc top, LeafCollector collector, + LeafCollector singleClauseCollector, Bits acceptDocs, int min, int max) throws IOException { + final int windowBase = top.next & ~MASK; // find the window that the next match belongs to + final int windowMin = Math.max(min, windowBase); + final int windowMax = Math.min(max, windowBase + SIZE); + + // Fill 'leads' with all scorers from 'head' that are in the right window + leads[0] = head.pop(); + int maxFreq = 1; + while (head.size() > 0 && head.top().next < windowMax) { + leads[maxFreq++] = head.pop(); + } + + if (minShouldMatch == 1 && maxFreq == 1) { + // special case: only one scorer can match in the current window, + // we can collect directly + final BulkScorerAndDoc bulkScorer = leads[0]; + scoreWindowSingleScorer(bulkScorer, singleClauseCollector, acceptDocs, windowMin, windowMax, max); + return head.add(bulkScorer); + } else { + // general case, collect through a bit set first and then replay + scoreWindowMultipleScorers(collector, acceptDocs, windowBase, windowMin, windowMax, maxFreq); + return head.top(); + } + } + @Override public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { fakeScorer.doc = -1; collector.setScorer(fakeScorer); + final LeafCollector singleClauseCollector; + if (coordFactors[1] == 1f) { + singleClauseCollector = collector; + } else { + singleClauseCollector = new FilterLeafCollector(collector) { + @Override + public void setScorer(Scorer scorer) throws IOException { + super.setScorer(new BooleanTopLevelScorers.BoostedScorer(scorer, coordFactors[1])); + } + }; + } + BulkScorerAndDoc top = advance(min); while (top.next < max) { - - final int windowBase = top.next & ~MASK; // find the window that the next match belongs to - final int windowMin = Math.max(min, windowBase); - final int windowMax = Math.min(max, windowBase + SIZE); - - // general case - scoreWindow(collector, acceptDocs, windowBase, windowMin, windowMax); - top = head.top(); + top = scoreWindow(top, collector, singleClauseCollector, acceptDocs, min, max); } return top.next; diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java index f44f254413e..443531f2536 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanTopLevelScorers.java @@ -22,6 +22,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import org.apache.lucene.util.Bits; + /** Internal document-at-a-time scorers used to deal with stupid coord() computation */ class BooleanTopLevelScorers { @@ -48,7 +50,39 @@ class BooleanTopLevelScorers { return Collections.singleton(new ChildScorer(in, "BOOSTED")); } } - + + /** + * Used when there is more than one scorer in a query, but a segment + * only had one non-null scorer. + */ + static class BoostedBulkScorer extends BulkScorer { + + final BulkScorer in; + final float boost; + + BoostedBulkScorer(BulkScorer scorer, float boost) { + this.in = scorer; + this.boost = boost; + } + + @Override + public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { + final LeafCollector wrapped = new FilterLeafCollector(collector) { + @Override + public void setScorer(Scorer scorer) throws IOException { + super.setScorer(new BoostedScorer(scorer, boost)); + } + }; + return in.score(wrapped, acceptDocs, min, max); + } + + @Override + public long cost() { + return in.cost(); + } + + } + /** * Used when there are both mandatory and optional clauses, but minShouldMatch * dictates that some of the optional clauses must match. The query is a conjunction, diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java index 44dc2dbeaf3..a54e7133af3 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java @@ -190,7 +190,7 @@ final class BooleanWeight extends Weight { /** Try to build a boolean scorer for this weight. Returns null if {@link BooleanScorer} * cannot be used. */ // pkg-private for forcing use of BooleanScorer in tests - BooleanScorer booleanScorer(LeafReaderContext context) throws IOException { + BulkScorer booleanScorer(LeafReaderContext context) throws IOException { List optional = new ArrayList(); Iterator cIter = query.iterator(); for (Weight w : weights) { @@ -222,12 +222,21 @@ final class BooleanWeight extends Weight { return null; } + if (optional.size() == 1) { + BulkScorer opt = optional.get(0); + if (!disableCoord && maxCoord > 1) { + return new BooleanTopLevelScorers.BoostedBulkScorer(opt, coord(1, maxCoord)); + } else { + return opt; + } + } + return new BooleanScorer(this, disableCoord, maxCoord, optional, Math.max(1, query.getMinimumNumberShouldMatch()), needsScores); } @Override public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - final BooleanScorer bulkScorer = booleanScorer(context); + final BulkScorer bulkScorer = booleanScorer(context); if (bulkScorer != null) { // BooleanScorer is applicable // TODO: what is the right heuristic here? final long costThreshold; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java index 72da86a3d99..428bc17e39b 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java @@ -18,17 +18,24 @@ package org.apache.lucene.search; */ import java.io.IOException; +import java.util.Arrays; import java.util.Set; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; +import org.apache.lucene.document.StringField; +import org.apache.lucene.document.Field.Store; 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.BooleanClause.Occur; +import org.apache.lucene.search.Weight.DefaultBulkScorer; +import org.apache.lucene.search.similarities.ClassicSimilarity; import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.TestUtil; public class TestBooleanScorer extends LuceneTestCase { private static final String FIELD = "category"; @@ -141,4 +148,93 @@ public class TestBooleanScorer extends LuceneTestCase { r.close(); dir.close(); } + + public void testOptimizeTopLevelClauseOrNull() throws IOException { + // When there is a single non-null scorer, this scorer should be used + // directly + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + Document doc = new Document(); + doc.add(new StringField("foo", "bar", Store.NO)); + w.addDocument(doc); + IndexReader reader = w.getReader(); + IndexSearcher searcher = new IndexSearcher(reader); + searcher.setQueryCache(null); // so that weights are not wrapped + final LeafReaderContext ctx = reader.leaves().get(0); + Query query = new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term + .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term + .build(); + + // no scores -> term scorer + Weight weight = searcher.createNormalizedWeight(query, false); + BulkScorer scorer = ((BooleanWeight) weight).booleanScorer(ctx); + assertTrue(scorer instanceof DefaultBulkScorer); // term scorer + + // disabled coords -> term scorer + query = new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term + .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term + .setDisableCoord(true) + .build(); + weight = searcher.createNormalizedWeight(query, true); + scorer = ((BooleanWeight) weight).booleanScorer(ctx); + assertTrue(scorer instanceof DefaultBulkScorer); // term scorer + + // enabled coords -> BoostedBulkScorer + searcher.setSimilarity(new ClassicSimilarity()); + query = new BooleanQuery.Builder() + .add(new TermQuery(new Term("foo", "bar")), Occur.SHOULD) // existing term + .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term + .build(); + weight = searcher.createNormalizedWeight(query, true); + scorer = ((BooleanWeight) weight).booleanScorer(ctx); + assertTrue(scorer instanceof BooleanTopLevelScorers.BoostedBulkScorer); + + w.close(); + reader.close(); + dir.close(); + } + + public void testSparseClauseOptimization() throws IOException { + // When some windows have only one scorer that can match, the scorer will + // directly call the collector in this window + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + Document emptyDoc = new Document(); + final int numDocs = atLeast(10); + for (int d = 0; d < numDocs; ++d) { + for (int i = random().nextInt(5000); i >= 0; --i) { + w.addDocument(emptyDoc); + } + Document doc = new Document(); + for (String value : Arrays.asList("foo", "bar", "baz")) { + if (random().nextBoolean()) { + doc.add(new StringField("field", value, Store.NO)); + } + } + } + for (int i = TestUtil.nextInt(random(), 3000, 5000); i >= 0; --i) { + w.addDocument(emptyDoc); + } + if (random().nextBoolean()) { + w.forceMerge(1); + } + IndexReader reader = w.getReader(); + IndexSearcher searcher = newSearcher(reader); + + Query query = new BooleanQuery.Builder() + .add(new BoostQuery(new TermQuery(new Term("field", "foo")), 3), Occur.SHOULD) + .add(new BoostQuery(new TermQuery(new Term("field", "bar")), 3), Occur.SHOULD) + .add(new BoostQuery(new TermQuery(new Term("field", "baz")), 3), Occur.SHOULD) + .setDisableCoord(random().nextBoolean()) + .build(); + + // duel BS1 vs. BS2 + QueryUtils.check(random(), query, searcher); + + reader.close(); + w.close(); + dir.close(); + } }