Move bulkScorer() from Weight to ScorerSupplier (#13408)

This relates to #13359: we want to take advantage of the `Weight#scorerSupplier` call to start scheduling some I/O in the background in parallel across clauses. For this to work properly with top-level disjunctions, we need to move `#bulkScorer()` from `Weight` to `ScorerSupplier` as well, so that the disjunctive `BooleanQuery` first performs a call to `Weight#scorerSupplier()` on all inner clauses, and then `ScorerSupplier#bulkScorer` on all inner clauses.

`ScorerSupplier#get` and `ScorerSupplier#bulkScorer` only support being called once. This forced me to fix some inefficiencies in `bulkScorer()` implementations when we would pull scorers and then throw it away when realizing that the strategy we were planning on using was not optimal. This is why e.g. `ReqExclBulkScorer` now also supports prohibited clauses that produce a two-phase iterator.
This commit is contained in:
Adrien Grand 2024-05-27 09:56:26 +02:00 committed by GitHub
parent 6e2a8fc9b7
commit ddf538d43e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
27 changed files with 615 additions and 614 deletions

View File

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

View File

@ -211,26 +211,6 @@ abstract class AbstractMultiTermQueryConstantScoreWrapper<Q extends MultiTermQue
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);
@ -276,6 +256,32 @@ abstract class AbstractMultiTermQueryConstantScoreWrapper<Q extends MultiTermQue
() -> 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;

View File

@ -194,11 +194,7 @@ final class BooleanScorer extends BulkScorer {
private final DocIdStreamView docIdStreamView = new DocIdStreamView();
BooleanScorer(
BooleanWeight weight,
Collection<BulkScorer> scorers,
int minShouldMatch,
boolean needsScores) {
BooleanScorer(Collection<BulkScorer> scorers, int minShouldMatch, boolean needsScores) {
if (minShouldMatch < 1 || minShouldMatch > scorers.size()) {
throw new IllegalArgumentException(
"minShouldMatch should be within 1..num_scorers. Got " + minShouldMatch);

View File

@ -23,24 +23,29 @@ import java.util.Collection;
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.search.BooleanClause.Occur;
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<BooleanClause.Occur, Collection<ScorerSupplier>> subs;
private final ScoreMode scoreMode;
private final int minShouldMatch;
private final int maxDoc;
private long cost = -1;
private boolean topLevelScoringClause;
Boolean2ScorerSupplier(
BooleanScorerSupplier(
Weight weight,
Map<Occur, Collection<ScorerSupplier>> subs,
ScoreMode scoreMode,
int minShouldMatch) {
int minShouldMatch,
int maxDoc) {
if (minShouldMatch < 0) {
throw new IllegalArgumentException(
"minShouldMatch must be positive, but got: " + minShouldMatch);
@ -64,6 +69,7 @@ final class Boolean2ScorerSupplier extends ScorerSupplier {
this.subs = subs;
this.scoreMode = scoreMode;
this.minShouldMatch = minShouldMatch;
this.maxDoc = maxDoc;
}
private long computeCost() {
@ -166,6 +172,217 @@ final class Boolean2ScorerSupplier extends ScorerSupplier {
}
}
@Override
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.bulkScorer();
}
}
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) {
// TODO: what is the right heuristic here?
final long costThreshold;
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?
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 = maxDoc / 3;
}
if (cost() < costThreshold) {
return null;
}
positiveScorer = optionalBulkScorer();
} else if (numRequiredClauses > 0 && numOptionalClauses == 0 && minShouldMatch == 0) {
positiveScorer = requiredBulkScorer();
} else {
// TODO: there are some cases where BooleanScorer
// would handle conjunctions faster than
// BooleanScorer2...
return null;
}
if (positiveScorer == null) {
return null;
}
final long positiveScorerCost = positiveScorer.cost();
List<Scorer> prohibited = new ArrayList<>();
for (ScorerSupplier ss : subs.get(Occur.MUST_NOT)) {
prohibited.add(ss.get(positiveScorerCost));
}
if (prohibited.isEmpty()) {
return positiveScorer;
} else {
Scorer prohibitedScorer =
prohibited.size() == 1
? prohibited.get(0)
: new DisjunctionSumScorer(weight, prohibited, ScoreMode.COMPLETE_NO_SCORES);
return new ReqExclBulkScorer(positiveScorer, prohibitedScorer);
}
}
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().bulkScorer();
}
if (scoreMode == ScoreMode.TOP_SCORES && minShouldMatch <= 1) {
List<Scorer> optionalScorers = new ArrayList<>();
for (ScorerSupplier ss : subs.get(Occur.SHOULD)) {
optionalScorers.add(ss.get(Long.MAX_VALUE));
}
return new MaxScoreBulkScorer(maxDoc, optionalScorers);
}
List<BulkScorer> optional = new ArrayList<BulkScorer>();
for (ScorerSupplier ss : subs.get(Occur.SHOULD)) {
optional.add(ss.bulkScorer());
}
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().bulkScorer();
} else {
scorer = subs.get(Occur.FILTER).iterator().next().bulkScorer();
if (scoreMode.needsScores()) {
scorer = disableScoring(scorer);
}
}
return scorer;
}
long leadCost =
subs.get(Occur.MUST).stream().mapToLong(ScorerSupplier::cost).min().orElse(Long.MAX_VALUE);
leadCost =
subs.get(Occur.FILTER).stream().mapToLong(ScorerSupplier::cost).min().orElse(leadCost);
List<Scorer> requiredNoScoring = new ArrayList<>();
for (ScorerSupplier ss : subs.get(Occur.FILTER)) {
requiredNoScoring.add(ss.get(leadCost));
}
List<Scorer> requiredScoring = new ArrayList<>();
Collection<ScorerSupplier> 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
&& 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)) {
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<Scorer> 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.

View File

@ -19,15 +19,12 @@ package org.apache.lucene.search;
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 +153,6 @@ final class BooleanWeight extends Weight {
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
@ -248,203 +214,7 @@ final class BooleanWeight extends Weight {
}
return new BooleanScorer(
this, 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<WeightedBooleanClause> 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<ScorerSupplier> requiredNoScoringSupplier = new ArrayList<>();
List<ScorerSupplier> 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<Scorer> requiredNoScoring = new ArrayList<>();
for (ScorerSupplier ss : requiredNoScoringSupplier) {
requiredNoScoring.add(ss.get(leadCost));
}
List<Scorer> 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<Scorer> 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<Scorer> 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 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);
}
optional, Math.max(1, query.getMinimumNumberShouldMatch()), scoreMode.needsScores());
}
@Override
@ -619,6 +389,7 @@ final class BooleanWeight extends Weight {
scorers.get(Occur.SHOULD).clear();
}
return new Boolean2ScorerSupplier(this, scorers, scoreMode, minShouldMatch);
return new BooleanScorerSupplier(
this, scorers, scoreMode, minShouldMatch, context.reader().maxDoc());
}
}

View File

@ -136,17 +136,6 @@ public final class ConstantScoreQuery extends Query {
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 final class ConstantScoreQuery extends Query {
}
}
@Override
public BulkScorer bulkScorer() throws IOException {
if (scoreMode.isExhaustive() == false) {
return super.bulkScorer();
}
final BulkScorer innerScorer = innerScorerSupplier.bulkScorer();
if (innerScorer == null) {
return null;
}
return new ConstantBulkScorer(innerScorer, innerWeight, score());
}
@Override
public long cost() {
return innerScorerSupplier.cost();

View File

@ -148,13 +148,6 @@ public final class IndexOrDocValuesQuery extends Query {
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 final class IndexOrDocValuesQuery extends Query {
}
}
@Override
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.bulkScorer();
}
@Override
public long cost() {
return indexScorerSupplier.cost();

View File

@ -704,8 +704,10 @@ public class IndexSearcher {
// 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.bulkScorer();
if (queryTimeout != null) {
scorer = new TimeLimitingBulkScorer(scorer, queryTimeout);
}

View File

@ -64,16 +64,6 @@ public class IndriAndWeight extends Weight {
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) {

View File

@ -711,15 +711,6 @@ public class LRUQueryCache implements QueryCache, Accountable {
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,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
return supplier.get(leadCost);
}
Scorer scorer = supplier.get(Long.MAX_VALUE);
CacheAndCount cached =
cacheImpl(new DefaultBulkScorer(scorer), context.reader().maxDoc());
CacheAndCount cached = cacheImpl(supplier.bulkScorer(), context.reader().maxDoc());
putIfAbsent(in.getQuery(), cached, cacheHelper);
DocIdSetIterator disi = cached.iterator();
if (disi == null) {
@ -884,63 +873,6 @@ public class LRUQueryCache implements QueryCache, Accountable {
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. */

View File

@ -33,10 +33,50 @@ public final class MatchAllDocsQuery extends Query {
@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);
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 bulkScorer() throws IOException {
if (scoreMode.isExhaustive() == false) {
return super.bulkScorer();
}
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 context.reader().maxDoc();
}
};
}
@Override
@ -44,36 +84,6 @@ public final class MatchAllDocsQuery extends Query {
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() {
@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 int count(LeafReaderContext context) {
return context.reader().numDocs();

View File

@ -22,26 +22,46 @@ import org.apache.lucene.util.Bits;
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;
}
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;
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));
}

View File

@ -17,6 +17,7 @@
package org.apache.lucene.search;
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
@ -35,6 +36,15 @@ public abstract class ScorerSupplier {
*/
public abstract Scorer get(long leadCost) 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));
}
/**
* 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.

View File

@ -61,7 +61,7 @@ final class TimeLimitingBulkScorer extends BulkScorer {
* TimeLimitingBulkScorer.TimeExceededException} is thrown
*/
public TimeLimitingBulkScorer(BulkScorer bulkScorer, QueryTimeout queryTimeout) {
this.in = bulkScorer;
this.in = Objects.requireNonNull(bulkScorer);
this.queryTimeout = Objects.requireNonNull(queryTimeout);
}

View File

@ -148,20 +148,21 @@ public abstract class Weight implements SegmentCacheable {
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
*
* @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
* <pre class="prettyprint">
* ScorerSupplier scorerSupplier = scorerSupplier(context);
* if (scorerSupplier == null) {
* // No docs match
* return null;
* }
*
* scorerSupplier.setTopLevelScoringClause();
* return scorerSupplier.bulkScorer();
* </pre>
*/
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,8 +170,7 @@ public abstract class Weight implements SegmentCacheable {
}
scorerSupplier.setTopLevelScoringClause();
return new DefaultBulkScorer(scorerSupplier.get(Long.MAX_VALUE));
return scorerSupplier.bulkScorer();
}
/**

View File

@ -137,24 +137,24 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
.add(new FakeScorerSupplier(42));
assertEquals(
42,
new Boolean2ScorerSupplier(
null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
new BooleanScorerSupplier(
null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.cost());
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST)))
.add(new FakeScorerSupplier(12));
assertEquals(
12,
new Boolean2ScorerSupplier(
null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
new BooleanScorerSupplier(
null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.cost());
subs.get(RandomPicks.randomFrom(random(), Arrays.asList(Occur.FILTER, Occur.MUST)))
.add(new FakeScorerSupplier(20));
assertEquals(
12,
new Boolean2ScorerSupplier(
null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
new BooleanScorerSupplier(
null, subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.cost());
}
@ -166,22 +166,22 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
ScorerSupplier s =
new Boolean2ScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0);
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,37 +195,37 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(42));
subs.get(Occur.SHOULD).add(new FakeScorerSupplier(12));
ScorerSupplier s =
new Boolean2ScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 1);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2);
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3);
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 class TestBoolean2ScorerSupplier extends LuceneTestCase {
continue;
}
int minShouldMatch = numShoulds == 0 ? 0 : TestUtil.nextInt(random(), 0, numShoulds - 1);
Boolean2ScorerSupplier supplier =
new Boolean2ScorerSupplier(new FakeWeight(), subs, scoreMode, minShouldMatch);
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,8 +287,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
.add(new FakeScorerSupplier(42, 12));
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 BooleanScorerSupplier(
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);
@ -301,8 +301,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
.add(new FakeScorerSupplier(42, 7));
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 BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.get(7); // triggers assertions as a side-effect
}
@ -313,15 +313,15 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
}
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 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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.get(20); // triggers assertions as a side-effect
}
@ -336,8 +336,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100)
.get(100); // triggers assertions as a side-effect
subs = new EnumMap<>(Occur.class);
@ -349,8 +349,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100)
.get(20); // triggers assertions as a side-effect
subs = new EnumMap<>(Occur.class);
@ -362,8 +362,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 2, 100)
.get(100); // triggers assertions as a side-effect
subs = new EnumMap<>(Occur.class);
@ -375,8 +375,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 3, 100)
.get(100); // triggers assertions as a side-effect
}
@ -389,24 +389,24 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
// 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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.get(100); // triggers assertions as a side-effect
subs.get(Occur.MUST).clear();
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
new BooleanScorerSupplier(
new FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0, 100)
.get(100); // triggers assertions as a side-effect
subs.get(Occur.MUST).clear();
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 FakeWeight(), subs, RandomPicks.randomFrom(random(), ScoreMode.values()), 0)
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 class TestBoolean2ScorerSupplier extends LuceneTestCase {
// 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 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)
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)
new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.COMPLETE, 0, 100)
.get(20); // triggers assertions as a side-effect
}
@ -448,7 +448,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
subs.get(Occur.SHOULD).add(clause2);
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
.setTopLevelScoringClause();
assertFalse(clause1.topLevelScoringClause);
assertFalse(clause2.topLevelScoringClause);
@ -465,7 +465,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
subs.get(Occur.MUST).add(clause2);
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
.setTopLevelScoringClause();
assertFalse(clause1.topLevelScoringClause);
assertFalse(clause2.topLevelScoringClause);
@ -482,7 +482,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
subs.get(Occur.FILTER).add(clause2);
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
.setTopLevelScoringClause();
assertFalse(clause1.topLevelScoringClause);
assertFalse(clause2.topLevelScoringClause);
@ -499,7 +499,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
subs.get(Occur.FILTER).add(clause2);
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
.setTopLevelScoringClause();
assertTrue(clause1.topLevelScoringClause);
assertFalse(clause2.topLevelScoringClause);
@ -516,7 +516,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
FakeScorerSupplier clause2 = new FakeScorerSupplier(10, 10);
subs.get(Occur.MUST_NOT).add(clause2);
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0)
new BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100)
.setTopLevelScoringClause();
assertTrue(clause1.topLevelScoringClause);
assertFalse(clause2.topLevelScoringClause);
@ -534,7 +534,7 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
subs.get(Occur.MUST).add(clause2);
Scorer scorer =
new Boolean2ScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0).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);
@ -545,7 +545,8 @@ public class TestBoolean2ScorerSupplier extends LuceneTestCase {
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 BooleanScorerSupplier(new FakeWeight(), subs, ScoreMode.TOP_SCORES, 0, 100).get(10);
assertEquals(2.0, scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0.0);
}
}

View File

@ -246,7 +246,7 @@ public class TestBooleanOr extends LuceneTestCase {
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<Integer> matches = new ArrayList<>();
scorer.score(
new LeafCollector() {

View File

@ -94,6 +94,25 @@ public class TestBooleanScorer extends LuceneTestCase {
public long cost() {
throw new UnsupportedOperationException();
}
@Override
public BulkScorer bulkScorer() 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 class TestBooleanScorer extends LuceneTestCase {
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;
}
};
}
};
}
@ -185,7 +185,8 @@ public class TestBooleanScorer extends LuceneTestCase {
// 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 = ((BooleanScorerSupplier) ss).booleanScorer();
assertTrue(scorer instanceof DefaultBulkScorer); // term scorer
// scores -> term scorer too
@ -195,7 +196,8 @@ public class TestBooleanScorer extends LuceneTestCase {
.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 = ((BooleanScorerSupplier) ss).booleanScorer();
assertTrue(scorer instanceof DefaultBulkScorer); // term scorer
w.close();
@ -225,7 +227,8 @@ public class TestBooleanScorer extends LuceneTestCase {
.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 = ((BooleanScorerSupplier) ss).booleanScorer();
assertTrue(scorer instanceof ReqExclBulkScorer);
query =
@ -235,7 +238,8 @@ public class TestBooleanScorer extends LuceneTestCase {
.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 = ((BooleanScorerSupplier) ss).booleanScorer();
assertTrue(scorer instanceof ReqExclBulkScorer);
query =
@ -244,7 +248,8 @@ public class TestBooleanScorer extends LuceneTestCase {
.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 = ((BooleanScorerSupplier) ss).booleanScorer();
assertTrue(scorer instanceof ReqExclBulkScorer);
query =
@ -253,7 +258,8 @@ public class TestBooleanScorer extends LuceneTestCase {
.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 = ((BooleanScorerSupplier) ss).booleanScorer();
assertTrue(scorer instanceof ReqExclBulkScorer);
w.close();

View File

@ -1368,18 +1368,18 @@ public class TestLRUQueryCache extends LuceneTestCase {
return scorer;
}
@Override
public BulkScorer bulkScorer() 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 {

View File

@ -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 class TestReqExclBulkScorer extends LuceneTestCase {
}
};
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(

View File

@ -154,35 +154,7 @@ class DrillSidewaysQuery extends Query {
@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 @@ class DrillSidewaysQuery extends Query {
// 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 @@ class DrillSidewaysQuery extends Query {
// 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 bulkScorer() 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;
}
};
}

View File

@ -76,6 +76,18 @@ class QueryProfilerWeight extends FilterWeight {
}
}
@Override
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
// 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();
}
@Override
public long cost() {
timer.start();
@ -93,18 +105,6 @@ class QueryProfilerWeight extends FilterWeight {
};
}
@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;

View File

@ -62,36 +62,6 @@ public class CompletionWeight extends Weight {
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 @@ public class CompletionWeight extends Weight {
@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 bulkScorer() 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();

View File

@ -78,6 +78,26 @@ class AssertingWeight extends FilterWeight {
topLevelScoringClause);
}
@Override
public BulkScorer bulkScorer() 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.bulkScorer();
} else {
// Don't set getCalled = true, since this calls #get under the hood
inScorer = super.bulkScorer();
assert getCalled;
}
return AssertingBulkScorer.wrap(
new Random(random.nextLong()), inScorer, context.reader().maxDoc(), scoreMode);
}
@Override
public long cost() {
final long cost = inScorerSupplier.cost();
@ -93,22 +113,4 @@ class AssertingWeight extends FilterWeight {
}
};
}
@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);
}
}

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.Query;
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 class DisablingBulkScorerQuery extends Query {
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 bulkScorer() throws IOException {
return new DefaultBulkScorer(get(Long.MAX_VALUE));
}
};
}
};
}

View File

@ -149,13 +149,18 @@ public class RandomApproximationQuery extends Query {
}
}
private static class RandomTwoPhaseView extends TwoPhaseIterator {
/**
* 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;
RandomTwoPhaseView(Random random, DocIdSetIterator disi) {
/** Constructor. */
public RandomTwoPhaseView(Random random, DocIdSetIterator disi) {
super(new RandomApproximation(random, disi));
this.disi = disi;
this.randomMatchCost = random.nextFloat() * 200; // between 0 and 200

View File

@ -19,7 +19,6 @@ package org.apache.lucene.tests.search;
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 class TestBaseExplanationTestCase extends BaseExplanationTestCase {
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();