From 7e3f89bee2cc2d0d472b0e266317147d92afd7ae Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Tue, 14 May 2024 18:42:26 +0200 Subject: [PATCH 01/10] iter --- ...actMultiTermQueryConstantScoreWrapper.java | 20 ----- .../lucene/search/Boolean2ScorerSupplier.java | 89 +++++++++++++++++++ .../apache/lucene/search/BooleanWeight.java | 12 --- .../lucene/search/ConstantScoreQuery.java | 23 ++--- .../lucene/search/IndexOrDocValuesQuery.java | 14 +-- .../apache/lucene/search/IndriAndWeight.java | 10 --- .../apache/lucene/search/LRUQueryCache.java | 84 ++++------------- .../lucene/search/MatchAllDocsQuery.java | 74 ++++++++------- .../apache/lucene/search/ScorerSupplier.java | 6 ++ .../java/org/apache/lucene/search/Weight.java | 24 +++-- 10 files changed, 183 insertions(+), 173 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java index b10aca74e948..e1e121cad52d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java @@ -211,26 +211,6 @@ private Scorer scorerForIterator(DocIdSetIterator iterator) { return new ConstantScoreScorer(this, score(), scoreMode, iterator); } - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - final Terms terms = context.reader().terms(q.getField()); - if (terms == null) { - return null; - } - final WeightOrDocIdSetIterator weightOrIterator = rewrite(context, terms); - if (weightOrIterator == null) { - return null; - } else if (weightOrIterator.weight != null) { - return weightOrIterator.weight.bulkScorer(context); - } else { - final Scorer scorer = scorerForIterator(weightOrIterator.iterator); - if (scorer == null) { - return null; - } - return new DefaultBulkScorer(scorer); - } - } - @Override public Matches matches(LeafReaderContext context, int doc) throws IOException { final Terms terms = context.reader().terms(q.field); diff --git a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java index 56b676b88cbb..bf56078038fa 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java @@ -25,7 +25,10 @@ import java.util.Map; import java.util.OptionalLong; import java.util.stream.Stream; + +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.BooleanClause.Occur; +import org.apache.lucene.search.BooleanWeight.WeightedBooleanClause; final class Boolean2ScorerSupplier extends ScorerSupplier { @@ -166,6 +169,92 @@ private Scorer getInternal(long leadCost) throws IOException { } } + @Override + public BulkScorer getBulkScorer() throws IOException { + final BulkScorer bulkScorer = booleanScorer(); + if (bulkScorer != null) { + // bulk scoring is applicable, use it + return bulkScorer; + } else { + // use a Scorer-based impl (BS2) + return super.getBulkScorer(); + } + } + + private BulkScorer booleanScorer() { + final int numOptionalClauses = subs.get(Occur.SHOULD).size(); + final int numRequiredClauses = subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size(); + + BulkScorer positiveScorer; + if (numRequiredClauses == 0) { + positiveScorer = optionalBulkScorer(context); + if (positiveScorer == null) { + return null; + } + + // TODO: what is the right heuristic here? + final long costThreshold; + if (query.getMinimumNumberShouldMatch() <= 1) { + // when all clauses are optional, use BooleanScorer aggressively + // TODO: is there actually a threshold under which we should rather + // use the regular scorer? + costThreshold = -1; + } else { + // when a minimum number of clauses should match, BooleanScorer is + // going to score all windows that have at least minNrShouldMatch + // matches in the window. But there is no way to know if there is + // an intersection (all clauses might match a different doc ID and + // there will be no matches in the end) so we should only use + // BooleanScorer if matches are very dense + costThreshold = context.reader().maxDoc() / 3; + } + + if (positiveScorer.cost() < costThreshold) { + return null; + } + + } else if (numRequiredClauses > 0 + && numOptionalClauses == 0 + && query.getMinimumNumberShouldMatch() == 0) { + positiveScorer = requiredBulkScorer(context); + } else { + // TODO: there are some cases where BooleanScorer + // would handle conjunctions faster than + // BooleanScorer2... + return null; + } + + if (positiveScorer == null) { + return null; + } + + List prohibited = new ArrayList<>(); + for (WeightedBooleanClause wc : weightedClauses) { + Weight w = wc.weight; + BooleanClause c = wc.clause; + if (c.isProhibited()) { + Scorer scorer = w.scorer(context); + if (scorer != null) { + prohibited.add(scorer); + } + } + } + + if (prohibited.isEmpty()) { + return positiveScorer; + } else { + Scorer prohibitedScorer = + prohibited.size() == 1 + ? prohibited.get(0) + : new DisjunctionSumScorer(this, prohibited, ScoreMode.COMPLETE_NO_SCORES); + if (prohibitedScorer.twoPhaseIterator() != null) { + // ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses + return null; + } + return new ReqExclBulkScorer(positiveScorer, prohibitedScorer.iterator()); + } + } + /** * Create a new scorer for the given required clauses. Note that {@code requiredScoring} is a * subset of {@code required} containing required clauses that should participate in scoring. 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 0417f3d598ce..fa3c61908542 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java @@ -435,18 +435,6 @@ BulkScorer booleanScorer(LeafReaderContext context) throws IOException { } } - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - final BulkScorer bulkScorer = booleanScorer(context); - if (bulkScorer != null) { - // bulk scoring is applicable, use it - return bulkScorer; - } else { - // use a Scorer-based impl (BS2) - return super.bulkScorer(context); - } - } - @Override public int count(LeafReaderContext context) throws IOException { final int numDocs = context.reader().numDocs(); diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java index d9f77841fda5..8a7e6141736e 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java @@ -136,17 +136,6 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo final Weight innerWeight = searcher.createWeight(query, innerScoreMode, 1f); if (scoreMode.needsScores()) { return new ConstantScoreWeight(this, boost) { - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - if (scoreMode.isExhaustive() == false) { - return super.bulkScorer(context); - } - final BulkScorer innerScorer = innerWeight.bulkScorer(context); - if (innerScorer == null) { - return null; - } - return new ConstantBulkScorer(innerScorer, this, score()); - } @Override public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { @@ -167,6 +156,18 @@ public Scorer get(long leadCost) throws IOException { } } + @Override + public BulkScorer getBulkScorer() throws IOException { + if (scoreMode.isExhaustive() == false) { + return super.getBulkScorer(); + } + final BulkScorer innerScorer = innerScorerSupplier.getBulkScorer(); + if (innerScorer == null) { + return null; + } + return new ConstantBulkScorer(innerScorer, innerWeight, score()); + } + @Override public long cost() { return innerScorerSupplier.cost(); diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java index cfabc0de4c16..cf652c4da824 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java @@ -148,13 +148,6 @@ public Explanation explain(LeafReaderContext context, int doc) throws IOExceptio return dvWeight.explain(context, doc); } - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - // Bulk scorers need to consume the entire set of docs, so using an - // index structure should perform better - return indexWeight.bulkScorer(context); - } - @Override public int count(LeafReaderContext context) throws IOException { final int count = indexWeight.count(context); @@ -186,6 +179,13 @@ public Scorer get(long leadCost) throws IOException { } } + @Override + public BulkScorer getBulkScorer() throws IOException { + // Bulk scorers need to consume the entire set of docs, so using an + // index structure should perform better + return indexScorerSupplier.getBulkScorer(); + } + @Override public long cost() { return indexScorerSupplier.cost(); diff --git a/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java b/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java index f33a3fbf7e60..58938be4333a 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndriAndWeight.java @@ -64,16 +64,6 @@ private Scorer getScorer(LeafReaderContext context) throws IOException { return scorer; } - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - Scorer scorer = getScorer(context); - if (scorer != null) { - BulkScorer bulkScorer = new DefaultBulkScorer(scorer); - return bulkScorer; - } - return null; - } - @Override public boolean isCacheable(LeafReaderContext ctx) { for (Weight w : weights) { diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java index a36622dcfcb1..390f8115bf88 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -711,15 +711,6 @@ private boolean cacheEntryHasReasonableWorstCaseSize(int maxDoc) { return worstCaseRamUsage * 5 < totalRamAvailable; } - private CacheAndCount cache(LeafReaderContext context) throws IOException { - final BulkScorer scorer = in.bulkScorer(context); - if (scorer == null) { - return CacheAndCount.EMPTY; - } else { - return cacheImpl(scorer, context.reader().maxDoc()); - } - } - /** Check whether this segment is eligible for caching, regardless of the query. */ private boolean shouldCache(LeafReaderContext context) throws IOException { return cacheEntryHasReasonableWorstCaseSize( @@ -779,9 +770,8 @@ public Scorer get(long leadCost) throws IOException { return supplier.get(leadCost); } - Scorer scorer = supplier.get(Long.MAX_VALUE); CacheAndCount cached = - cacheImpl(new DefaultBulkScorer(scorer), context.reader().maxDoc()); + cacheImpl(supplier.getBulkScorer(), context.reader().maxDoc()); putIfAbsent(in.getQuery(), cached, cacheHelper); DocIdSetIterator disi = cached.iterator(); if (disi == null) { @@ -794,6 +784,21 @@ public Scorer get(long leadCost) throws IOException { CachingWrapperWeight.this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi); } + @Override + public BulkScorer getBulkScorer() throws IOException { + CacheAndCount cached = + cacheImpl(supplier.getBulkScorer(), context.reader().maxDoc()); + putIfAbsent(in.getQuery(), cached, cacheHelper); + + DocIdSetIterator disi = cached.iterator(); + if (disi == null) { + disi = DocIdSetIterator.empty(); + } + + return new DefaultBulkScorer( + new ConstantScoreScorer(CachingWrapperWeight.this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi)); + } + @Override public long cost() { return cost; @@ -884,63 +889,6 @@ public int count(LeafReaderContext context) throws IOException { public boolean isCacheable(LeafReaderContext ctx) { return in.isCacheable(ctx); } - - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - if (used.compareAndSet(false, true)) { - policy.onUse(getQuery()); - } - - if (in.isCacheable(context) == false) { - // this segment is not suitable for caching - return in.bulkScorer(context); - } - - // Short-circuit: Check whether this segment is eligible for caching - // before we take a lock because of #get - if (shouldCache(context) == false) { - return in.bulkScorer(context); - } - - final IndexReader.CacheHelper cacheHelper = context.reader().getCoreCacheHelper(); - if (cacheHelper == null) { - // this reader has no cacheHelper - return in.bulkScorer(context); - } - - // If the lock is already busy, prefer using the uncached version than waiting - if (readLock.tryLock() == false) { - return in.bulkScorer(context); - } - - CacheAndCount cached; - try { - cached = get(in.getQuery(), cacheHelper); - } finally { - readLock.unlock(); - } - - if (cached == null) { - if (policy.shouldCache(in.getQuery())) { - cached = cache(context); - putIfAbsent(in.getQuery(), cached, cacheHelper); - } else { - return in.bulkScorer(context); - } - } - - assert cached != null; - if (cached == CacheAndCount.EMPTY) { - return null; - } - final DocIdSetIterator disi = cached.iterator(); - if (disi == null) { - return null; - } - - return new DefaultBulkScorer( - new ConstantScoreScorer(this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi)); - } } /** Cache of doc ids with a count. */ diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java index 533a14f50220..f39c2ee4e064 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java @@ -33,47 +33,57 @@ public String toString() { @Override public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { - final var scorer = - new ConstantScoreScorer( - this, score(), scoreMode, DocIdSetIterator.all(context.reader().maxDoc())); - return new DefaultScorerSupplier(scorer); - } - - @Override - public boolean isCacheable(LeafReaderContext ctx) { - return true; - } - - @Override - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - if (scoreMode.isExhaustive() == false) { - return super.bulkScorer(context); - } - final float score = score(); - final int maxDoc = context.reader().maxDoc(); - return new BulkScorer() { + final Weight weight = this; + return new ScorerSupplier() { + @Override - public int score(LeafCollector collector, Bits acceptDocs, int min, int max) - throws IOException { - max = Math.min(max, maxDoc); - Score scorer = new Score(); - scorer.score = score; - collector.setScorer(scorer); - for (int doc = min; doc < max; ++doc) { - if (acceptDocs == null || acceptDocs.get(doc)) { - collector.collect(doc); - } - } - return max == maxDoc ? DocIdSetIterator.NO_MORE_DOCS : max; + public Scorer get(long leadCost) throws IOException { + return new ConstantScoreScorer( + weight, score(), scoreMode, DocIdSetIterator.all(context.reader().maxDoc())); } + + @Override + public BulkScorer getBulkScorer() throws IOException { + if (scoreMode.isExhaustive() == false) { + return super.getBulkScorer(); + } + final float score = score(); + final int maxDoc = context.reader().maxDoc(); + return new BulkScorer() { + @Override + public int score(LeafCollector collector, Bits acceptDocs, int min, int max) + throws IOException { + max = Math.min(max, maxDoc); + Score scorer = new Score(); + scorer.score = score; + collector.setScorer(scorer); + for (int doc = min; doc < max; ++doc) { + if (acceptDocs == null || acceptDocs.get(doc)) { + collector.collect(doc); + } + } + return max == maxDoc ? DocIdSetIterator.NO_MORE_DOCS : max; + } + @Override + public long cost() { + return maxDoc; + } + }; + } + @Override public long cost() { - return maxDoc; + return context.reader().maxDoc(); } }; } + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return true; + } + @Override public int count(LeafReaderContext context) { return context.reader().numDocs(); diff --git a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java index 8053ac945402..46628972747c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java @@ -18,6 +18,8 @@ import java.io.IOException; +import org.apache.lucene.search.Weight.DefaultBulkScorer; + /** * A supplier of {@link Scorer}. This allows to get an estimate of the cost before building the * {@link Scorer}. @@ -35,6 +37,10 @@ public abstract class ScorerSupplier { */ public abstract Scorer get(long leadCost) throws IOException; + public BulkScorer getBulkScorer() throws IOException { + return new DefaultBulkScorer(get(Long.MAX_VALUE)); + } + /** * Get an estimate of the {@link Scorer} that would be returned by {@link #get}. This may be a * costly operation, so it should only be called if necessary. diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index 7ecca6658c7b..fe0b3eb1b9db 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -148,20 +148,19 @@ public final Scorer scorer(LeafReaderContext context) throws IOException { public abstract ScorerSupplier scorerSupplier(LeafReaderContext context) 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 which are not marked as deleted. + * Helper method that delegates to {@link #scorerSupplier(LeafReaderContext)}. It is implemented as + *
+   * ScorerSupplier scorerSupplier = scorerSupplier(context);
+   * if (scorerSupplier == null) {
+   *   // No docs match
+   *   return null;
+   * }
    *
-   * @param context the {@link org.apache.lucene.index.LeafReaderContext} for which to return the
-   *     {@link Scorer}.
-   * @return a {@link BulkScorer} which scores documents and passes them to a collector. Like {@link
-   *     #scorer(LeafReaderContext)}, this method can return null if this query matches no
-   *     documents.
-   * @throws IOException if there is a low-level I/O error
+   * scorerSupplier.setTopLevelScoringClause();
+   * return new DefaultBulkScorer(scorerSupplier.get(Long.MAX_VALUE));
+   * 
*/ - public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { - + public final BulkScorer bulkScorer(LeafReaderContext context) throws IOException { ScorerSupplier scorerSupplier = scorerSupplier(context); if (scorerSupplier == null) { // No docs match @@ -169,7 +168,6 @@ public BulkScorer bulkScorer(LeafReaderContext context) throws IOException { } scorerSupplier.setTopLevelScoringClause(); - return new DefaultBulkScorer(scorerSupplier.get(Long.MAX_VALUE)); } From 5bdd17c2bd397be5c849812f8a7067555a69358c Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Thu, 23 May 2024 11:07:19 +0200 Subject: [PATCH 02/10] iter --- .../lucene/search/Boolean2ScorerSupplier.java | 182 +++++++++++++++--- .../apache/lucene/search/BooleanScorer.java | 6 +- .../apache/lucene/search/BooleanWeight.java | 5 +- .../lucene/search/ConstantScoreQuery.java | 2 +- .../apache/lucene/search/IndexSearcher.java | 6 +- .../apache/lucene/search/LRUQueryCache.java | 18 +- .../lucene/search/MatchAllDocsQuery.java | 6 +- .../apache/lucene/search/ScorerSupplier.java | 1 - .../lucene/search/TimeLimitingBulkScorer.java | 2 +- .../java/org/apache/lucene/search/Weight.java | 6 +- .../search/TestBoolean2ScorerSupplier.java | 69 +++---- .../apache/lucene/search/TestBooleanOr.java | 2 +- .../lucene/search/TestBooleanScorer.java | 38 ++-- .../lucene/search/TestLRUQueryCache.java | 12 +- .../lucene/facet/DrillSidewaysQuery.java | 68 ++++--- .../sandbox/search/QueryProfilerWeight.java | 24 +-- .../suggest/document/CompletionWeight.java | 62 +++--- .../lucene/tests/search/AssertingWeight.java | 37 ++-- .../search/DisablingBulkScorerQuery.java | 26 ++- .../search/TestBaseExplanationTestCase.java | 6 - 20 files changed, 357 insertions(+), 221 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java index bf56078038fa..7cb5c45bdd72 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java @@ -23,12 +23,12 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.OptionalLong; import java.util.stream.Stream; - -import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.BooleanClause.Occur; -import org.apache.lucene.search.BooleanWeight.WeightedBooleanClause; +import org.apache.lucene.search.Weight.DefaultBulkScorer; +import org.apache.lucene.util.Bits; final class Boolean2ScorerSupplier extends ScorerSupplier { @@ -36,6 +36,7 @@ final class Boolean2ScorerSupplier extends ScorerSupplier { private final Map> subs; private final ScoreMode scoreMode; private final int minShouldMatch; + private final int maxDoc; private long cost = -1; private boolean topLevelScoringClause; @@ -43,7 +44,8 @@ final class Boolean2ScorerSupplier extends ScorerSupplier { Weight weight, Map> subs, ScoreMode scoreMode, - int minShouldMatch) { + int minShouldMatch, + int maxDoc) { if (minShouldMatch < 0) { throw new IllegalArgumentException( "minShouldMatch must be positive, but got: " + minShouldMatch); @@ -67,6 +69,7 @@ final class Boolean2ScorerSupplier extends ScorerSupplier { this.subs = subs; this.scoreMode = scoreMode; this.minShouldMatch = minShouldMatch; + this.maxDoc = maxDoc; } private long computeCost() { @@ -181,20 +184,20 @@ public BulkScorer getBulkScorer() throws IOException { } } - private BulkScorer booleanScorer() { + private BulkScorer booleanScorer() throws IOException { final int numOptionalClauses = subs.get(Occur.SHOULD).size(); final int numRequiredClauses = subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size(); - + BulkScorer positiveScorer; if (numRequiredClauses == 0) { - positiveScorer = optionalBulkScorer(context); + positiveScorer = optionalBulkScorer(); if (positiveScorer == null) { return null; } // TODO: what is the right heuristic here? final long costThreshold; - if (query.getMinimumNumberShouldMatch() <= 1) { + if (minShouldMatch <= 1) { // when all clauses are optional, use BooleanScorer aggressively // TODO: is there actually a threshold under which we should rather // use the regular scorer? @@ -206,17 +209,15 @@ private BulkScorer booleanScorer() { // an intersection (all clauses might match a different doc ID and // there will be no matches in the end) so we should only use // BooleanScorer if matches are very dense - costThreshold = context.reader().maxDoc() / 3; + costThreshold = maxDoc / 3; } if (positiveScorer.cost() < costThreshold) { return null; } - } else if (numRequiredClauses > 0 - && numOptionalClauses == 0 - && query.getMinimumNumberShouldMatch() == 0) { - positiveScorer = requiredBulkScorer(context); + } else if (numRequiredClauses > 0 && numOptionalClauses == 0 && minShouldMatch == 0) { + positiveScorer = requiredBulkScorer(); } else { // TODO: there are some cases where BooleanScorer // would handle conjunctions faster than @@ -227,17 +228,11 @@ private BulkScorer booleanScorer() { if (positiveScorer == null) { return null; } + final long positiveScorerCost = positiveScorer.cost(); List prohibited = new ArrayList<>(); - for (WeightedBooleanClause wc : weightedClauses) { - Weight w = wc.weight; - BooleanClause c = wc.clause; - if (c.isProhibited()) { - Scorer scorer = w.scorer(context); - if (scorer != null) { - prohibited.add(scorer); - } - } + for (ScorerSupplier ss : subs.get(Occur.MUST_NOT)) { + prohibited.add(ss.get(positiveScorerCost)); } if (prohibited.isEmpty()) { @@ -246,7 +241,7 @@ private BulkScorer booleanScorer() { Scorer prohibitedScorer = prohibited.size() == 1 ? prohibited.get(0) - : new DisjunctionSumScorer(this, prohibited, ScoreMode.COMPLETE_NO_SCORES); + : new DisjunctionSumScorer(weight, prohibited, ScoreMode.COMPLETE_NO_SCORES); if (prohibitedScorer.twoPhaseIterator() != null) { // ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses return null; @@ -255,6 +250,147 @@ private BulkScorer booleanScorer() { } } + static BulkScorer disableScoring(final BulkScorer scorer) { + Objects.requireNonNull(scorer); + return new BulkScorer() { + + @Override + public int score(final LeafCollector collector, Bits acceptDocs, int min, int max) + throws IOException { + final LeafCollector noScoreCollector = + new LeafCollector() { + Score fake = new Score(); + + @Override + public void setScorer(Scorable scorer) throws IOException { + collector.setScorer(fake); + } + + @Override + public void collect(int doc) throws IOException { + collector.collect(doc); + } + }; + return scorer.score(noScoreCollector, acceptDocs, min, max); + } + + @Override + public long cost() { + return scorer.cost(); + } + }; + } + + // Return a BulkScorer for the optional clauses only, + // or null if it is not applicable + // pkg-private for forcing use of BooleanScorer in tests + BulkScorer optionalBulkScorer() throws IOException { + if (subs.get(Occur.SHOULD).size() == 0) { + return null; + } else if (subs.get(Occur.SHOULD).size() == 1 && minShouldMatch <= 1) { + return subs.get(Occur.SHOULD).iterator().next().getBulkScorer(); + } + + if (scoreMode == ScoreMode.TOP_SCORES && minShouldMatch <= 1) { + List optionalScorers = new ArrayList<>(); + for (ScorerSupplier ss : subs.get(Occur.SHOULD)) { + optionalScorers.add(ss.get(Long.MAX_VALUE)); + } + + return new MaxScoreBulkScorer(maxDoc, optionalScorers); + } + + List optional = new ArrayList(); + for (ScorerSupplier ss : subs.get(Occur.SHOULD)) { + optional.add(ss.getBulkScorer()); + } + + return new BooleanScorer(optional, Math.max(1, minShouldMatch), scoreMode.needsScores()); + } + + // Return a BulkScorer for the required clauses only + private BulkScorer requiredBulkScorer() throws IOException { + if (subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() == 0) { + // No required clauses at all. + return null; + } else if (subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() == 1) { + BulkScorer scorer; + if (subs.get(Occur.MUST).isEmpty() == false) { + scorer = subs.get(Occur.MUST).iterator().next().getBulkScorer(); + } else { + scorer = subs.get(Occur.FILTER).iterator().next().getBulkScorer(); + if (scoreMode.needsScores()) { + scorer = disableScoring(scorer); + } + } + return scorer; + } + + long leadCost = + subs.get(Occur.MUST).stream().mapToLong(ScorerSupplier::cost).max().orElse(Long.MAX_VALUE); + leadCost = + subs.get(Occur.FILTER).stream().mapToLong(ScorerSupplier::cost).max().orElse(leadCost); + + List requiredNoScoring = new ArrayList<>(); + for (ScorerSupplier ss : subs.get(Occur.FILTER)) { + requiredNoScoring.add(ss.get(leadCost)); + } + List requiredScoring = new ArrayList<>(); + Collection requiredScoringSupplier = subs.get(Occur.MUST); + for (ScorerSupplier ss : requiredScoringSupplier) { + if (requiredScoringSupplier.size() == 1) { + ss.setTopLevelScoringClause(); + } + requiredScoring.add(ss.get(leadCost)); + } + if (scoreMode == ScoreMode.TOP_SCORES + && subs.get(Occur.FILTER).isEmpty() + && requiredScoring.size() > 1 + // Only specialize top-level conjunctions for clauses that don't have a two-phase iterator. + && requiredScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)) { + return new BlockMaxConjunctionBulkScorer(maxDoc, requiredScoring); + } + if (scoreMode != ScoreMode.TOP_SCORES + && requiredScoring.size() + requiredNoScoring.size() >= 2 + && requiredScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull) + && requiredNoScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)) { + return new ConjunctionBulkScorer(requiredScoring, requiredNoScoring); + } + if (scoreMode == ScoreMode.TOP_SCORES && requiredScoring.size() > 1) { + requiredScoring = + Collections.singletonList(new BlockMaxConjunctionScorer(weight, requiredScoring)); + } + Scorer conjunctionScorer; + if (requiredNoScoring.size() + requiredScoring.size() == 1) { + if (requiredScoring.size() == 1) { + conjunctionScorer = requiredScoring.get(0); + } else { + conjunctionScorer = requiredNoScoring.get(0); + if (scoreMode.needsScores()) { + Scorer inner = conjunctionScorer; + conjunctionScorer = + new FilterScorer(inner) { + @Override + public float score() throws IOException { + return 0f; + } + + @Override + public float getMaxScore(int upTo) throws IOException { + return 0f; + } + }; + } + } + } else { + List required = new ArrayList<>(); + required.addAll(requiredScoring); + required.addAll(requiredNoScoring); + conjunctionScorer = new ConjunctionScorer(weight, required, requiredScoring); + } + return new DefaultBulkScorer(conjunctionScorer); + } + /** * Create a new scorer for the given required clauses. Note that {@code requiredScoring} is a * subset of {@code required} containing required clauses that should participate in scoring. 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 6ab767f4ee81..919cc1680cf0 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java @@ -194,11 +194,7 @@ public int count() throws IOException { private final DocIdStreamView docIdStreamView = new DocIdStreamView(); - BooleanScorer( - BooleanWeight weight, - Collection scorers, - int minShouldMatch, - boolean needsScores) { + BooleanScorer(Collection scorers, int minShouldMatch, boolean needsScores) { if (minShouldMatch < 1 || minShouldMatch > scorers.size()) { throw new IllegalArgumentException( "minShouldMatch should be within 1..num_scorers. Got " + minShouldMatch); 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 fa3c61908542..bfc25ead0617 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java @@ -248,7 +248,7 @@ BulkScorer optionalBulkScorer(LeafReaderContext context) throws IOException { } return new BooleanScorer( - this, optional, Math.max(1, query.getMinimumNumberShouldMatch()), scoreMode.needsScores()); + optional, Math.max(1, query.getMinimumNumberShouldMatch()), scoreMode.needsScores()); } // Return a BulkScorer for the required clauses only @@ -607,6 +607,7 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti scorers.get(Occur.SHOULD).clear(); } - return new Boolean2ScorerSupplier(this, scorers, scoreMode, minShouldMatch); + return new Boolean2ScorerSupplier( + this, scorers, scoreMode, minShouldMatch, context.reader().maxDoc()); } } diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java index 8a7e6141736e..ec1741a4b1a1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java @@ -167,7 +167,7 @@ public BulkScorer getBulkScorer() throws IOException { } return new ConstantBulkScorer(innerScorer, innerWeight, score()); } - + @Override public long cost() { return innerScorerSupplier.cost(); diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java index 3c2a79e196a6..a8104f8f5a99 100644 --- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java +++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java @@ -704,8 +704,10 @@ protected void search(List leaves, Weight weight, Collector c // continue with the following leaf continue; } - BulkScorer scorer = weight.bulkScorer(ctx); - if (scorer != null) { + ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx); + if (scorerSupplier != null) { + scorerSupplier.setTopLevelScoringClause(); + BulkScorer scorer = scorerSupplier.getBulkScorer(); if (queryTimeout != null) { scorer = new TimeLimitingBulkScorer(scorer, queryTimeout); } diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java index 390f8115bf88..d59860126ef6 100644 --- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java +++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java @@ -770,8 +770,7 @@ public Scorer get(long leadCost) throws IOException { return supplier.get(leadCost); } - CacheAndCount cached = - cacheImpl(supplier.getBulkScorer(), context.reader().maxDoc()); + CacheAndCount cached = cacheImpl(supplier.getBulkScorer(), context.reader().maxDoc()); putIfAbsent(in.getQuery(), cached, cacheHelper); DocIdSetIterator disi = cached.iterator(); if (disi == null) { @@ -784,21 +783,6 @@ public Scorer get(long leadCost) throws IOException { CachingWrapperWeight.this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi); } - @Override - public BulkScorer getBulkScorer() throws IOException { - CacheAndCount cached = - cacheImpl(supplier.getBulkScorer(), context.reader().maxDoc()); - putIfAbsent(in.getQuery(), cached, cacheHelper); - - DocIdSetIterator disi = cached.iterator(); - if (disi == null) { - disi = DocIdSetIterator.empty(); - } - - return new DefaultBulkScorer( - new ConstantScoreScorer(CachingWrapperWeight.this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi)); - } - @Override public long cost() { return cost; diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java index f39c2ee4e064..9301657e291b 100644 --- a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java @@ -35,13 +35,13 @@ public String toString() { public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException { final Weight weight = this; return new ScorerSupplier() { - + @Override public Scorer get(long leadCost) throws IOException { return new ConstantScoreScorer( weight, score(), scoreMode, DocIdSetIterator.all(context.reader().maxDoc())); } - + @Override public BulkScorer getBulkScorer() throws IOException { if (scoreMode.isExhaustive() == false) { @@ -71,7 +71,7 @@ public long cost() { } }; } - + @Override public long cost() { return context.reader().maxDoc(); diff --git a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java index 46628972747c..67a21e4aa3cf 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java @@ -17,7 +17,6 @@ package org.apache.lucene.search; import java.io.IOException; - import org.apache.lucene.search.Weight.DefaultBulkScorer; /** diff --git a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java index 517f0a0e77b2..240dee7f4fd1 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingBulkScorer.java @@ -61,7 +61,7 @@ public Throwable fillInStackTrace() { * TimeLimitingBulkScorer.TimeExceededException} is thrown */ public TimeLimitingBulkScorer(BulkScorer bulkScorer, QueryTimeout queryTimeout) { - this.in = bulkScorer; + this.in = Objects.requireNonNull(bulkScorer); this.queryTimeout = Objects.requireNonNull(queryTimeout); } diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java index fe0b3eb1b9db..ce8e560cf8e9 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -148,7 +148,9 @@ public final Scorer scorer(LeafReaderContext context) throws IOException { public abstract ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException; /** - * Helper method that delegates to {@link #scorerSupplier(LeafReaderContext)}. It is implemented as + * Helper method that delegates to {@link #scorerSupplier(LeafReaderContext)}. It is implemented + * as + * *
    * ScorerSupplier scorerSupplier = scorerSupplier(context);
    * if (scorerSupplier == null) {
@@ -168,7 +170,7 @@ public final BulkScorer bulkScorer(LeafReaderContext context) throws IOException
     }
 
     scorerSupplier.setTopLevelScoringClause();
-    return new DefaultBulkScorer(scorerSupplier.get(Long.MAX_VALUE));
+    return scorerSupplier.getBulkScorer();
   }
 
   /**
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java
index eab15389b00e..30d7c23d25ef 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java
@@ -138,7 +138,7 @@ public void testConjunctionCost() {
     assertEquals(
         42,
         new Boolean2ScorerSupplier(
-                null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+                null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
             .cost());
 
     subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST)))
@@ -146,7 +146,7 @@ public void testConjunctionCost() {
     assertEquals(
         12,
         new Boolean2ScorerSupplier(
-                null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+                null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
             .cost());
 
     subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST)))
@@ -154,7 +154,7 @@ public void testConjunctionCost() {
     assertEquals(
         12,
         new Boolean2ScorerSupplier(
-                null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+                null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
             .cost());
   }
 
@@ -167,21 +167,21 @@ public void testDisjunctionCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
     ScorerSupplier s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100);
     assertEquals(42, s.cost());
     assertEquals(42, s.get(random().nextInt(100)).iterator().cost());
 
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100);
     assertEquals(42 + 12, s.cost());
     assertEquals(42 + 12, s.get(random().nextInt(100)).iterator().cost());
 
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100);
     assertEquals(42 + 12 + 20, s.cost());
     assertEquals(42 + 12 + 20, s.get(random().nextInt(100)).iterator().cost());
   }
@@ -196,36 +196,36 @@ public void testDisjunctionWithMinShouldMatchCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
     ScorerSupplier s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1, 100);
     assertEquals(42 + 12, s.cost());
     assertEquals(42 + 12, s.get(random().nextInt(100)).iterator().cost());
 
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20));
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1, 100);
     assertEquals(42 + 12 + 20, s.cost());
     assertEquals(42 + 12 + 20, s.get(random().nextInt(100)).iterator().cost());
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100);
     assertEquals(12 + 20, s.cost());
     assertEquals(12 + 20, s.get(random().nextInt(100)).iterator().cost());
 
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30));
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1, 100);
     assertEquals(42 + 12 + 20 + 30, s.cost());
     assertEquals(42 + 12 + 20 + 30, s.get(random().nextInt(100)).iterator().cost());
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100);
     assertEquals(12 + 20 + 30, s.cost());
     assertEquals(12 + 20 + 30, s.get(random().nextInt(100)).iterator().cost());
     s =
         new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3);
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3, 100);
     assertEquals(12 + 20, s.cost());
     assertEquals(12 + 20, s.get(random().nextInt(100)).iterator().cost());
   }
@@ -261,7 +261,7 @@ public void testDuelCost() throws Exception {
       }
       int minShouldMatch = numShoulds == 0 ? 0 : TestUtil.nextInt(random(), 0, numShoulds - 1);
       Boolean2ScorerSupplier supplier =
-          new Boolean2ScorerSupplier(new FakeWeight(), subs, scoreMode, minShouldMatch);
+          new Boolean2ScorerSupplier(new FakeWeight(), subs, scoreMode, minShouldMatch, 100);
       long cost1 = supplier.cost();
       long cost2 = supplier.get(Long.MAX_VALUE).iterator().cost();
       assertEquals("clauses=" + subs + ", minShouldMatch=" + minShouldMatch, cost1, cost2);
@@ -288,7 +288,7 @@ public void testConjunctionLeadCost() throws IOException {
     subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST)))
         .add(new FakeScorerSupplier(12, 12));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(Long.MAX_VALUE); // triggers assertions as a side-effect
 
     subs = new EnumMap<>(Occur.class);
@@ -302,7 +302,7 @@ public void testConjunctionLeadCost() throws IOException {
     subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST)))
         .add(new FakeScorerSupplier(12, 7));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(7); // triggers assertions as a side-effect
   }
 
@@ -314,14 +314,14 @@ public void testDisjunctionLeadCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 54));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 54));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs.get(Occur.SHOULD).clear();
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 20));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 20));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(20); // triggers assertions as a side-effect
   }
 
@@ -337,7 +337,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 42));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 42));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs = new EnumMap<>(Occur.class);
@@ -350,7 +350,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 20));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 20));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100)
         .get(20); // triggers assertions as a side-effect
 
     subs = new EnumMap<>(Occur.class);
@@ -363,7 +363,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 62));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, 62));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs = new EnumMap<>(Occur.class);
@@ -376,7 +376,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException {
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 32));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, 32));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3, 100)
         .get(100); // triggers assertions as a side-effect
   }
 
@@ -390,7 +390,7 @@ public void testProhibitedLeadCost() throws IOException {
     subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
     subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(30, 42));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs.get(Occur.MUST).clear();
@@ -398,7 +398,7 @@ public void testProhibitedLeadCost() throws IOException {
     subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
     subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(80, 42));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs.get(Occur.MUST).clear();
@@ -406,7 +406,7 @@ public void testProhibitedLeadCost() throws IOException {
     subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 20));
     subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(30, 20));
     new Boolean2ScorerSupplier(
-            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
+            new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
         .get(20); // triggers assertions as a side-effect
   }
 
@@ -419,21 +419,21 @@ public void testMixedLeadCost() throws IOException {
     // The SHOULD clause is always called with the same lead cost as the MUST clause
     subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 42));
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs.get(Occur.MUST).clear();
     subs.get(Occur.SHOULD).clear();
     subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(80, 42));
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100)
         .get(100); // triggers assertions as a side-effect
 
     subs.get(Occur.MUST).clear();
     subs.get(Occur.SHOULD).clear();
     subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 20));
     subs.get(Occur.SHOULD).add(new FakeScorerSupplier(80, 20));
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100)
         .get(20); // triggers assertions as a side-effect
   }
 
@@ -448,7 +448,7 @@ public void testDisjunctionTopLevelScoringClause() throws Exception {
     FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
     subs.get(Occur.SHOULD).add(clause2);
 
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
         .setTopLevelScoringClause();
     assertFalse(clause1.topLevelScoringClause);
     assertFalse(clause2.topLevelScoringClause);
@@ -465,7 +465,7 @@ public void testConjunctionTopLevelScoringClause() throws Exception {
     FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
     subs.get(Occur.MUST).add(clause2);
 
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
         .setTopLevelScoringClause();
     assertFalse(clause1.topLevelScoringClause);
     assertFalse(clause2.topLevelScoringClause);
@@ -482,7 +482,7 @@ public void testFilterTopLevelScoringClause() throws Exception {
     FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
     subs.get(Occur.FILTER).add(clause2);
 
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
         .setTopLevelScoringClause();
     assertFalse(clause1.topLevelScoringClause);
     assertFalse(clause2.topLevelScoringClause);
@@ -499,7 +499,7 @@ public void testSingleMustScoringClause() throws Exception {
     FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
     subs.get(Occur.FILTER).add(clause2);
 
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
         .setTopLevelScoringClause();
     assertTrue(clause1.topLevelScoringClause);
     assertFalse(clause2.topLevelScoringClause);
@@ -516,7 +516,7 @@ public void testSingleShouldScoringClause() throws Exception {
     FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
     subs.get(Occur.MUST_NOT).add(clause2);
 
-    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
+    new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
         .setTopLevelScoringClause();
     assertTrue(clause1.topLevelScoringClause);
     assertFalse(clause2.topLevelScoringClause);
@@ -534,7 +534,7 @@ public void testMaxScoreNonTopLevelScoringClause() throws Exception {
     subs.get(Occur.MUST).add(clause2);
 
     Scorer scorer =
-        new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0).get(10);
+        new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10);
     assertEquals(2.0, scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0.0);
 
     subs = new EnumMap<>(Occur.class);
@@ -545,7 +545,8 @@ public void testMaxScoreNonTopLevelScoringClause() throws Exception {
     subs.get(Occur.SHOULD).add(clause1);
     subs.get(Occur.SHOULD).add(clause2);
 
-    scorer = new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0).get(10);
+    scorer =
+        new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10);
     assertEquals(2.0, scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0.0);
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
index d895b269b75c..b54dbeff9ef6 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
@@ -246,7 +246,7 @@ public void testSubScorerNextIsNotMatch() throws IOException {
             scorer(4000, 1000051),
             scorer(5000, 100000, 9999998, 9999999));
     Collections.shuffle(optionalScorers, random());
-    BooleanScorer scorer = new BooleanScorer(null, optionalScorers, 1, random().nextBoolean());
+    BooleanScorer scorer = new BooleanScorer(optionalScorers, 1, random().nextBoolean());
     final List matches = new ArrayList<>();
     scorer.score(
         new LeafCollector() {
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 87941b745227..d00279fc1a26 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
@@ -94,6 +94,25 @@ public Scorer get(long leadCost) throws IOException {
             public long cost() {
               throw new UnsupportedOperationException();
             }
+
+            @Override
+            public BulkScorer getBulkScorer() throws IOException {
+              return new BulkScorer() {
+                @Override
+                public int score(LeafCollector collector, Bits acceptDocs, int min, int max)
+                    throws IOException {
+                  assert min == 0;
+                  collector.setScorer(new Score());
+                  collector.collect(0);
+                  return DocIdSetIterator.NO_MORE_DOCS;
+                }
+
+                @Override
+                public long cost() {
+                  return 1;
+                }
+              };
+            }
           };
         }
 
@@ -101,25 +120,6 @@ public long cost() {
         public boolean isCacheable(LeafReaderContext ctx) {
           return false;
         }
-
-        @Override
-        public BulkScorer bulkScorer(LeafReaderContext context) {
-          return new BulkScorer() {
-            @Override
-            public int score(LeafCollector collector, Bits acceptDocs, int min, int max)
-                throws IOException {
-              assert min == 0;
-              collector.setScorer(new Score());
-              collector.collect(0);
-              return DocIdSetIterator.NO_MORE_DOCS;
-            }
-
-            @Override
-            public long cost() {
-              return 1;
-            }
-          };
-        }
       };
     }
 
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index dc7b607707d3..7c5b0ad61cc1 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -1368,18 +1368,18 @@ public Scorer get(long leadCost) throws IOException {
           return scorer;
         }
 
+        @Override
+        public BulkScorer getBulkScorer() throws IOException {
+          bulkScorerCalled.set(true);
+          return in.bulkScorer(context);
+        }
+
         @Override
         public long cost() {
           return scorer.iterator().cost();
         }
       };
     }
-
-    @Override
-    public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-      bulkScorerCalled.set(true);
-      return in.bulkScorer(context);
-    }
   }
 
   public void testPropagateBulkScorer() throws IOException {
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
index d5724c7dd81f..3b8939bb88f9 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
@@ -154,35 +154,7 @@ public Explanation explain(LeafReaderContext context, int doc) throws IOExceptio
 
       @Override
       public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
-        return new ScorerSupplier() {
-          @Override
-          public Scorer get(long leadCost) throws IOException {
-            // We can only run as a top scorer:
-            throw new UnsupportedOperationException();
-          }
-
-          @Override
-          public long cost() {
-            throw new UnsupportedOperationException();
-          }
-        };
-      }
-
-      @Override
-      public boolean isCacheable(LeafReaderContext ctx) {
-        // We can never cache DSQ instances. It's critical that the BulkScorer produced by this
-        // Weight runs through the "normal" execution path so that it has access to an
-        // "acceptDocs" instance that accurately reflects deleted docs. During caching,
-        // "acceptDocs" is null so that caching over-matches (since the final BulkScorer would
-        // account for deleted docs). The problem is that this BulkScorer has a side-effect of
-        // populating the "sideways" FacetsCollectors, so it will use deleted docs in its
-        // sideways counting if caching kicks in. See LUCENE-10060:
-        return false;
-      }
-
-      @Override
-      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        Scorer baseScorer = baseWeight.scorer(context);
+        ScorerSupplier baseScorerSupplier = baseWeight.scorerSupplier(context);
 
         int drillDownCount = drillDowns.length;
 
@@ -223,7 +195,7 @@ public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
         // If baseScorer is null or the dim nullCount > 1, then we have nothing to score. We return
         // a null scorer in this case, but we need to make sure #finish gets called on all facet
         // collectors since IndexSearcher won't handle this for us:
-        if (baseScorer == null || nullCount > 1) {
+        if (baseScorerSupplier == null || nullCount > 1) {
           if (drillDownCollector != null) {
             drillDownCollector.finish();
           }
@@ -236,8 +208,40 @@ public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
         // Sort drill-downs by most restrictive first:
         Arrays.sort(dims, Comparator.comparingLong(o -> o.approximation.cost()));
 
-        return new DrillSidewaysScorer(
-            context, baseScorer, drillDownLeafCollector, dims, scoreSubDocsAtOnce);
+        return new ScorerSupplier() {
+          @Override
+          public Scorer get(long leadCost) throws IOException {
+            // We can only run as a top scorer:
+            throw new UnsupportedOperationException();
+          }
+
+          @Override
+          public BulkScorer getBulkScorer() throws IOException {
+            return new DrillSidewaysScorer(
+                context,
+                baseScorerSupplier.get(Long.MAX_VALUE),
+                drillDownLeafCollector,
+                dims,
+                scoreSubDocsAtOnce);
+          }
+
+          @Override
+          public long cost() {
+            throw new UnsupportedOperationException();
+          }
+        };
+      }
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        // We can never cache DSQ instances. It's critical that the BulkScorer produced by this
+        // Weight runs through the "normal" execution path so that it has access to an
+        // "acceptDocs" instance that accurately reflects deleted docs. During caching,
+        // "acceptDocs" is null so that caching over-matches (since the final BulkScorer would
+        // account for deleted docs). The problem is that this BulkScorer has a side-effect of
+        // populating the "sideways" FacetsCollectors, so it will use deleted docs in its
+        // sideways counting if caching kicks in. See LUCENE-10060:
+        return false;
       }
     };
   }
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java
index 786dff06bed9..7b0d26b2e10f 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java
@@ -76,6 +76,18 @@ public Scorer get(long loadCost) throws IOException {
         }
       }
 
+      @Override
+      public BulkScorer getBulkScorer() throws IOException {
+        // We use the default bulk scorer instead of the specialized one. The reason
+        // is that BulkScorers do everything at once: finding matches,
+        // scoring them and calling the collector, so they make it impossible to
+        // see where time is spent, which is the purpose of query profiling.
+        // The default bulk scorer will pull a scorer and iterate over matches,
+        // this might be a significantly different execution path for some queries
+        // like disjunctions, but in general this is what is done anyway
+        return super.getBulkScorer();
+      }
+
       @Override
       public long cost() {
         timer.start();
@@ -93,18 +105,6 @@ public void setTopLevelScoringClause() throws IOException {
     };
   }
 
-  @Override
-  public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-    // We use the default bulk scorer instead of the specialized one. The reason
-    // is that BulkScorers do everything at once: finding matches,
-    // scoring them and calling the collector, so they make it impossible to
-    // see where time is spent, which is the purpose of query profiling.
-    // The default bulk scorer will pull a scorer and iterate over matches,
-    // this might be a significantly different execution path for some queries
-    // like disjunctions, but in general this is what is done anyway
-    return super.bulkScorer(context);
-  }
-
   @Override
   public boolean isCacheable(LeafReaderContext ctx) {
     return false;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
index 800e6e113522..8b9a89352743 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java
@@ -62,36 +62,6 @@ public Automaton getAutomaton() {
     return automaton;
   }
 
-  @Override
-  public BulkScorer bulkScorer(final LeafReaderContext context) throws IOException {
-    final LeafReader reader = context.reader();
-    final Terms terms;
-    final NRTSuggester suggester;
-    if ((terms = reader.terms(completionQuery.getField())) == null) {
-      return null;
-    }
-    if (terms instanceof CompletionTerms) {
-      CompletionTerms completionTerms = (CompletionTerms) terms;
-      if ((suggester = completionTerms.suggester()) == null) {
-        // a segment can have a null suggester
-        // i.e. no FST was built
-        return null;
-      }
-    } else {
-      throw new IllegalArgumentException(completionQuery.getField() + " is not a SuggestField");
-    }
-
-    BitsProducer filter = completionQuery.getFilter();
-    Bits filteredDocs = null;
-    if (filter != null) {
-      filteredDocs = filter.getBits(context);
-      if (filteredDocs.getClass() == Bits.MatchNoBits.class) {
-        return null;
-      }
-    }
-    return new CompletionScorer(this, suggester, reader, filteredDocs, filter != null, automaton);
-  }
-
   /**
    * Set for every partial path in the index that matched the query automaton.
    *
@@ -122,12 +92,44 @@ protected CharSequence context() {
 
   @Override
   public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+    final CompletionWeight weight = this;
     return new ScorerSupplier() {
       @Override
       public Scorer get(long leadCost) throws IOException {
         throw new UnsupportedOperationException();
       }
 
+      @Override
+      public BulkScorer getBulkScorer() throws IOException {
+        final LeafReader reader = context.reader();
+        final Terms terms;
+        final NRTSuggester suggester;
+        if ((terms = reader.terms(completionQuery.getField())) == null) {
+          return null;
+        }
+        if (terms instanceof CompletionTerms) {
+          CompletionTerms completionTerms = (CompletionTerms) terms;
+          if ((suggester = completionTerms.suggester()) == null) {
+            // a segment can have a null suggester
+            // i.e. no FST was built
+            return null;
+          }
+        } else {
+          throw new IllegalArgumentException(completionQuery.getField() + " is not a SuggestField");
+        }
+
+        BitsProducer filter = completionQuery.getFilter();
+        Bits filteredDocs = null;
+        if (filter != null) {
+          filteredDocs = filter.getBits(context);
+          if (filteredDocs.getClass() == Bits.MatchNoBits.class) {
+            return null;
+          }
+        }
+        return new CompletionScorer(
+            weight, suggester, reader, filteredDocs, filter != null, automaton);
+      }
+
       @Override
       public long cost() {
         throw new UnsupportedOperationException();
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java
index 3368599d3652..075db88b4795 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java
@@ -78,6 +78,25 @@ public Scorer get(long leadCost) throws IOException {
             topLevelScoringClause);
       }
 
+      @Override
+      public BulkScorer getBulkScorer() throws IOException {
+        assert getCalled == false;
+
+        BulkScorer inScorer;
+        // We explicitly test both the delegate's bulk scorer, and also the normal scorer.
+        // This ensures that normal scorers are sometimes tested with an asserting wrapper.
+        if (usually(random)) {
+          getCalled = true;
+          inScorer = inScorerSupplier.getBulkScorer();
+        } else {
+          // Don't set getCalled = true, since this calls #get under the hood
+          inScorer = super.getBulkScorer();
+        }
+
+        return AssertingBulkScorer.wrap(
+            new Random(random.nextLong()), inScorer, context.reader().maxDoc(), scoreMode);
+      }
+
       @Override
       public long cost() {
         final long cost = inScorerSupplier.cost();
@@ -93,22 +112,4 @@ public void setTopLevelScoringClause() throws IOException {
       }
     };
   }
-
-  @Override
-  public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-    BulkScorer inScorer;
-    // We explicitly test both the delegate's bulk scorer, and also the normal scorer.
-    // This ensures that normal scorers are sometimes tested with an asserting wrapper.
-    if (usually(random)) {
-      inScorer = in.bulkScorer(context);
-    } else {
-      inScorer = super.bulkScorer(context);
-    }
-
-    if (inScorer == null) {
-      return null;
-    }
-    return AssertingBulkScorer.wrap(
-        new Random(random.nextLong()), inScorer, context.reader().maxDoc(), scoreMode);
-  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java
index e6cee512dece..23dafb2707ff 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java
@@ -25,6 +25,7 @@
 import org.apache.lucene.search.QueryVisitor;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.ScorerSupplier;
 import org.apache.lucene.search.Weight;
 
 /** A {@link Query} wrapper that disables bulk-scoring optimizations. */
@@ -52,12 +53,25 @@ public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float bo
     Weight in = query.createWeight(searcher, scoreMode, boost);
     return new FilterWeight(in) {
       @Override
-      public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-        Scorer scorer = scorer(context);
-        if (scorer == null) {
-          return null;
-        }
-        return new DefaultBulkScorer(scorer);
+      public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+        ScorerSupplier scorerSupplier = super.scorerSupplier(context);
+        return new ScorerSupplier() {
+
+          @Override
+          public Scorer get(long leadCost) throws IOException {
+            return scorerSupplier.get(leadCost);
+          }
+
+          @Override
+          public long cost() {
+            return scorerSupplier.cost();
+          }
+
+          @Override
+          public BulkScorer getBulkScorer() throws IOException {
+            return new DefaultBulkScorer(get(Long.MAX_VALUE));
+          }
+        };
       }
     };
   }
diff --git a/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java b/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java
index 5dcc1658df6d..6d184fbbe3fd 100644
--- a/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java
+++ b/lucene/test-framework/src/test/org/apache/lucene/tests/search/TestBaseExplanationTestCase.java
@@ -19,7 +19,6 @@
 import java.io.IOException;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.BulkScorer;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.FilterWeight;
 import org.apache.lucene.search.IndexSearcher;
@@ -100,11 +99,6 @@ public BrokenExplainWeight(BrokenExplainTermQuery q, Weight in) {
       super(q, in);
     }
 
-    @Override
-    public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-      return in.bulkScorer(context);
-    }
-
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
       BrokenExplainTermQuery q = (BrokenExplainTermQuery) this.getQuery();

From d8f9a4f8b6e556369a68aaa9bf1c814278e8140b Mon Sep 17 00:00:00 2001
From: Adrien Grand 
Date: Thu, 23 May 2024 15:56:22 +0200
Subject: [PATCH 03/10] iter

---
 .../lucene/search/Boolean2ScorerSupplier.java |  20 +-
 .../apache/lucene/search/BooleanWeight.java   | 219 +-----------------
 .../lucene/search/ReqExclBulkScorer.java      |  28 ++-
 .../lucene/search/TestBooleanScorer.java      |  18 +-
 .../lucene/search/TestReqExclBulkScorer.java  |   1 +
 .../lucene/tests/search/AssertingWeight.java  |  13 ++
 6 files changed, 54 insertions(+), 245 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
index 7cb5c45bdd72..da2ca72dea10 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
@@ -184,17 +184,12 @@ public BulkScorer getBulkScorer() throws IOException {
     }
   }
 
-  private BulkScorer booleanScorer() throws IOException {
+  BulkScorer booleanScorer() throws IOException {
     final int numOptionalClauses = subs.get(Occur.SHOULD).size();
     final int numRequiredClauses = subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size();
 
     BulkScorer positiveScorer;
     if (numRequiredClauses == 0) {
-      positiveScorer = optionalBulkScorer();
-      if (positiveScorer == null) {
-        return null;
-      }
-
       // TODO: what is the right heuristic here?
       final long costThreshold;
       if (minShouldMatch <= 1) {
@@ -212,10 +207,11 @@ private BulkScorer booleanScorer() throws IOException {
         costThreshold = maxDoc / 3;
       }
 
-      if (positiveScorer.cost() < costThreshold) {
+      if (cost() < costThreshold) {
         return null;
       }
 
+      positiveScorer = optionalBulkScorer();
     } else if (numRequiredClauses > 0 && numOptionalClauses == 0 && minShouldMatch == 0) {
       positiveScorer = requiredBulkScorer();
     } else {
@@ -242,11 +238,7 @@ private BulkScorer booleanScorer() throws IOException {
           prohibited.size() == 1
               ? prohibited.get(0)
               : new DisjunctionSumScorer(weight, prohibited, ScoreMode.COMPLETE_NO_SCORES);
-      if (prohibitedScorer.twoPhaseIterator() != null) {
-        // ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses
-        return null;
-      }
-      return new ReqExclBulkScorer(positiveScorer, prohibitedScorer.iterator());
+      return new ReqExclBulkScorer(positiveScorer, prohibitedScorer);
     }
   }
 
@@ -327,9 +319,9 @@ private BulkScorer requiredBulkScorer() throws IOException {
     }
 
     long leadCost =
-        subs.get(Occur.MUST).stream().mapToLong(ScorerSupplier::cost).max().orElse(Long.MAX_VALUE);
+        subs.get(Occur.MUST).stream().mapToLong(ScorerSupplier::cost).min().orElse(Long.MAX_VALUE);
     leadCost =
-        subs.get(Occur.FILTER).stream().mapToLong(ScorerSupplier::cost).max().orElse(leadCost);
+        subs.get(Occur.FILTER).stream().mapToLong(ScorerSupplier::cost).min().orElse(leadCost);
 
     List requiredNoScoring = new ArrayList<>();
     for (ScorerSupplier ss : subs.get(Occur.FILTER)) {
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 bfc25ead0617..617a59d3e7e3 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -19,15 +19,13 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.EnumMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Objects;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.util.Bits;
 
 /** Expert: the Weight for BooleanQuery, used to normalize, score and explain these queries. */
 final class BooleanWeight extends Weight {
@@ -156,37 +154,6 @@ public Matches matches(LeafReaderContext context, int doc) throws IOException {
     return MatchesUtils.fromSubMatches(matches);
   }
 
-  static BulkScorer disableScoring(final BulkScorer scorer) {
-    Objects.requireNonNull(scorer);
-    return new BulkScorer() {
-
-      @Override
-      public int score(final LeafCollector collector, Bits acceptDocs, int min, int max)
-          throws IOException {
-        final LeafCollector noScoreCollector =
-            new LeafCollector() {
-              Score fake = new Score();
-
-              @Override
-              public void setScorer(Scorable scorer) throws IOException {
-                collector.setScorer(fake);
-              }
-
-              @Override
-              public void collect(int doc) throws IOException {
-                collector.collect(doc);
-              }
-            };
-        return scorer.score(noScoreCollector, acceptDocs, min, max);
-      }
-
-      @Override
-      public long cost() {
-        return scorer.cost();
-      }
-    };
-  }
-
   // Return a BulkScorer for the optional clauses only,
   // or null if it is not applicable
   // pkg-private for forcing use of BooleanScorer in tests
@@ -251,190 +218,6 @@ BulkScorer optionalBulkScorer(LeafReaderContext context) throws IOException {
         optional, Math.max(1, query.getMinimumNumberShouldMatch()), scoreMode.needsScores());
   }
 
-  // Return a BulkScorer for the required clauses only
-  private BulkScorer requiredBulkScorer(LeafReaderContext context) throws IOException {
-    // Is there a single required clause by any chance? Then pull its bulk scorer.
-    List requiredClauses = new ArrayList<>();
-    for (WeightedBooleanClause wc : weightedClauses) {
-      if (wc.clause.isRequired()) {
-        requiredClauses.add(wc);
-      }
-    }
-
-    if (requiredClauses.isEmpty()) {
-      // No required clauses at all.
-      return null;
-    } else if (requiredClauses.size() == 1) {
-      WeightedBooleanClause clause = requiredClauses.get(0);
-      BulkScorer scorer = clause.weight.bulkScorer(context);
-      if (scorer == null) {
-        return null;
-      }
-      if (clause.clause.isScoring() == false && scoreMode.needsScores()) {
-        scorer = disableScoring(scorer);
-      }
-      return scorer;
-    }
-
-    List requiredNoScoringSupplier = new ArrayList<>();
-    List requiredScoringSupplier = new ArrayList<>();
-
-    long leadCost = Long.MAX_VALUE;
-    for (WeightedBooleanClause wc : requiredClauses) {
-      Weight w = wc.weight;
-      BooleanClause c = wc.clause;
-      ScorerSupplier scorerSupplier = w.scorerSupplier(context);
-      if (scorerSupplier == null) {
-        // One clause doesn't have matches, so the entire conjunction doesn't have matches.
-        return null;
-      }
-      leadCost = Math.min(leadCost, scorerSupplier.cost());
-
-      if (c.isScoring() && scoreMode.needsScores()) {
-        requiredScoringSupplier.add(scorerSupplier);
-      } else {
-        requiredNoScoringSupplier.add(scorerSupplier);
-      }
-    }
-
-    List requiredNoScoring = new ArrayList<>();
-    for (ScorerSupplier ss : requiredNoScoringSupplier) {
-      requiredNoScoring.add(ss.get(leadCost));
-    }
-    List requiredScoring = new ArrayList<>();
-    for (ScorerSupplier ss : requiredScoringSupplier) {
-      if (requiredScoringSupplier.size() == 1) {
-        ss.setTopLevelScoringClause();
-      }
-      requiredScoring.add(ss.get(leadCost));
-    }
-    if (scoreMode == ScoreMode.TOP_SCORES
-        && requiredNoScoringSupplier.isEmpty()
-        && requiredScoring.size() > 1
-        // Only specialize top-level conjunctions for clauses that don't have a two-phase iterator.
-        && requiredScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)) {
-      return new BlockMaxConjunctionBulkScorer(context.reader().maxDoc(), requiredScoring);
-    }
-    if (scoreMode != ScoreMode.TOP_SCORES
-        && requiredScoring.size() + requiredNoScoring.size() >= 2
-        && requiredScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)
-        && requiredNoScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)) {
-      return new ConjunctionBulkScorer(requiredScoring, requiredNoScoring);
-    }
-    if (scoreMode == ScoreMode.TOP_SCORES && requiredScoring.size() > 1) {
-      requiredScoring =
-          Collections.singletonList(new BlockMaxConjunctionScorer(this, requiredScoring));
-    }
-    Scorer conjunctionScorer;
-    if (requiredNoScoring.size() + requiredScoring.size() == 1) {
-      if (requiredScoring.size() == 1) {
-        conjunctionScorer = requiredScoring.get(0);
-      } else {
-        conjunctionScorer = requiredNoScoring.get(0);
-        if (scoreMode.needsScores()) {
-          Scorer inner = conjunctionScorer;
-          conjunctionScorer =
-              new FilterScorer(inner) {
-                @Override
-                public float score() throws IOException {
-                  return 0f;
-                }
-
-                @Override
-                public float getMaxScore(int upTo) throws IOException {
-                  return 0f;
-                }
-              };
-        }
-      }
-    } else {
-      List required = new ArrayList<>();
-      required.addAll(requiredScoring);
-      required.addAll(requiredNoScoring);
-      conjunctionScorer = new ConjunctionScorer(this, required, requiredScoring);
-    }
-    return new DefaultBulkScorer(conjunctionScorer);
-  }
-
-  /**
-   * Try to build a boolean scorer for this weight. Returns null if {@link BooleanScorer} cannot be
-   * used.
-   */
-  BulkScorer booleanScorer(LeafReaderContext context) throws IOException {
-    final int numOptionalClauses = query.getClauses(Occur.SHOULD).size();
-    final int numRequiredClauses =
-        query.getClauses(Occur.MUST).size() + query.getClauses(Occur.FILTER).size();
-
-    BulkScorer positiveScorer;
-    if (numRequiredClauses == 0) {
-      positiveScorer = optionalBulkScorer(context);
-      if (positiveScorer == null) {
-        return null;
-      }
-
-      // TODO: what is the right heuristic here?
-      final long costThreshold;
-      if (query.getMinimumNumberShouldMatch() <= 1) {
-        // when all clauses are optional, use BooleanScorer aggressively
-        // TODO: is there actually a threshold under which we should rather
-        // use the regular scorer?
-        costThreshold = -1;
-      } else {
-        // when a minimum number of clauses should match, BooleanScorer is
-        // going to score all windows that have at least minNrShouldMatch
-        // matches in the window. But there is no way to know if there is
-        // an intersection (all clauses might match a different doc ID and
-        // there will be no matches in the end) so we should only use
-        // BooleanScorer if matches are very dense
-        costThreshold = context.reader().maxDoc() / 3;
-      }
-
-      if (positiveScorer.cost() < costThreshold) {
-        return null;
-      }
-
-    } else if (numRequiredClauses > 0
-        && numOptionalClauses == 0
-        && query.getMinimumNumberShouldMatch() == 0) {
-      positiveScorer = requiredBulkScorer(context);
-    } else {
-      // TODO: there are some cases where BooleanScorer
-      // would handle conjunctions faster than
-      // BooleanScorer2...
-      return null;
-    }
-
-    if (positiveScorer == null) {
-      return null;
-    }
-
-    List prohibited = new ArrayList<>();
-    for (WeightedBooleanClause wc : weightedClauses) {
-      Weight w = wc.weight;
-      BooleanClause c = wc.clause;
-      if (c.isProhibited()) {
-        Scorer scorer = w.scorer(context);
-        if (scorer != null) {
-          prohibited.add(scorer);
-        }
-      }
-    }
-
-    if (prohibited.isEmpty()) {
-      return positiveScorer;
-    } else {
-      Scorer prohibitedScorer =
-          prohibited.size() == 1
-              ? prohibited.get(0)
-              : new DisjunctionSumScorer(this, prohibited, ScoreMode.COMPLETE_NO_SCORES);
-      if (prohibitedScorer.twoPhaseIterator() != null) {
-        // ReqExclBulkScorer can't deal efficiently with two-phased prohibited clauses
-        return null;
-      }
-      return new ReqExclBulkScorer(positiveScorer, prohibitedScorer.iterator());
-    }
-  }
-
   @Override
   public int count(LeafReaderContext context) throws IOException {
     final int numDocs = context.reader().numDocs();
diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java
index b8e249c08c24..91043bb8f680 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java
@@ -22,26 +22,40 @@
 final class ReqExclBulkScorer extends BulkScorer {
 
   private final BulkScorer req;
-  private final DocIdSetIterator excl;
+  private final DocIdSetIterator exclApproximation;
+  private final TwoPhaseIterator exclTwoPhase;
+
+  ReqExclBulkScorer(BulkScorer req, Scorer excl) {
+    this.req = req;
+    this.exclTwoPhase = excl.twoPhaseIterator();
+    if (exclTwoPhase != null) {
+      this.exclApproximation = exclTwoPhase.approximation();
+    } else {
+      this.exclApproximation = excl.iterator();
+    }
+  }
 
   ReqExclBulkScorer(BulkScorer req, DocIdSetIterator excl) {
     this.req = req;
-    this.excl = excl;
+    this.exclTwoPhase = null;
+    this.exclApproximation = excl;
   }
 
   @Override
   public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException {
     int upTo = min;
-    int exclDoc = excl.docID();
+    int exclDoc = exclApproximation.docID();
 
     while (upTo < max) {
       if (exclDoc < upTo) {
-        exclDoc = excl.advance(upTo);
+        exclDoc = exclApproximation.advance(upTo);
       }
       if (exclDoc == upTo) {
-        // upTo is excluded so we can consider that we scored up to upTo+1
-        upTo += 1;
-        exclDoc = excl.nextDoc();
+        if (exclTwoPhase == null || exclTwoPhase.matches()) {
+          // upTo is excluded so we can consider that we scored up to upTo+1
+          upTo += 1;
+        }
+        exclDoc = exclApproximation.nextDoc();
       } else {
         upTo = req.score(collector, acceptDocs, upTo, Math.min(exclDoc, max));
       }
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 d00279fc1a26..73ef2abef59d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
@@ -185,7 +185,8 @@ public void testOptimizeTopLevelClauseOrNull() throws IOException {
 
     // no scores -> term scorer
     Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1);
-    BulkScorer scorer = ((BooleanWeight) weight).booleanScorer(ctx);
+    ScorerSupplier ss = weight.scorerSupplier(ctx);
+    BulkScorer scorer = ((Boolean2ScorerSupplier) ss).booleanScorer();
     assertTrue(scorer instanceof DefaultBulkScorer); // term scorer
 
     // scores -> term scorer too
@@ -195,7 +196,8 @@ public void testOptimizeTopLevelClauseOrNull() throws IOException {
             .add(new TermQuery(new Term("foo", "baz")), Occur.SHOULD) // missing term
             .build();
     weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
-    scorer = ((BooleanWeight) weight).booleanScorer(ctx);
+    ss = weight.scorerSupplier(ctx);
+    scorer = ((Boolean2ScorerSupplier) ss).booleanScorer();
     assertTrue(scorer instanceof DefaultBulkScorer); // term scorer
 
     w.close();
@@ -225,7 +227,8 @@ public void testOptimizeProhibitedClauses() throws IOException {
             .add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
             .build();
     Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
-    BulkScorer scorer = ((BooleanWeight) weight).booleanScorer(ctx);
+    ScorerSupplier ss = weight.scorerSupplier(ctx);
+    BulkScorer scorer = ((Boolean2ScorerSupplier) ss).booleanScorer();
     assertTrue(scorer instanceof ReqExclBulkScorer);
 
     query =
@@ -235,7 +238,8 @@ public void testOptimizeProhibitedClauses() throws IOException {
             .add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
             .build();
     weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
-    scorer = ((BooleanWeight) weight).booleanScorer(ctx);
+    ss = weight.scorerSupplier(ctx);
+    scorer = ((Boolean2ScorerSupplier) ss).booleanScorer();
     assertTrue(scorer instanceof ReqExclBulkScorer);
 
     query =
@@ -244,7 +248,8 @@ public void testOptimizeProhibitedClauses() throws IOException {
             .add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
             .build();
     weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
-    scorer = ((BooleanWeight) weight).booleanScorer(ctx);
+    ss = weight.scorerSupplier(ctx);
+    scorer = ((Boolean2ScorerSupplier) ss).booleanScorer();
     assertTrue(scorer instanceof ReqExclBulkScorer);
 
     query =
@@ -253,7 +258,8 @@ public void testOptimizeProhibitedClauses() throws IOException {
             .add(new TermQuery(new Term("foo", "bar")), Occur.MUST_NOT)
             .build();
     weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1);
-    scorer = ((BooleanWeight) weight).booleanScorer(ctx);
+    ss = weight.scorerSupplier(ctx);
+    scorer = ((Boolean2ScorerSupplier) ss).booleanScorer();
     assertTrue(scorer instanceof ReqExclBulkScorer);
 
     w.close();
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java
index 4efda423c2ec..d67e209809c6 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.search;
 
 import java.io.IOException;
+
 import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.Bits;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java
index 075db88b4795..f0c5be94c29a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java
@@ -65,9 +65,16 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti
     return new ScorerSupplier() {
       private boolean getCalled = false;
       private boolean topLevelScoringClause = false;
+      Exception prev;
 
       @Override
       public Scorer get(long leadCost) throws IOException {
+        if (prev == null) {
+          prev = new RuntimeException();
+        } else {
+          prev.printStackTrace();
+          new RuntimeException().printStackTrace();
+        }
         assert getCalled == false;
         getCalled = true;
         assert leadCost >= 0 : leadCost;
@@ -86,6 +93,12 @@ public BulkScorer getBulkScorer() throws IOException {
         // We explicitly test both the delegate's bulk scorer, and also the normal scorer.
         // This ensures that normal scorers are sometimes tested with an asserting wrapper.
         if (usually(random)) {
+          if (prev == null) {
+            prev = new RuntimeException();
+          } else {
+            prev.printStackTrace();
+            new RuntimeException().printStackTrace();
+          }
           getCalled = true;
           inScorer = inScorerSupplier.getBulkScorer();
         } else {

From 8747774595d552f948e2d92a3294cc63b6d3a0aa Mon Sep 17 00:00:00 2001
From: Adrien Grand 
Date: Fri, 24 May 2024 09:01:02 +0200
Subject: [PATCH 04/10] iter

---
 ...actMultiTermQueryConstantScoreWrapper.java | 26 +++++++
 ...pplier.java => BooleanScorerSupplier.java} | 16 ++---
 .../apache/lucene/search/BooleanWeight.java   |  3 +-
 .../lucene/search/ConstantScoreQuery.java     |  6 +-
 .../lucene/search/IndexOrDocValuesQuery.java  |  4 +-
 .../apache/lucene/search/IndexSearcher.java   |  2 +-
 .../apache/lucene/search/LRUQueryCache.java   |  2 +-
 .../lucene/search/MatchAllDocsQuery.java      |  4 +-
 .../apache/lucene/search/ScorerSupplier.java  |  7 +-
 .../java/org/apache/lucene/search/Weight.java |  4 +-
 .../search/TestBoolean2ScorerSupplier.java    | 70 +++++++++----------
 .../lucene/search/TestBooleanScorer.java      | 14 ++--
 .../lucene/search/TestLRUQueryCache.java      |  2 +-
 .../lucene/search/TestReqExclBulkScorer.java  |  1 -
 .../lucene/facet/DrillSidewaysQuery.java      |  2 +-
 .../sandbox/search/QueryProfilerWeight.java   |  4 +-
 .../suggest/document/CompletionWeight.java    |  2 +-
 .../lucene/tests/search/AssertingWeight.java  |  6 +-
 .../search/DisablingBulkScorerQuery.java      |  2 +-
 19 files changed, 103 insertions(+), 74 deletions(-)
 rename lucene/core/src/java/org/apache/lucene/search/{Boolean2ScorerSupplier.java => BooleanScorerSupplier.java} (97%)

diff --git a/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java
index e1e121cad52d..1a9b18f0f39e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/search/AbstractMultiTermQueryConstantScoreWrapper.java
@@ -256,6 +256,32 @@ public Scorer get(long leadCost) throws IOException {
               () -> new ConstantScoreScorer(weight, score(), scoreMode, DocIdSetIterator.empty()));
         }
 
+        @Override
+        public BulkScorer bulkScorer() throws IOException {
+          WeightOrDocIdSetIterator weightOrIterator = rewrite(context, terms);
+          final BulkScorer bulkScorer;
+          if (weightOrIterator == null) {
+            bulkScorer = null;
+          } else if (weightOrIterator.weight != null) {
+            bulkScorer = weightOrIterator.weight.bulkScorer(context);
+          } else {
+            bulkScorer =
+                new DefaultBulkScorer(
+                    new ConstantScoreScorer(weight, score(), scoreMode, weightOrIterator.iterator));
+          }
+
+          // It's against the API contract to return a null scorer from a non-null ScoreSupplier.
+          // So if our ScoreSupplier was non-null (i.e., thought there might be hits) but we now
+          // find that there are actually no hits, we need to return an empty BulkScorer as opposed
+          // to null:
+          return Objects.requireNonNullElseGet(
+              bulkScorer,
+              () ->
+                  new DefaultBulkScorer(
+                      new ConstantScoreScorer(
+                          weight, score(), scoreMode, DocIdSetIterator.empty())));
+        }
+
         @Override
         public long cost() {
           return cost;
diff --git a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java
similarity index 97%
rename from lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
rename to lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java
index da2ca72dea10..3f8ccb81e664 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java
@@ -30,7 +30,7 @@
 import org.apache.lucene.search.Weight.DefaultBulkScorer;
 import org.apache.lucene.util.Bits;
 
-final class Boolean2ScorerSupplier extends ScorerSupplier {
+final class BooleanScorerSupplier extends ScorerSupplier {
 
   private final Weight weight;
   private final Map> subs;
@@ -40,7 +40,7 @@ final class Boolean2ScorerSupplier extends ScorerSupplier {
   private long cost = -1;
   private boolean topLevelScoringClause;
 
-  Boolean2ScorerSupplier(
+  BooleanScorerSupplier(
       Weight weight,
       Map> subs,
       ScoreMode scoreMode,
@@ -173,14 +173,14 @@ private Scorer getInternal(long leadCost) throws IOException {
   }
 
   @Override
-  public BulkScorer getBulkScorer() throws IOException {
+  public BulkScorer bulkScorer() throws IOException {
     final BulkScorer bulkScorer = booleanScorer();
     if (bulkScorer != null) {
       // bulk scoring is applicable, use it
       return bulkScorer;
     } else {
       // use a Scorer-based impl (BS2)
-      return super.getBulkScorer();
+      return super.bulkScorer();
     }
   }
 
@@ -280,7 +280,7 @@ BulkScorer optionalBulkScorer() throws IOException {
     if (subs.get(Occur.SHOULD).size() == 0) {
       return null;
     } else if (subs.get(Occur.SHOULD).size() == 1 && minShouldMatch <= 1) {
-      return subs.get(Occur.SHOULD).iterator().next().getBulkScorer();
+      return subs.get(Occur.SHOULD).iterator().next().bulkScorer();
     }
 
     if (scoreMode == ScoreMode.TOP_SCORES && minShouldMatch <= 1) {
@@ -294,7 +294,7 @@ BulkScorer optionalBulkScorer() throws IOException {
 
     List optional = new ArrayList();
     for (ScorerSupplier ss : subs.get(Occur.SHOULD)) {
-      optional.add(ss.getBulkScorer());
+      optional.add(ss.bulkScorer());
     }
 
     return new BooleanScorer(optional, Math.max(1, minShouldMatch), scoreMode.needsScores());
@@ -308,9 +308,9 @@ private BulkScorer requiredBulkScorer() throws IOException {
     } else if (subs.get(Occur.MUST).size() + subs.get(Occur.FILTER).size() == 1) {
       BulkScorer scorer;
       if (subs.get(Occur.MUST).isEmpty() == false) {
-        scorer = subs.get(Occur.MUST).iterator().next().getBulkScorer();
+        scorer = subs.get(Occur.MUST).iterator().next().bulkScorer();
       } else {
-        scorer = subs.get(Occur.FILTER).iterator().next().getBulkScorer();
+        scorer = subs.get(Occur.FILTER).iterator().next().bulkScorer();
         if (scoreMode.needsScores()) {
           scorer = disableScoring(scorer);
         }
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 617a59d3e7e3..f73f504f3cfa 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanWeight.java
@@ -22,7 +22,6 @@
 import java.util.EnumMap;
 import java.util.List;
 import java.util.Map;
-
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.similarities.Similarity;
@@ -390,7 +389,7 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti
       scorers.get(Occur.SHOULD).clear();
     }
 
-    return new Boolean2ScorerSupplier(
+    return new BooleanScorerSupplier(
         this, scorers, scoreMode, minShouldMatch, context.reader().maxDoc());
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
index ec1741a4b1a1..80c94a30079d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
@@ -157,11 +157,11 @@ public Scorer get(long leadCost) throws IOException {
             }
 
             @Override
-            public BulkScorer getBulkScorer() throws IOException {
+            public BulkScorer bulkScorer() throws IOException {
               if (scoreMode.isExhaustive() == false) {
-                return super.getBulkScorer();
+                return super.bulkScorer();
               }
-              final BulkScorer innerScorer = innerScorerSupplier.getBulkScorer();
+              final BulkScorer innerScorer = innerScorerSupplier.bulkScorer();
               if (innerScorer == null) {
                 return null;
               }
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
index cf652c4da824..24d084d92f5d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexOrDocValuesQuery.java
@@ -180,10 +180,10 @@ public Scorer get(long leadCost) throws IOException {
           }
 
           @Override
-          public BulkScorer getBulkScorer() throws IOException {
+          public BulkScorer bulkScorer() throws IOException {
             // Bulk scorers need to consume the entire set of docs, so using an
             // index structure should perform better
-            return indexScorerSupplier.getBulkScorer();
+            return indexScorerSupplier.bulkScorer();
           }
 
           @Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
index a8104f8f5a99..8c0da9539ebe 100644
--- a/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
+++ b/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
@@ -707,7 +707,7 @@ protected void search(List leaves, Weight weight, Collector c
       ScorerSupplier scorerSupplier = weight.scorerSupplier(ctx);
       if (scorerSupplier != null) {
         scorerSupplier.setTopLevelScoringClause();
-        BulkScorer scorer = scorerSupplier.getBulkScorer();
+        BulkScorer scorer = scorerSupplier.bulkScorer();
         if (queryTimeout != null) {
           scorer = new TimeLimitingBulkScorer(scorer, queryTimeout);
         }
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index d59860126ef6..1bbafc1908de 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -770,7 +770,7 @@ public Scorer get(long leadCost) throws IOException {
                 return supplier.get(leadCost);
               }
 
-              CacheAndCount cached = cacheImpl(supplier.getBulkScorer(), context.reader().maxDoc());
+              CacheAndCount cached = cacheImpl(supplier.bulkScorer(), context.reader().maxDoc());
               putIfAbsent(in.getQuery(), cached, cacheHelper);
               DocIdSetIterator disi = cached.iterator();
               if (disi == null) {
diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
index 9301657e291b..426fc7c2ff52 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
@@ -43,9 +43,9 @@ public Scorer get(long leadCost) throws IOException {
           }
 
           @Override
-          public BulkScorer getBulkScorer() throws IOException {
+          public BulkScorer bulkScorer() throws IOException {
             if (scoreMode.isExhaustive() == false) {
-              return super.getBulkScorer();
+              return super.bulkScorer();
             }
             final float score = score();
             final int maxDoc = context.reader().maxDoc();
diff --git a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java
index 67a21e4aa3cf..edfaad873fe0 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ScorerSupplier.java
@@ -36,7 +36,12 @@ public abstract class ScorerSupplier {
    */
   public abstract Scorer get(long leadCost) throws IOException;
 
-  public BulkScorer getBulkScorer() throws IOException {
+  /**
+   * Optional method: Get a scorer that is optimized for bulk-scoring. The default implementation
+   * iterates matches from the {@link Scorer} but some queries can have more efficient approaches
+   * for matching all hits.
+   */
+  public BulkScorer bulkScorer() throws IOException {
     return new DefaultBulkScorer(get(Long.MAX_VALUE));
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/search/Weight.java b/lucene/core/src/java/org/apache/lucene/search/Weight.java
index ce8e560cf8e9..5c0e1f45eb41 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Weight.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java
@@ -159,7 +159,7 @@ public final Scorer scorer(LeafReaderContext context) throws IOException {
    * }
    *
    * scorerSupplier.setTopLevelScoringClause();
-   * return new DefaultBulkScorer(scorerSupplier.get(Long.MAX_VALUE));
+   * return scorerSupplier.bulkScorer();
    * 
*/ public final BulkScorer bulkScorer(LeafReaderContext context) throws IOException { @@ -170,7 +170,7 @@ public final BulkScorer bulkScorer(LeafReaderContext context) throws IOException } scorerSupplier.setTopLevelScoringClause(); - return scorerSupplier.getBulkScorer(); + return scorerSupplier.bulkScorer(); } /** diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java index 30d7c23d25ef..58cd8a652bd2 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBoolean2ScorerSupplier.java @@ -137,7 +137,7 @@ public void testConjunctionCost() { .add(new FakeScorerSupplier(42)); assertEquals( 42, - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .cost()); @@ -145,7 +145,7 @@ public void testConjunctionCost() { .add(new FakeScorerSupplier(12)); assertEquals( 12, - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .cost()); @@ -153,7 +153,7 @@ public void testConjunctionCost() { .add(new FakeScorerSupplier(20)); assertEquals( 12, - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .cost()); } @@ -166,21 +166,21 @@ public void testDisjunctionCost() throws IOException { subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42)); ScorerSupplier s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100); assertEquals(42, s.cost()); assertEquals(42, s.get(random().nextInt(100)).iterator().cost()); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12)); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100); assertEquals(42 + 12, s.cost()); assertEquals(42 + 12, s.get(random().nextInt(100)).iterator().cost()); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20)); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100); assertEquals(42 + 12 + 20, s.cost()); assertEquals(42 + 12 + 20, s.get(random().nextInt(100)).iterator().cost()); @@ -195,36 +195,36 @@ public void testDisjunctionWithMinShouldMatchCost() throws IOException { subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12)); ScorerSupplier s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1, 100); assertEquals(42 + 12, s.cost()); assertEquals(42 + 12, s.get(random().nextInt(100)).iterator().cost()); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20)); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1, 100); assertEquals(42 + 12 + 20, s.cost()); assertEquals(42 + 12 + 20, s.get(random().nextInt(100)).iterator().cost()); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100); assertEquals(12 + 20, s.cost()); assertEquals(12 + 20, s.get(random().nextInt(100)).iterator().cost()); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30)); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1, 100); assertEquals(42 + 12 + 20 + 30, s.cost()); assertEquals(42 + 12 + 20 + 30, s.get(random().nextInt(100)).iterator().cost()); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100); assertEquals(12 + 20 + 30, s.cost()); assertEquals(12 + 20 + 30, s.get(random().nextInt(100)).iterator().cost()); s = - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3, 100); assertEquals(12 + 20, s.cost()); assertEquals(12 + 20, s.get(random().nextInt(100)).iterator().cost()); @@ -260,8 +260,8 @@ public void testDuelCost() throws Exception { continue; } int minShouldMatch = numShoulds == 0 ? 0 : TestUtil.nextInt(random(), 0, numShoulds - 1); - Boolean2ScorerSupplier supplier = - new Boolean2ScorerSupplier(new FakeWeight(), subs, scoreMode, minShouldMatch, 100); + BooleanScorerSupplier supplier = + new BooleanScorerSupplier(new FakeWeight(), subs, scoreMode, minShouldMatch, 100); long cost1 = supplier.cost(); long cost2 = supplier.get(Long.MAX_VALUE).iterator().cost(); assertEquals("clauses=" + subs + ", minShouldMatch=" + minShouldMatch, cost1, cost2); @@ -287,7 +287,7 @@ public void testConjunctionLeadCost() throws IOException { .add(new FakeScorerSupplier(42, 12)); subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))) .add(new FakeScorerSupplier(12, 12)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(Long.MAX_VALUE); // triggers assertions as a side-effect @@ -301,7 +301,7 @@ public void testConjunctionLeadCost() throws IOException { .add(new FakeScorerSupplier(42, 7)); subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST))) .add(new FakeScorerSupplier(12, 7)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(7); // triggers assertions as a side-effect } @@ -313,14 +313,14 @@ public void testDisjunctionLeadCost() throws IOException { } subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 54)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 54)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(100); // triggers assertions as a side-effect subs.get(Occur.SHOULD).clear(); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 20)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 20)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(20); // triggers assertions as a side-effect } @@ -336,7 +336,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException { subs.get(Occur.SHOULD).add(new FakeScorerSupplier(50, 42)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 42)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 42)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100) .get(100); // triggers assertions as a side-effect @@ -349,7 +349,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException { subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42, 20)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 20)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 20)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100) .get(20); // triggers assertions as a side-effect @@ -362,7 +362,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException { subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 62)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 62)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, 62)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100) .get(100); // triggers assertions as a side-effect @@ -375,7 +375,7 @@ public void testDisjunctionWithMinShouldMatchLeadCost() throws IOException { subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12, 32)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 32)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(20, 32)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3, 100) .get(100); // triggers assertions as a side-effect } @@ -389,7 +389,7 @@ public void testProhibitedLeadCost() throws IOException { // The MUST_NOT clause is called with the same lead cost as the MUST clause subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42)); subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(30, 42)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(100); // triggers assertions as a side-effect @@ -397,7 +397,7 @@ public void testProhibitedLeadCost() throws IOException { subs.get(Occur.MUST_NOT).clear(); subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42)); subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(80, 42)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(100); // triggers assertions as a side-effect @@ -405,7 +405,7 @@ public void testProhibitedLeadCost() throws IOException { subs.get(Occur.MUST_NOT).clear(); subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 20)); subs.get(Occur.MUST_NOT).add(new FakeScorerSupplier(30, 20)); - new Boolean2ScorerSupplier( + new BooleanScorerSupplier( new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100) .get(20); // triggers assertions as a side-effect } @@ -419,21 +419,21 @@ public void testMixedLeadCost() throws IOException { // The SHOULD clause is always called with the same lead cost as the MUST clause subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(30, 42)); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100) .get(100); // triggers assertions as a side-effect subs.get(Occur.MUST).clear(); subs.get(Occur.SHOULD).clear(); subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 42)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(80, 42)); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100) .get(100); // triggers assertions as a side-effect subs.get(Occur.MUST).clear(); subs.get(Occur.SHOULD).clear(); subs.get(Occur.MUST).add(new FakeScorerSupplier(42, 20)); subs.get(Occur.SHOULD).add(new FakeScorerSupplier(80, 20)); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100) .get(20); // triggers assertions as a side-effect } @@ -448,7 +448,7 @@ public void testDisjunctionTopLevelScoringClause() throws Exception { FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10); subs.get(Occur.SHOULD).add(clause2); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) .setTopLevelScoringClause(); assertFalse(clause1.topLevelScoringClause); assertFalse(clause2.topLevelScoringClause); @@ -465,7 +465,7 @@ public void testConjunctionTopLevelScoringClause() throws Exception { FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10); subs.get(Occur.MUST).add(clause2); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) .setTopLevelScoringClause(); assertFalse(clause1.topLevelScoringClause); assertFalse(clause2.topLevelScoringClause); @@ -482,7 +482,7 @@ public void testFilterTopLevelScoringClause() throws Exception { FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10); subs.get(Occur.FILTER).add(clause2); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) .setTopLevelScoringClause(); assertFalse(clause1.topLevelScoringClause); assertFalse(clause2.topLevelScoringClause); @@ -499,7 +499,7 @@ public void testSingleMustScoringClause() throws Exception { FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10); subs.get(Occur.FILTER).add(clause2); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) .setTopLevelScoringClause(); assertTrue(clause1.topLevelScoringClause); assertFalse(clause2.topLevelScoringClause); @@ -516,7 +516,7 @@ public void testSingleShouldScoringClause() throws Exception { FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10); subs.get(Occur.MUST_NOT).add(clause2); - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100) .setTopLevelScoringClause(); assertTrue(clause1.topLevelScoringClause); assertFalse(clause2.topLevelScoringClause); @@ -534,7 +534,7 @@ public void testMaxScoreNonTopLevelScoringClause() throws Exception { subs.get(Occur.MUST).add(clause2); Scorer scorer = - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10); + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10); assertEquals(2.0, scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0.0); subs = new EnumMap<>(Occur.class); @@ -546,7 +546,7 @@ public void testMaxScoreNonTopLevelScoringClause() throws Exception { subs.get(Occur.SHOULD).add(clause2); scorer = - new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10); + new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10); assertEquals(2.0, scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0.0); } } 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 73ef2abef59d..fec70cb3dcca 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java @@ -96,7 +96,7 @@ public long cost() { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { return new BulkScorer() { @Override public int score(LeafCollector collector, Bits acceptDocs, int min, int max) @@ -186,7 +186,7 @@ public void testOptimizeTopLevelClauseOrNull() throws IOException { // no scores -> term scorer Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1); ScorerSupplier ss = weight.scorerSupplier(ctx); - BulkScorer scorer = ((Boolean2ScorerSupplier) ss).booleanScorer(); + BulkScorer scorer = ((BooleanScorerSupplier) ss).booleanScorer(); assertTrue(scorer instanceof DefaultBulkScorer); // term scorer // scores -> term scorer too @@ -197,7 +197,7 @@ public void testOptimizeTopLevelClauseOrNull() throws IOException { .build(); weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1); ss = weight.scorerSupplier(ctx); - scorer = ((Boolean2ScorerSupplier) ss).booleanScorer(); + scorer = ((BooleanScorerSupplier) ss).booleanScorer(); assertTrue(scorer instanceof DefaultBulkScorer); // term scorer w.close(); @@ -228,7 +228,7 @@ public void testOptimizeProhibitedClauses() throws IOException { .build(); Weight weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1); ScorerSupplier ss = weight.scorerSupplier(ctx); - BulkScorer scorer = ((Boolean2ScorerSupplier) ss).booleanScorer(); + BulkScorer scorer = ((BooleanScorerSupplier) ss).booleanScorer(); assertTrue(scorer instanceof ReqExclBulkScorer); query = @@ -239,7 +239,7 @@ public void testOptimizeProhibitedClauses() throws IOException { .build(); weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1); ss = weight.scorerSupplier(ctx); - scorer = ((Boolean2ScorerSupplier) ss).booleanScorer(); + scorer = ((BooleanScorerSupplier) ss).booleanScorer(); assertTrue(scorer instanceof ReqExclBulkScorer); query = @@ -249,7 +249,7 @@ public void testOptimizeProhibitedClauses() throws IOException { .build(); weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1); ss = weight.scorerSupplier(ctx); - scorer = ((Boolean2ScorerSupplier) ss).booleanScorer(); + scorer = ((BooleanScorerSupplier) ss).booleanScorer(); assertTrue(scorer instanceof ReqExclBulkScorer); query = @@ -259,7 +259,7 @@ public void testOptimizeProhibitedClauses() throws IOException { .build(); weight = searcher.createWeight(searcher.rewrite(query), ScoreMode.COMPLETE, 1); ss = weight.scorerSupplier(ctx); - scorer = ((Boolean2ScorerSupplier) ss).booleanScorer(); + scorer = ((BooleanScorerSupplier) ss).booleanScorer(); assertTrue(scorer instanceof ReqExclBulkScorer); w.close(); diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java index 7c5b0ad61cc1..4b65e52dd44d 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java @@ -1369,7 +1369,7 @@ public Scorer get(long leadCost) throws IOException { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { bulkScorerCalled.set(true); return in.bulkScorer(context); } diff --git a/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java index d67e209809c6..4efda423c2ec 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java @@ -17,7 +17,6 @@ package org.apache.lucene.search; import java.io.IOException; - import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.Bits; diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java index 3b8939bb88f9..ef41ae380710 100644 --- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java +++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java @@ -216,7 +216,7 @@ public Scorer get(long leadCost) throws IOException { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { return new DrillSidewaysScorer( context, baseScorerSupplier.get(Long.MAX_VALUE), diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java index 7b0d26b2e10f..77113cd181a9 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java +++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java @@ -77,7 +77,7 @@ public Scorer get(long loadCost) throws IOException { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { // We use the default bulk scorer instead of the specialized one. The reason // is that BulkScorers do everything at once: finding matches, // scoring them and calling the collector, so they make it impossible to @@ -85,7 +85,7 @@ public BulkScorer getBulkScorer() throws IOException { // The default bulk scorer will pull a scorer and iterate over matches, // this might be a significantly different execution path for some queries // like disjunctions, but in general this is what is done anyway - return super.getBulkScorer(); + return super.bulkScorer(); } @Override diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java index 8b9a89352743..b4572c0c7295 100644 --- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java +++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/document/CompletionWeight.java @@ -100,7 +100,7 @@ public Scorer get(long leadCost) throws IOException { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { final LeafReader reader = context.reader(); final Terms terms; final NRTSuggester suggester; diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java index f0c5be94c29a..ec665ccaff34 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java @@ -86,7 +86,7 @@ public Scorer get(long leadCost) throws IOException { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { assert getCalled == false; BulkScorer inScorer; @@ -100,10 +100,10 @@ public BulkScorer getBulkScorer() throws IOException { new RuntimeException().printStackTrace(); } getCalled = true; - inScorer = inScorerSupplier.getBulkScorer(); + inScorer = inScorerSupplier.bulkScorer(); } else { // Don't set getCalled = true, since this calls #get under the hood - inScorer = super.getBulkScorer(); + inScorer = super.bulkScorer(); } return AssertingBulkScorer.wrap( diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java index 23dafb2707ff..b042e0bd9c08 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/DisablingBulkScorerQuery.java @@ -68,7 +68,7 @@ public long cost() { } @Override - public BulkScorer getBulkScorer() throws IOException { + public BulkScorer bulkScorer() throws IOException { return new DefaultBulkScorer(get(Long.MAX_VALUE)); } }; From 770bcf001825c204fe9545ec16e93b920d459056 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 24 May 2024 09:57:04 +0200 Subject: [PATCH 05/10] iter --- .../java/org/apache/lucene/search/BooleanScorerSupplier.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java index 3f8ccb81e664..4f957aad4be7 100644 --- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java +++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorerSupplier.java @@ -336,7 +336,7 @@ private BulkScorer requiredBulkScorer() throws IOException { requiredScoring.add(ss.get(leadCost)); } if (scoreMode == ScoreMode.TOP_SCORES - && subs.get(Occur.FILTER).isEmpty() + && requiredNoScoring.isEmpty() && requiredScoring.size() > 1 // Only specialize top-level conjunctions for clauses that don't have a two-phase iterator. && requiredScoring.stream().map(Scorer::twoPhaseIterator).allMatch(Objects::isNull)) { From f967447b935551656912c5241eb853018c50d02e Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 24 May 2024 14:00:17 +0200 Subject: [PATCH 06/10] iter --- .../lucene/tests/search/AssertingWeight.java | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java index ec665ccaff34..c25296b9a53f 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/AssertingWeight.java @@ -65,16 +65,9 @@ public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOExcepti return new ScorerSupplier() { private boolean getCalled = false; private boolean topLevelScoringClause = false; - Exception prev; @Override public Scorer get(long leadCost) throws IOException { - if (prev == null) { - prev = new RuntimeException(); - } else { - prev.printStackTrace(); - new RuntimeException().printStackTrace(); - } assert getCalled == false; getCalled = true; assert leadCost >= 0 : leadCost; @@ -93,17 +86,12 @@ public BulkScorer bulkScorer() throws IOException { // We explicitly test both the delegate's bulk scorer, and also the normal scorer. // This ensures that normal scorers are sometimes tested with an asserting wrapper. if (usually(random)) { - if (prev == null) { - prev = new RuntimeException(); - } else { - prev.printStackTrace(); - new RuntimeException().printStackTrace(); - } getCalled = true; inScorer = inScorerSupplier.bulkScorer(); } else { // Don't set getCalled = true, since this calls #get under the hood inScorer = super.bulkScorer(); + assert getCalled; } return AssertingBulkScorer.wrap( From 13337f3b8381634521ce5072fc05e292f5557918 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 24 May 2024 16:37:19 +0200 Subject: [PATCH 07/10] improve testing --- .../lucene/search/ReqExclBulkScorer.java | 6 ++++++ .../lucene/search/TestReqExclBulkScorer.java | 20 ++++++++++++++++--- .../search/RandomApproximationQuery.java | 4 ++-- 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java b/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java index 91043bb8f680..7c07ef0386db 100644 --- a/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/ReqExclBulkScorer.java @@ -41,6 +41,12 @@ final class ReqExclBulkScorer extends BulkScorer { this.exclApproximation = excl; } + ReqExclBulkScorer(BulkScorer req, TwoPhaseIterator excl) { + this.req = req; + this.exclTwoPhase = excl; + this.exclApproximation = excl.approximation(); + } + @Override public int score(LeafCollector collector, Bits acceptDocs, int min, int max) throws IOException { int upTo = min; diff --git a/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java index 4efda423c2ec..ccd298e25caf 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestReqExclBulkScorer.java @@ -17,6 +17,7 @@ package org.apache.lucene.search; import java.io.IOException; +import org.apache.lucene.tests.search.RandomApproximationQuery.RandomTwoPhaseView; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.Bits; @@ -28,11 +29,18 @@ public class TestReqExclBulkScorer extends LuceneTestCase { public void testRandom() throws IOException { final int iters = atLeast(10); for (int iter = 0; iter < iters; ++iter) { - doTestRandom(); + doTestRandom(false); } } - public void doTestRandom() throws IOException { + public void testRandomTwoPhase() throws IOException { + final int iters = atLeast(10); + for (int iter = 0; iter < iters; ++iter) { + doTestRandom(true); + } + } + + public void doTestRandom(boolean twoPhase) throws IOException { final int maxDoc = TestUtil.nextInt(random(), 1, 1000); DocIdSetBuilder reqBuilder = new DocIdSetBuilder(maxDoc); DocIdSetBuilder exclBuilder = new DocIdSetBuilder(maxDoc); @@ -76,7 +84,13 @@ public long cost() { } }; - ReqExclBulkScorer reqExcl = new ReqExclBulkScorer(reqBulkScorer, excl.iterator()); + ReqExclBulkScorer reqExcl; + if (twoPhase) { + reqExcl = + new ReqExclBulkScorer(reqBulkScorer, new RandomTwoPhaseView(random(), excl.iterator())); + } else { + reqExcl = new ReqExclBulkScorer(reqBulkScorer, excl.iterator()); + } final FixedBitSet actualMatches = new FixedBitSet(maxDoc); if (random().nextBoolean()) { reqExcl.score( diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java index a06228ed98e6..195eb9048515 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java @@ -149,13 +149,13 @@ public DocIdSetIterator iterator() { } } - private static class RandomTwoPhaseView extends TwoPhaseIterator { + public static class RandomTwoPhaseView extends TwoPhaseIterator { private final DocIdSetIterator disi; private int lastDoc = -1; private final float randomMatchCost; - RandomTwoPhaseView(Random random, DocIdSetIterator disi) { + public RandomTwoPhaseView(Random random, DocIdSetIterator disi) { super(new RandomApproximation(random, disi)); this.disi = disi; this.randomMatchCost = random.nextFloat() * 200; // between 0 and 200 From 459f83a5efa0bad6336caf9e47ab07e2eac3d4f5 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 24 May 2024 16:52:43 +0200 Subject: [PATCH 08/10] javadocs --- .../apache/lucene/tests/search/RandomApproximationQuery.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java index 195eb9048515..3267f70bd03c 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java @@ -149,12 +149,16 @@ public DocIdSetIterator iterator() { } } + /** + * A wrapper around a {@link DocIdSetIterator} that matches the same documents, but introduces false positives that need to be verified via {@link TwoPhaseIterator#matches()}. + */ public static class RandomTwoPhaseView extends TwoPhaseIterator { private final DocIdSetIterator disi; private int lastDoc = -1; private final float randomMatchCost; + /** Constructor. */ public RandomTwoPhaseView(Random random, DocIdSetIterator disi) { super(new RandomApproximation(random, disi)); this.disi = disi; From dee24f4790cdddaad083c7630cd1b2273f737834 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 24 May 2024 16:57:32 +0200 Subject: [PATCH 09/10] tidy --- .../apache/lucene/tests/search/RandomApproximationQuery.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java index 3267f70bd03c..73adcb5e95e2 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java +++ b/lucene/test-framework/src/java/org/apache/lucene/tests/search/RandomApproximationQuery.java @@ -150,7 +150,8 @@ public DocIdSetIterator iterator() { } /** - * A wrapper around a {@link DocIdSetIterator} that matches the same documents, but introduces false positives that need to be verified via {@link TwoPhaseIterator#matches()}. + * A wrapper around a {@link DocIdSetIterator} that matches the same documents, but introduces + * false positives that need to be verified via {@link TwoPhaseIterator#matches()}. */ public static class RandomTwoPhaseView extends TwoPhaseIterator { From d7c72db81d1aa0a65db10fd462f3b49b4d786666 Mon Sep 17 00:00:00 2001 From: Adrien Grand Date: Fri, 24 May 2024 17:18:48 +0200 Subject: [PATCH 10/10] CHANGES --- lucene/CHANGES.txt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index d5e8f12227cb..3e464cc40542 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -106,6 +106,10 @@ API Changes the directory about bytes that are about to be read. (Adrien Grand, Uwe Schindler) +* GITHUB#13408: Moved Weight#bulkScorer() to ScorerSupplier#bulkScorer() to better help parallelize + I/O for top-level disjunctions. Weight#bulkScorer() still exists for compatibility, but delegates + to ScorerSupplier#bulkScorer(). (Adrien Grand) + New Features ---------------------