Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id AE016200D4E for ; Thu, 7 Dec 2017 13:45:10 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id ACE88160BFE; Thu, 7 Dec 2017 12:45:10 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 626CC160C0C for ; Thu, 7 Dec 2017 13:45:08 +0100 (CET) Received: (qmail 73747 invoked by uid 500); 7 Dec 2017 12:45:00 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 70057 invoked by uid 99); 7 Dec 2017 12:44:57 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 07 Dec 2017 12:44:57 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 5F2F9F611E; Thu, 7 Dec 2017 12:44:56 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: ab@apache.org To: commits@lucene.apache.org Date: Thu, 07 Dec 2017 12:45:40 -0000 Message-Id: <69346a0ff05b45aab76a24095548b4db@git.apache.org> In-Reply-To: <021402763f9a47438e6e1d24785713d8@git.apache.org> References: <021402763f9a47438e6e1d24785713d8@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [46/50] [abbrv] lucene-solr:jira/solr-11285-sim: LUCENE-4100: Faster disjunctions when the hit count is not needed. archived-at: Thu, 07 Dec 2017 12:45:10 -0000 http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java index 19b3922..95f6564 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java @@ -205,7 +205,7 @@ public class TermInSetQuery extends Query implements Accountable { } @Override - public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { return new ConstantScoreWeight(this, boost) { @Override @@ -269,7 +269,7 @@ public class TermInSetQuery extends Query implements Accountable { bq.add(new TermQuery(new Term(t.field, t.term), termContext), Occur.SHOULD); } Query q = new ConstantScoreQuery(bq.build()); - final Weight weight = searcher.rewrite(q).createWeight(searcher, needsScores, score()); + final Weight weight = searcher.rewrite(q).createWeight(searcher, scoreMode, score()); return new WeightOrDocIdSet(weight); } else { assert builder != null; http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TermQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java index 5b3f1ad..925fe93 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermQuery.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Objects; import java.util.Set; +import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReaderContext; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -94,9 +95,25 @@ public class TermQuery extends Query { if (termsEnum == null) { return null; } + IndexOptions indexOptions = context.reader() + .getFieldInfos() + .fieldInfo(getTerm().field()) + .getIndexOptions(); PostingsEnum docs = termsEnum.postings(null, needsScores ? PostingsEnum.FREQS : PostingsEnum.NONE); assert docs != null; - return new TermScorer(this, docs, similarity.simScorer(stats, context)); + return new TermScorer(this, docs, similarity.simScorer(stats, context), + getMaxFreq(indexOptions, termsEnum.totalTermFreq(), termsEnum.docFreq())); + } + + private long getMaxFreq(IndexOptions indexOptions, long ttf, long df) { + // TODO: store the max term freq? + if (indexOptions.compareTo(IndexOptions.DOCS) <= 0) { + // omitTFAP field, tf values are implicitly 1. + return 1; + } else { + assert ttf >= 0; + return Math.min(Integer.MAX_VALUE, ttf - df + 1); + } } @Override @@ -185,12 +202,12 @@ public class TermQuery extends Query { } @Override - public Weight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { final IndexReaderContext context = searcher.getTopReaderContext(); final TermContext termState; if (perReaderTermState == null || perReaderTermState.wasBuiltFor(context) == false) { - if (needsScores) { + if (scoreMode.needsScores()) { // make TermQuery single-pass if we don't have a PRTS or if the context // differs! termState = TermContext.build(context, term); @@ -204,7 +221,7 @@ public class TermQuery extends Query { termState = this.perReaderTermState; } - return new TermWeight(searcher, needsScores, boost, termState); + return new TermWeight(searcher, scoreMode.needsScores(), boost, termState); } /** Prints a user-readable version of this query. */ http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TermScorer.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java index c478a25..a4aeb04 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TermScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/TermScorer.java @@ -27,6 +27,7 @@ import org.apache.lucene.search.similarities.Similarity; final class TermScorer extends Scorer { private final PostingsEnum postingsEnum; private final Similarity.SimScorer docScorer; + private final float maxFreq; /** * Construct a TermScorer. @@ -38,11 +39,14 @@ final class TermScorer extends Scorer { * @param docScorer * The Similarity.SimScorer implementation * to be used for score computations. + * @param maxFreq + * An upper bound of the term frequency of the searched term in any document. */ - TermScorer(Weight weight, PostingsEnum td, Similarity.SimScorer docScorer) { + TermScorer(Weight weight, PostingsEnum td, Similarity.SimScorer docScorer, float maxFreq) { super(weight); this.docScorer = docScorer; this.postingsEnum = td; + this.maxFreq = maxFreq; } @Override @@ -65,6 +69,11 @@ final class TermScorer extends Scorer { return docScorer.score(postingsEnum.docID(), postingsEnum.freq()); } + @Override + public float maxScore() { + return docScorer.maxScore(maxFreq); + } + /** Returns a string representation of this TermScorer. */ @Override public String toString() { return "scorer(" + weight + ")[" + super.toString() + "]"; } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java index 4708b20..6957994 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java @@ -162,8 +162,8 @@ public class TimeLimitingCollector implements Collector { } @Override - public boolean needsScores() { - return collector.needsScores(); + public ScoreMode scoreMode() { + return collector.scoreMode(); } /** http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java index 86bf17e..219ee3a 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopDocsCollector.java @@ -43,7 +43,7 @@ public abstract class TopDocsCollector implements Collector * HitQueue for example aggregates the top scoring documents, while other PQ * implementations may hold documents sorted by other criteria. */ - protected PriorityQueue pq; + protected final PriorityQueue pq; /** The total number of documents that the collector encountered. */ protected int totalHits; http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java index 94f156e..c3597e9 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java @@ -338,8 +338,8 @@ public abstract class TopFieldCollector extends TopDocsCollector { } @Override - public boolean needsScores() { - return needsScores; + public ScoreMode scoreMode() { + return needsScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES; } /** http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java index bc5ba16..afae1fc 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java @@ -49,17 +49,37 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { private static class SimpleTopScoreDocCollector extends TopScoreDocCollector { - SimpleTopScoreDocCollector(int numHits) { + private final int numHits; + private final boolean trackTotalHits; + private int sumMaxDoc; + private int maxCollectedExactly = -1; + + SimpleTopScoreDocCollector(int numHits, boolean trackTotalHits) { super(numHits); + this.numHits = numHits; + this.trackTotalHits = trackTotalHits; } @Override public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { final int docBase = context.docBase; + sumMaxDoc += context.reader().maxDoc(); return new ScorerLeafCollector() { @Override + public void setScorer(Scorer scorer) throws IOException { + super.setScorer(scorer); + if (trackTotalHits == false + && pqTop != null + && pqTop.score != Float.NEGATIVE_INFINITY) { + // since we tie-break on doc id and collect in doc id order, we can require + // the next float + scorer.setMinCompetitiveScore(Math.nextUp(pqTop.score)); + } + } + + @Override public void collect(int doc) throws IOException { float score = scorer.score(); @@ -76,11 +96,38 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { pqTop.doc = doc + docBase; pqTop.score = score; pqTop = pq.updateTop(); + if (trackTotalHits == false && pqTop.score != Float.NEGATIVE_INFINITY) { // -Infinity is the score of sentinels + // since we tie-break on doc id and collect in doc id order, we can require + // the next float + scorer.setMinCompetitiveScore(Math.nextUp(pqTop.score)); + if (maxCollectedExactly < 0) { + assert totalHits == numHits; + maxCollectedExactly = doc + docBase; + } + } } }; } + @Override + public TopDocs topDocs() { + TopDocs topDocs = super.topDocs(); + if (trackTotalHits == false && maxCollectedExactly >= 0) { + // assume matches are evenly spread in the doc id space + // this may be completely off + long totalHitsEstimate = (long) numHits * sumMaxDoc / (maxCollectedExactly + 1); + // we take the max since the current topDocs.totalHits is a lower bound + // of the total hit count + topDocs.totalHits = Math.max(topDocs.totalHits, totalHitsEstimate); + } + return topDocs; + } + + @Override + public ScoreMode scoreMode() { + return trackTotalHits ? ScoreMode.COMPLETE : ScoreMode.TOP_SCORES; + } } private static class PagingTopScoreDocCollector extends TopScoreDocCollector { @@ -140,8 +187,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { /** * Creates a new {@link TopScoreDocCollector} given the number of hits to - * collect and whether documents are scored in order by the input - * {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}. + * collect. * *

NOTE: The instances returned by this method * pre-allocate a full array of length @@ -149,27 +195,30 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { * objects. */ public static TopScoreDocCollector create(int numHits) { - return create(numHits, null); + return create(numHits, null, true); } /** * Creates a new {@link TopScoreDocCollector} given the number of hits to - * collect, the bottom of the previous page, and whether documents are scored in order by the input - * {@link Scorer} to {@link LeafCollector#setScorer(Scorer)}. + * collect, the bottom of the previous page, and whether the total hit count + * is needed. * + *

NOTE: If {@code trackTotalHits} is {@code false} then the + * {@link TopDocs#totalHits} of the returned {@link TopDocs} will be an + * approximation and may be completely off. *

NOTE: The instances returned by this method * pre-allocate a full array of length * numHits, and fill the array with sentinel * objects. */ - public static TopScoreDocCollector create(int numHits, ScoreDoc after) { + public static TopScoreDocCollector create(int numHits, ScoreDoc after, boolean trackTotalHits) { if (numHits <= 0) { throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count"); } if (after == null) { - return new SimpleTopScoreDocCollector(numHits); + return new SimpleTopScoreDocCollector(numHits, trackTotalHits); } else { return new PagingTopScoreDocCollector(numHits, after); } @@ -207,7 +256,7 @@ public abstract class TopScoreDocCollector extends TopDocsCollector { } @Override - public boolean needsScores() { - return true; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java b/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java index e5180f9..2d0139c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java +++ b/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java @@ -35,7 +35,7 @@ public class TotalHitCountCollector extends SimpleCollector { } @Override - public boolean needsScores() { - return false; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java new file mode 100644 index 0000000..2f3b600 --- /dev/null +++ b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java @@ -0,0 +1,478 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.lucene.search; + +import static org.apache.lucene.search.DisiPriorityQueue.leftNode; +import static org.apache.lucene.search.DisiPriorityQueue.parentNode; +import static org.apache.lucene.search.DisiPriorityQueue.rightNode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.OptionalInt; + +/** + * This implements the WAND (Weak AND) algorithm for dynamic pruning + * described in "Efficient Query Evaluation using a Two-Level Retrieval + * Process" by Broder, Carmel, Herscovici, Soffer and Zien. + * This scorer maintains a feedback loop with the collector in order to + * know at any time the minimum score that is required in order for a hit + * to be competitive. Then it leverages the {@link Scorer#maxScore() max score} + * from each scorer in order to know when it may call + * {@link DocIdSetIterator#advance} rather than {@link DocIdSetIterator#nextDoc} + * to move to the next competitive hit. + * Implementation is similar to {@link MinShouldMatchSumScorer} except that + * instead of enforcing that {@code freq >= minShouldMatch}, we enforce that + * {@code ∑ max_score >= minCompetitiveScore}. + */ +final class WANDScorer extends Scorer { + + /** Return a scaling factor for the given float so that + * f x 2^scalingFactor would be in ]2^15, 2^16]. Special + * cases: + * scalingFactor(0) = scalingFactor(MIN_VALUE) - 1 + * scalingFactor(+Infty) = scalingFactor(MAX_VALUE) + 1 + */ + static int scalingFactor(float f) { + if (f < 0) { + throw new IllegalArgumentException(""); + } else if (f == 0) { + return scalingFactor(Float.MIN_VALUE) - 1; + } else if (Float.isInfinite(f)) { + return scalingFactor(Float.MAX_VALUE) + 1; + } else { + double d = f; + // Since doubles have more amplitude than floats for the + // exponent, the cast produces a normal value. + assert d == 0 || Math.getExponent(d) >= Double.MIN_EXPONENT; // normal double + return 15 - Math.getExponent(Math.nextDown(d)); + } + } + + /** + * Scale max scores in an unsigned integer to avoid overflows + * (only the lower 32 bits of the long are used) as well as + * floating-point arithmetic errors. Those are rounded up in order + * to make sure we do not miss any matches. + */ + private static long scaleMaxScore(float maxScore, int scalingFactor) { + assert Float.isNaN(maxScore) == false; + assert maxScore >= 0; + + if (Float.isInfinite(maxScore)) { + return (1L << 32) - 1; // means +Infinity in practice for this scorer + } + + // NOTE: because doubles have more amplitude than floats for the + // exponent, the scalb call produces an accurate value. + double scaled = Math.scalb((double) maxScore, scalingFactor); + assert scaled <= 1 << 16 : scaled + " " + maxScore; // regular values of max_score go into 0..2^16 + return (long) Math.ceil(scaled); // round up, cast is accurate since value is <= 2^16 + } + + /** + * Scale min competitive scores the same way as max scores but this time + * by rounding down in order to make sure that we do not miss any matches. + */ + private static long scaleMinScore(float minScore, int scalingFactor) { + assert Float.isNaN(minScore) == false; + assert minScore >= 0; + + // like for scaleMaxScore, this scalb call is accurate + double scaled = Math.scalb((double) minScore, scalingFactor); + return (long) Math.floor(scaled); // round down, cast might lower the value again if scaled > Long.MAX_VALUE, which is fine + } + + private final int scalingFactor; + // scaled min competitive score + private long minCompetitiveScore = 0; + + // list of scorers which 'lead' the iteration and are currently + // positioned on 'doc'. This is sometimes called the 'pivot' in + // some descriptions of WAND (Weak AND). + DisiWrapper lead; + int doc; // current doc ID of the leads + long leadMaxScore; // sum of the max scores of scorers in 'lead' + + // priority queue of scorers that are too advanced compared to the current + // doc. Ordered by doc ID. + final DisiPriorityQueue head; + + // priority queue of scorers which are behind the current doc. + // Ordered by maxScore. + final DisiWrapper[] tail; + long tailMaxScore; // sum of the max scores of scorers in 'tail' + int tailSize; + + final long cost; + + WANDScorer(Weight weight, Collection scorers) { + super(weight); + + this.minCompetitiveScore = 0; + this.doc = -1; + + head = new DisiPriorityQueue(scorers.size()); + // there can be at most num_scorers - 1 scorers beyond the current position + tail = new DisiWrapper[scorers.size() - 1]; + + OptionalInt scalingFactor = OptionalInt.empty(); + for (Scorer scorer : scorers) { + float maxScore = scorer.maxScore(); + if (maxScore != 0 && Float.isFinite(maxScore)) { + // 0 and +Infty should not impact the scale + scalingFactor = OptionalInt.of(Math.min(scalingFactor.orElse(Integer.MAX_VALUE), scalingFactor(maxScore))); + } + } + // Use a scaling factor of 0 if all max scores are either 0 or +Infty + this.scalingFactor = scalingFactor.orElse(0); + + for (Scorer scorer : scorers) { + DisiWrapper w = new DisiWrapper(scorer); + float maxScore = scorer.maxScore(); + w.maxScore = scaleMaxScore(maxScore, this.scalingFactor); + addLead(w); + } + + long cost = 0; + for (DisiWrapper w = lead; w != null; w = w.next) { + cost += w.cost; + } + this.cost = cost; + } + + // returns a boolean so that it can be called from assert + // the return value is useless: it always returns true + private boolean ensureConsistent() { + long maxScoreSum = 0; + for (int i = 0; i < tailSize; ++i) { + assert tail[i].doc < doc; + maxScoreSum = Math.addExact(maxScoreSum, tail[i].maxScore); + } + assert maxScoreSum == tailMaxScore : maxScoreSum + " " + tailMaxScore; + + maxScoreSum = 0; + for (DisiWrapper w = lead; w != null; w = w.next) { + assert w.doc == doc; + maxScoreSum = Math.addExact(maxScoreSum, w.maxScore); + } + assert maxScoreSum == leadMaxScore : maxScoreSum + " " + leadMaxScore; + + for (DisiWrapper w : head) { + assert w.doc > doc; + } + + assert tailSize == 0 || tailMaxScore < minCompetitiveScore; + + return true; + } + + @Override + public void setMinCompetitiveScore(float minScore) { + assert minScore >= 0; + long scaledMinScore = scaleMinScore(minScore, scalingFactor); + assert scaledMinScore >= minCompetitiveScore; + minCompetitiveScore = scaledMinScore; + } + + @Override + public final Collection getChildren() throws IOException { + List matchingChildren = new ArrayList<>(); + updateFreq(); + for (DisiWrapper s = lead; s != null; s = s.next) { + matchingChildren.add(new ChildScorer(s.scorer, "SHOULD")); + } + return matchingChildren; + } + + @Override + public DocIdSetIterator iterator() { + return TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator()); + } + + @Override + public TwoPhaseIterator twoPhaseIterator() { + DocIdSetIterator approximation = new DocIdSetIterator() { + + @Override + public int docID() { + return doc; + } + + @Override + public int nextDoc() throws IOException { + return advance(doc + 1); + } + + @Override + public int advance(int target) throws IOException { + assert ensureConsistent(); + + // Move 'lead' iterators back to the tail + pushBackLeads(target); + + // Advance 'head' as well + advanceHead(target); + + // Pop the new 'lead' from the 'head' + setDocAndFreq(); + + assert ensureConsistent(); + + // Advance to the next possible match + return doNextCandidate(); + } + + @Override + public long cost() { + return cost; + } + }; + return new TwoPhaseIterator(approximation) { + + @Override + public boolean matches() throws IOException { + while (leadMaxScore < minCompetitiveScore) { + if (leadMaxScore + tailMaxScore >= minCompetitiveScore) { + // a match on doc is still possible, try to + // advance scorers from the tail + advanceTail(); + } else { + return false; + } + } + return true; + } + + @Override + public float matchCost() { + // maximum number of scorer that matches() might advance + return tail.length; + } + + }; + } + + private void addLead(DisiWrapper lead) { + lead.next = this.lead; + this.lead = lead; + leadMaxScore += lead.maxScore; + } + + private void pushBackLeads(int target) throws IOException { + for (DisiWrapper s = lead; s != null; s = s.next) { + final DisiWrapper evicted = insertTailWithOverFlow(s); + if (evicted != null) { + evicted.doc = evicted.iterator.advance(target); + head.add(evicted); + } + } + } + + private void advanceHead(int target) throws IOException { + DisiWrapper headTop = head.top(); + while (headTop.doc < target) { + final DisiWrapper evicted = insertTailWithOverFlow(headTop); + if (evicted != null) { + evicted.doc = evicted.iterator.advance(target); + headTop = head.updateTop(evicted); + } else { + head.pop(); + headTop = head.top(); + } + } + } + + private void advanceTail(DisiWrapper disi) throws IOException { + disi.doc = disi.iterator.advance(doc); + if (disi.doc == doc) { + addLead(disi); + } else { + head.add(disi); + } + } + + private void advanceTail() throws IOException { + final DisiWrapper top = popTail(); + advanceTail(top); + } + + /** Reinitializes head, freq and doc from 'head' */ + private void setDocAndFreq() { + assert head.size() > 0; + + // The top of `head` defines the next potential match + // pop all documents which are on this doc + lead = head.pop(); + lead.next = null; + leadMaxScore = lead.maxScore; + doc = lead.doc; + while (head.size() > 0 && head.top().doc == doc) { + addLead(head.pop()); + } + } + + /** Move iterators to the tail until there is a potential match. */ + private int doNextCandidate() throws IOException { + while (leadMaxScore + tailMaxScore < minCompetitiveScore) { + // no match on doc is possible, move to the next potential match + if (head.size() == 0) { + // special case: the total max score is less than the min competitive score, there are no more matches + return doc = DocIdSetIterator.NO_MORE_DOCS; + } + pushBackLeads(doc + 1); + setDocAndFreq(); + assert ensureConsistent(); + } + + return doc; + } + + /** Advance all entries from the tail to know about all matches on the + * current doc. */ + private void updateFreq() throws IOException { + // we return the next doc when the sum of the scores of the potential + // matching clauses is high enough but some of the clauses in 'tail' might + // match as well + // in general we want to advance least-costly clauses first in order to + // skip over non-matching documents as fast as possible. However here, + // we are advancing everything anyway so iterating over clauses in + // (roughly) cost-descending order might help avoid some permutations in + // the head heap + for (int i = tailSize - 1; i >= 0; --i) { + advanceTail(tail[i]); + } + tailSize = 0; + tailMaxScore = 0; + assert ensureConsistent(); + } + + @Override + public float score() throws IOException { + // we need to know about all matches + updateFreq(); + double score = 0; + for (DisiWrapper s = lead; s != null; s = s.next) { + score += s.scorer.score(); + } + return (float) score; + } + + @Override + public float maxScore() { + // TODO: implement but be careful about floating-point errors. + return Float.POSITIVE_INFINITY; + } + + @Override + public int docID() { + return doc; + } + + /** Insert an entry in 'tail' and evict the least-costly scorer if full. */ + private DisiWrapper insertTailWithOverFlow(DisiWrapper s) { + if (tailSize < tail.length && tailMaxScore + s.maxScore < minCompetitiveScore) { + // we have free room for this new entry + addTail(s); + tailMaxScore += s.maxScore; + return null; + } else if (tailSize == 0) { + return s; + } else { + final DisiWrapper top = tail[0]; + if (greaterMaxScore(top, s) == false) { + return s; + } + // Swap top and s + tail[0] = s; + downHeapMaxScore(tail, tailSize); + tailMaxScore = tailMaxScore - top.maxScore + s.maxScore; + return top; + } + } + + /** Add an entry to 'tail'. Fails if over capacity. */ + private void addTail(DisiWrapper s) { + tail[tailSize] = s; + upHeapMaxScore(tail, tailSize); + tailSize += 1; + } + + /** Pop the least-costly scorer from 'tail'. */ + private DisiWrapper popTail() { + assert tailSize > 0; + final DisiWrapper result = tail[0]; + tail[0] = tail[--tailSize]; + downHeapMaxScore(tail, tailSize); + tailMaxScore -= result.maxScore; + return result; + } + + /** Heap helpers */ + + private static void upHeapMaxScore(DisiWrapper[] heap, int i) { + final DisiWrapper node = heap[i]; + int j = parentNode(i); + while (j >= 0 && greaterMaxScore(node, heap[j])) { + heap[i] = heap[j]; + i = j; + j = parentNode(j); + } + heap[i] = node; + } + + private static void downHeapMaxScore(DisiWrapper[] heap, int size) { + int i = 0; + final DisiWrapper node = heap[0]; + int j = leftNode(i); + if (j < size) { + int k = rightNode(j); + if (k < size && greaterMaxScore(heap[k], heap[j])) { + j = k; + } + if (greaterMaxScore(heap[j], node)) { + do { + heap[i] = heap[j]; + i = j; + j = leftNode(i); + k = rightNode(j); + if (k < size && greaterMaxScore(heap[k], heap[j])) { + j = k; + } + } while (j < size && greaterMaxScore(heap[j], node)); + heap[i] = node; + } + } + } + + /** + * In the tail, we want to get first entries that produce the maximum scores + * and in case of ties (eg. constant-score queries), those that have the least + * cost so that they are likely to advance further. + */ + private static boolean greaterMaxScore(DisiWrapper w1, DisiWrapper w2) { + if (w1.maxScore > w2.maxScore) { + return true; + } else if (w1.maxScore < w2.maxScore) { + return false; + } else { + return w1.cost < w2.cost; + } + } + +} http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/Weight.java ---------------------------------------------------------------------- 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 3892d4f..7853ccf 100644 --- a/lucene/core/src/java/org/apache/lucene/search/Weight.java +++ b/lucene/core/src/java/org/apache/lucene/search/Weight.java @@ -43,7 +43,7 @@ import org.apache.lucene.util.Bits; * A Weight is used in the following way: *

    *
  1. A Weight is constructed by a top-level query, given a - * IndexSearcher ({@link Query#createWeight(IndexSearcher, boolean, float)}). + * IndexSearcher ({@link Query#createWeight(IndexSearcher, ScoreMode, float)}). *
  2. A Scorer is constructed by * {@link #scorer(org.apache.lucene.index.LeafReaderContext)}. *
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/package-info.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/package-info.java b/lucene/core/src/java/org/apache/lucene/search/package-info.java index 52b9411..69c5c2a 100644 --- a/lucene/core/src/java/org/apache/lucene/search/package-info.java +++ b/lucene/core/src/java/org/apache/lucene/search/package-info.java @@ -338,7 +338,7 @@ * {@link org.apache.lucene.search.Query Query} class has several methods that are important for * derived classes: *
    - *
  1. {@link org.apache.lucene.search.Query#createWeight(IndexSearcher,boolean,float) createWeight(IndexSearcher searcher, boolean needsScores, float boost)} — A + *
  2. {@link org.apache.lucene.search.Query#createWeight(IndexSearcher,ScoreMode,float) createWeight(IndexSearcher searcher, boolean needsScores, float boost)} — A * {@link org.apache.lucene.search.Weight Weight} is the internal representation of the * Query, so each Query implementation must * provide an implementation of Weight. See the subsection on {@link org.apache.lucene.search.Query#rewrite(org.apache.lucene.index.IndexReader) rewrite(IndexReader reader)} — Rewrites queries into primitive queries. Primitive queries are: * {@link org.apache.lucene.search.TermQuery TermQuery}, * {@link org.apache.lucene.search.BooleanQuery BooleanQuery}, and other queries that implement {@link org.apache.lucene.search.Query#createWeight(IndexSearcher,boolean,float) createWeight(IndexSearcher searcher,boolean needsScores, float boost)}
  3. + * >and other queries that implement {@link org.apache.lucene.search.Query#createWeight(IndexSearcher,ScoreMode,float) createWeight(IndexSearcher searcher,boolean needsScores, float boost)} *
* *

The Weight Interface

@@ -453,7 +453,7 @@ *

Assuming we are not sorting (since sorting doesn't affect the raw Lucene score), * we call one of the search methods of the IndexSearcher, passing in the * {@link org.apache.lucene.search.Weight Weight} object created by - * {@link org.apache.lucene.search.IndexSearcher#createNormalizedWeight(org.apache.lucene.search.Query,boolean) + * {@link org.apache.lucene.search.IndexSearcher#createNormalizedWeight(org.apache.lucene.search.Query,ScoreMode) * IndexSearcher.createNormalizedWeight(Query,boolean)} and the number of results we want. * This method returns a {@link org.apache.lucene.search.TopDocs TopDocs} object, * which is an internal collection of search results. The IndexSearcher creates http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java index 9183cb6..2a7f353 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Axiomatic.java @@ -109,6 +109,12 @@ public abstract class Axiomatic extends SimilarityBase { } @Override + protected double maxScore(BasicStats stats, double maxFreq) { + // TODO: can we compute a better upper bound on the produced scores + return Double.POSITIVE_INFINITY; + } + + @Override protected void explain(List subs, BasicStats stats, int doc, double freq, double docLen) { if (stats.getBoost() != 1.0d) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java index 9810d3d..c920891 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java @@ -223,7 +223,13 @@ public class BM25Similarity extends Similarity { } return weightValue * (float) (freq / (freq + norm)); } - + + @Override + public float maxScore(float maxFreq) { + // TODO: leverage maxFreq and the min norm from the cache + return weightValue; + } + @Override public Explanation explain(int doc, Explanation freq) throws IOException { List subs = new ArrayList<>(); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java index e1ad2e8..35f7083 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/BooleanSimilarity.java @@ -71,6 +71,11 @@ public class BooleanSimilarity extends Similarity { } @Override + public float maxScore(float maxFreq) { + return boost; + } + + @Override public Explanation explain(int doc, Explanation freq) throws IOException { Explanation queryBoostExpl = Explanation.match(boost, "boost"); return Explanation.match( http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java index 8b7e43a..ca0f4aa 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/DFISimilarity.java @@ -62,6 +62,12 @@ public class DFISimilarity extends SimilarityBase { return stats.getBoost() * log2(measure + 1); } + @Override + protected double maxScore(BasicStats stats, double maxFreq) { + // TODO: can we compute a better upper bound on the produced scores + return Double.POSITIVE_INFINITY; + } + /** * Returns the measure of independence */ http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java index d793d94..788f30a 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/DFRSimilarity.java @@ -111,7 +111,13 @@ public class DFRSimilarity extends SimilarityBase { double aeTimes1pTfn = afterEffect.scoreTimes1pTfn(stats); return stats.getBoost() * basicModel.score(stats, tfn, aeTimes1pTfn); } - + + @Override + protected double maxScore(BasicStats stats, double maxFreq) { + // TODO: can we compute a better upper bound on the produced scores + return Double.POSITIVE_INFINITY; + } + @Override protected void explain(List subs, BasicStats stats, int doc, double freq, double docLen) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java index 875cbe4..a71614c 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/IBSimilarity.java @@ -104,6 +104,12 @@ public class IBSimilarity extends SimilarityBase { } @Override + protected double maxScore(BasicStats stats, double maxFreq) { + // TODO: can we compute a better upper bound on the produced scores + return Double.POSITIVE_INFINITY; + } + + @Override protected void explain( List subs, BasicStats stats, int doc, double freq, double docLen) { if (stats.getBoost() != 1.0d) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java index a901bad..2a4354e 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMDirichletSimilarity.java @@ -75,7 +75,13 @@ public class LMDirichletSimilarity extends LMSimilarity { Math.log(mu / (docLen + mu))); return score > 0.0d ? score : 0.0d; } - + + @Override + protected double maxScore(BasicStats stats, double maxFreq) { + // TODO: can we compute a better upper bound on the produced scores + return Double.POSITIVE_INFINITY; + } + @Override protected void explain(List subs, BasicStats stats, int doc, double freq, double docLen) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java index 2799e3a..fa0ebcf 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/LMJelinekMercerSimilarity.java @@ -65,7 +65,13 @@ public class LMJelinekMercerSimilarity extends LMSimilarity { ((1 - lambda) * freq / docLen) / (lambda * ((LMStats)stats).getCollectionProbability())); } - + + @Override + protected double maxScore(BasicStats stats, double maxFreq) { + // TODO: can we compute a better upper bound on the produced scores + return Double.POSITIVE_INFINITY; + } + @Override protected void explain(List subs, BasicStats stats, int doc, double freq, double docLen) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java index 15b472f..cbd61dd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java @@ -83,6 +83,15 @@ public class MultiSimilarity extends Similarity { } @Override + public float maxScore(float freq) { + float sumMaxScore = 0; + for (SimScorer subScorer : subScorers) { + sumMaxScore += subScorer.maxScore(freq); + } + return sumMaxScore; + } + + @Override public Explanation explain(int doc, Explanation freq) throws IOException { List subs = new ArrayList<>(); for (SimScorer subScorer : subScorers) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java index 23f5b62..30895eb 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java @@ -159,6 +159,13 @@ public abstract class Similarity { public abstract float score(int doc, float freq) throws IOException; /** + * Return the maximum score that this scorer may produce for freqs in {@code ]0, maxFreq]}. + * {@code Float.POSITIVE_INFINITY} is a fine return value if scores are not bounded. + * @param maxFreq the maximum frequency + */ + public abstract float maxScore(float maxFreq); + + /** * Explain the score for a single document * @param doc document id within the inverted index segment * @param freq Explanation of how the sloppy term frequency was computed http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java index 380673a..99a2e36 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java @@ -120,7 +120,14 @@ public abstract class SimilarityBase extends Similarity { * @return the score. */ protected abstract double score(BasicStats stats, double freq, double docLen); - + + /** + * Return the maximum value that may be returned by {@link #score(BasicStats, double, double)} + * for the given stats. + * @see org.apache.lucene.search.similarities.Similarity.SimScorer#maxScore(float) + */ + protected abstract double maxScore(BasicStats stats, double maxFreq); + /** * Subclasses should implement this method to explain the score. {@code expl} * already contains the score, the name of the class and the doc id, as well @@ -250,6 +257,11 @@ public abstract class SimilarityBase extends Similarity { } @Override + public float maxScore(float maxFreq) { + return (float) SimilarityBase.this.maxScore(stats, maxFreq); + } + + @Override public Explanation explain(int doc, Explanation freq) throws IOException { return SimilarityBase.this.explain(stats, doc, freq, getLengthValue(doc)); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java index 87b8b52..51e6278 100644 --- a/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java +++ b/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java @@ -561,6 +561,20 @@ public abstract class TFIDFSimilarity extends Similarity { } @Override + public float maxScore(float maxFreq) { + final float raw = tf(maxFreq) * weightValue; + if (norms == null) { + return raw; + } else { + float maxNormValue = Float.NEGATIVE_INFINITY; + for (float norm : normTable) { + maxNormValue = Math.max(maxNormValue, norm); + } + return raw * maxNormValue; + } + } + + @Override public Explanation explain(int doc, Explanation freq) throws IOException { return explainScore(doc, freq, stats, norms, normTable); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java index 7759683..4a4c4fb 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java @@ -20,6 +20,7 @@ package org.apache.lucene.search.spans; import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import java.io.IOException; import java.util.Objects; @@ -89,8 +90,8 @@ public final class FieldMaskingSpanQuery extends SpanQuery { // ...this is done to be more consistent with things like SpanFirstQuery @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { - return maskedQuery.createWeight(searcher, needsScores, boost); + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + return maskedQuery.createWeight(searcher, scoreMode, boost); } @Override http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanBoostQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanBoostQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanBoostQuery.java index 0ce3b0a..9556959 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanBoostQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanBoostQuery.java @@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; /** * Counterpart of {@link BoostQuery} for spans. @@ -108,8 +109,8 @@ public final class SpanBoostQuery extends SpanQuery { } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { - return query.createWeight(searcher, needsScores, SpanBoostQuery.this.boost * boost); + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + return query.createWeight(searcher, scoreMode, SpanBoostQuery.this.boost * boost); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java index 552d146..0d62f74 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanContainingQuery.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermContext; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreMode; /** Keep matches that contain another SpanScorer. */ public final class SpanContainingQuery extends SpanContainQuery { @@ -43,10 +44,10 @@ public final class SpanContainingQuery extends SpanContainQuery { } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { - SpanWeight bigWeight = big.createWeight(searcher, false, boost); - SpanWeight littleWeight = little.createWeight(searcher, false, boost); - return new SpanContainingWeight(searcher, needsScores ? getTermContexts(bigWeight, littleWeight) : null, + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + SpanWeight bigWeight = big.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + SpanWeight littleWeight = little.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + return new SpanContainingWeight(searcher, scoreMode.needsScores() ? getTermContexts(bigWeight, littleWeight) : null, bigWeight, littleWeight, boost); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java index 9c844d1..ee3f5de 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java @@ -29,6 +29,7 @@ import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.ScoringRewrite; import org.apache.lucene.search.TopTermsRewrite; @@ -95,7 +96,7 @@ public class SpanMultiTermQueryWrapper extends SpanQue } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { throw new IllegalArgumentException("Rewrite first!"); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java index 720d943..24a047f 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanNearQuery.java @@ -33,6 +33,7 @@ import org.apache.lucene.index.TermContext; import org.apache.lucene.index.Terms; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Weight; /** Matches spans which are near one another. One can specify slop, the @@ -177,12 +178,12 @@ public class SpanNearQuery extends SpanQuery implements Cloneable { } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { List subWeights = new ArrayList<>(); for (SpanQuery q : clauses) { - subWeights.add(q.createWeight(searcher, false, boost)); + subWeights.add(q.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost)); } - return new SpanNearWeight(subWeights, searcher, needsScores ? getTermContexts(subWeights) : null, boost); + return new SpanNearWeight(subWeights, searcher, scoreMode.needsScores() ? getTermContexts(subWeights) : null, boost); } public class SpanNearWeight extends SpanWeight { @@ -306,7 +307,7 @@ public class SpanNearQuery extends SpanQuery implements Cloneable { } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { return new SpanGapWeight(searcher, boost); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java index d8c7862..5b97f8d 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanNotQuery.java @@ -29,6 +29,7 @@ import org.apache.lucene.index.TermContext; import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.TwoPhaseIterator; /** Removes matches which overlap with another SpanQuery or which are @@ -97,10 +98,10 @@ public final class SpanNotQuery extends SpanQuery { @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { - SpanWeight includeWeight = include.createWeight(searcher, false, boost); - SpanWeight excludeWeight = exclude.createWeight(searcher, false, boost); - return new SpanNotWeight(searcher, needsScores ? getTermContexts(includeWeight, excludeWeight) : null, + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + SpanWeight includeWeight = include.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + SpanWeight excludeWeight = exclude.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + return new SpanNotWeight(searcher, scoreMode.needsScores() ? getTermContexts(includeWeight, excludeWeight) : null, includeWeight, excludeWeight, boost); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java index 59dbac7..2e15c92 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanOrQuery.java @@ -33,6 +33,7 @@ import org.apache.lucene.search.DisiWrapper; import org.apache.lucene.search.DisjunctionDISIApproximation; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.TwoPhaseIterator; import org.apache.lucene.search.Weight; @@ -115,12 +116,12 @@ public final class SpanOrQuery extends SpanQuery { } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { List subWeights = new ArrayList<>(clauses.size()); for (SpanQuery q : clauses) { - subWeights.add(q.createWeight(searcher, false, boost)); + subWeights.add(q.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost)); } - return new SpanOrWeight(searcher, needsScores ? getTermContexts(subWeights) : null, subWeights, boost); + return new SpanOrWeight(searcher, scoreMode.needsScores() ? getTermContexts(subWeights) : null, subWeights, boost); } public class SpanOrWeight extends SpanWeight { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java index dddf766..f9b7697 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java @@ -28,6 +28,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.TermContext; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.spans.FilterSpans.AcceptStatus; @@ -67,9 +68,9 @@ public abstract class SpanPositionCheckQuery extends SpanQuery implements Clonea protected abstract AcceptStatus acceptPosition(Spans spans) throws IOException; @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { - SpanWeight matchWeight = match.createWeight(searcher, false, boost); - return new SpanPositionCheckWeight(matchWeight, searcher, needsScores ? getTermContexts(matchWeight) : null, boost); + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + SpanWeight matchWeight = match.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + return new SpanPositionCheckWeight(matchWeight, searcher, scoreMode.needsScores() ? getTermContexts(matchWeight) : null, boost); } public class SpanPositionCheckWeight extends SpanWeight { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanQuery.java index 965f80e..607a375 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanQuery.java @@ -26,6 +26,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.index.TermContext; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreMode; /** Base class for span-based queries. */ public abstract class SpanQuery extends Query { @@ -36,7 +37,7 @@ public abstract class SpanQuery extends Query { public abstract String getField(); @Override - public abstract SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException; + public abstract SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException; /** * Build a map of terms to termcontexts, for use in constructing SpanWeights http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java index 699fa1b..57a68e4 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java @@ -134,6 +134,11 @@ public class SpanScorer extends Scorer { return scoreCurrentDoc(); } + @Override + public float maxScore() { + return Float.POSITIVE_INFINITY; + } + /** Returns the intermediate "sloppy freq" adjusted for edit distance * @lucene.internal */ final float sloppyFreq() throws IOException { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java index 4a2e6bc..9eea3aa 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java @@ -33,6 +33,7 @@ import org.apache.lucene.index.TermState; import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreMode; /** Matches spans containing a term. * This should not be used for terms that are indexed at position Integer.MAX_VALUE. @@ -64,7 +65,7 @@ public class SpanTermQuery extends SpanQuery { public String getField() { return term.field(); } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { final TermContext context; final IndexReaderContext topContext = searcher.getTopReaderContext(); if (termContext == null || termContext.wasBuiltFor(topContext) == false) { @@ -73,7 +74,7 @@ public class SpanTermQuery extends SpanQuery { else { context = termContext; } - return new SpanTermWeight(context, searcher, needsScores ? Collections.singletonMap(term, context) : null, boost); + return new SpanTermWeight(context, searcher, scoreMode.needsScores() ? Collections.singletonMap(term, context) : null, boost); } public class SpanTermWeight extends SpanWeight { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java b/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java index 0a8f4e4..9c618dd 100644 --- a/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java +++ b/lucene/core/src/java/org/apache/lucene/search/spans/SpanWithinQuery.java @@ -25,6 +25,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; import org.apache.lucene.index.TermContext; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ScoreMode; /** Keep matches that are contained within another Spans. */ public final class SpanWithinQuery extends SpanContainQuery { @@ -44,10 +45,10 @@ public final class SpanWithinQuery extends SpanContainQuery { } @Override - public SpanWeight createWeight(IndexSearcher searcher, boolean needsScores, float boost) throws IOException { - SpanWeight bigWeight = big.createWeight(searcher, false, boost); - SpanWeight littleWeight = little.createWeight(searcher, false, boost); - return new SpanWithinWeight(searcher, needsScores ? getTermContexts(bigWeight, littleWeight) : null, + public SpanWeight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { + SpanWeight bigWeight = big.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + SpanWeight littleWeight = little.createWeight(searcher, ScoreMode.COMPLETE_NO_SCORES, boost); + return new SpanWithinWeight(searcher, scoreMode.needsScores() ? getTermContexts(bigWeight, littleWeight) : null, bigWeight, littleWeight, boost); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java index 9b755c9..f391c5a 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestMaxTermFrequency.java @@ -123,6 +123,10 @@ public class TestMaxTermFrequency extends LuceneTestCase { return 0; } + @Override + public float maxScore(float maxFreq) { + return 0; + } }; } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java b/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java index 2c179c5..a995e16 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java @@ -31,6 +31,7 @@ import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.Explanation; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.PhraseQuery; +import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SimpleCollector; import org.apache.lucene.search.TermQuery; @@ -310,8 +311,8 @@ public class TestOmitTf extends LuceneTestCase { new CountingHitCollector() { private Scorer scorer; @Override - public boolean needsScores() { - return true; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; } @Override public final void setScorer(Scorer scorer) { @@ -332,8 +333,8 @@ public class TestOmitTf extends LuceneTestCase { new CountingHitCollector() { private Scorer scorer; @Override - public boolean needsScores() { - return true; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; } @Override public final void setScorer(Scorer scorer) { @@ -357,8 +358,8 @@ public class TestOmitTf extends LuceneTestCase { new CountingHitCollector() { private Scorer scorer; @Override - public boolean needsScores() { - return true; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; } @Override public final void setScorer(Scorer scorer) { @@ -380,8 +381,8 @@ public class TestOmitTf extends LuceneTestCase { new CountingHitCollector() { private Scorer scorer; @Override - public boolean needsScores() { - return true; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; } @Override public final void setScorer(Scorer scorer) { @@ -438,8 +439,8 @@ public class TestOmitTf extends LuceneTestCase { } @Override - public boolean needsScores() { - return false; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE_NO_SCORES; } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java b/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java index 18dae47..0523e2c 100644 --- a/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java +++ b/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java @@ -55,7 +55,7 @@ final class JustCompileSearch { } @Override - public boolean needsScores() { + public ScoreMode scoreMode() { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } } @@ -176,6 +176,11 @@ final class JustCompileSearch { } @Override + public float maxScore() { + throw new UnsupportedOperationException(UNSUPPORTED_MSG); + } + + @Override public int docID() { throw new UnsupportedOperationException(UNSUPPORTED_MSG); } @@ -231,7 +236,7 @@ final class JustCompileSearch { } @Override - public boolean needsScores() { + public ScoreMode scoreMode() { throw new UnsupportedOperationException( UNSUPPORTED_MSG ); } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4fc5a872/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java b/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java index de97954..d4e1e24 100644 --- a/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java +++ b/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java @@ -51,8 +51,8 @@ public class MultiCollectorTest extends LuceneTestCase { } @Override - public boolean needsScores() { - return true; + public ScoreMode scoreMode() { + return ScoreMode.COMPLETE; } } @@ -102,7 +102,7 @@ public class MultiCollectorTest extends LuceneTestCase { } - private static Collector collector(boolean needsScores, Class expectedScorer) { + private static Collector collector(ScoreMode scoreMode, Class expectedScorer) { return new Collector() { @Override @@ -121,8 +121,8 @@ public class MultiCollectorTest extends LuceneTestCase { } @Override - public boolean needsScores() { - return needsScores; + public ScoreMode scoreMode() { + return scoreMode; } }; @@ -139,22 +139,22 @@ public class MultiCollectorTest extends LuceneTestCase { final LeafReaderContext ctx = reader.leaves().get(0); expectThrows(AssertionError.class, () -> { - collector(false, ScoreCachingWrappingScorer.class).getLeafCollector(ctx).setScorer(new FakeScorer()); + collector(ScoreMode.COMPLETE_NO_SCORES, ScoreCachingWrappingScorer.class).getLeafCollector(ctx).setScorer(new FakeScorer()); }); // no collector needs scores => no caching - Collector c1 = collector(false, FakeScorer.class); - Collector c2 = collector(false, FakeScorer.class); + Collector c1 = collector(ScoreMode.COMPLETE_NO_SCORES, FakeScorer.class); + Collector c2 = collector(ScoreMode.COMPLETE_NO_SCORES, FakeScorer.class); MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new FakeScorer()); // only one collector needs scores => no caching - c1 = collector(true, FakeScorer.class); - c2 = collector(false, FakeScorer.class); + c1 = collector(ScoreMode.COMPLETE, FakeScorer.class); + c2 = collector(ScoreMode.COMPLETE_NO_SCORES, FakeScorer.class); MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new FakeScorer()); // several collectors need scores => caching - c1 = collector(true, ScoreCachingWrappingScorer.class); - c2 = collector(true, ScoreCachingWrappingScorer.class); + c1 = collector(ScoreMode.COMPLETE, ScoreCachingWrappingScorer.class); + c2 = collector(ScoreMode.COMPLETE, ScoreCachingWrappingScorer.class); MultiCollector.wrap(c1, c2).getLeafCollector(ctx).setScorer(new FakeScorer()); reader.close();