You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2021/01/25 06:45:24 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9668: Deprecate MinShouldMatchSumScorer with WANDScorer (#2205)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new d266a09  LUCENE-9668: Deprecate MinShouldMatchSumScorer with WANDScorer (#2205)
d266a09 is described below

commit d266a09ff4c82a891f38799b2c4d19a0a4d0db74
Author: zacharymorn <za...@yahoo.com>
AuthorDate: Sun Jan 24 22:28:19 2021 -0800

    LUCENE-9668: Deprecate MinShouldMatchSumScorer with WANDScorer (#2205)
---
 .../lucene/search/Boolean2ScorerSupplier.java      |  13 +-
 .../org/apache/lucene/search/BooleanScorer.java    |   2 +-
 .../lucene/search/MinShouldMatchSumScorer.java     | 386 ---------------------
 .../java/org/apache/lucene/search/WANDScorer.java  | 127 ++++---
 .../search/TestBooleanQueryVisitSubscorers.java    |   2 +-
 .../org/apache/lucene/search/TestWANDScorer.java   | 163 +++++++++
 6 files changed, 252 insertions(+), 441 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
index a40264d..bdfb868 100644
--- a/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
+++ b/lucene/core/src/java/org/apache/lucene/search/Boolean2ScorerSupplier.java
@@ -198,10 +198,15 @@ final class Boolean2ScorerSupplier extends ScorerSupplier {
         optionalScorers.add(scorer.get(leadCost));
       }
 
-      if (scoreMode == ScoreMode.TOP_SCORES) {
-        return new WANDScorer(weight, optionalScorers, minShouldMatch);
-      } else if (minShouldMatch > 1) {
-        return new MinShouldMatchSumScorer(weight, optionalScorers, minShouldMatch);
+      // Technically speaking, WANDScorer should be able to handle the following 3 conditions now
+      // 1. Any ScoreMode (with scoring or not)
+      // 2. Any minCompetitiveScore ( >= 0 )
+      // 3. Any minShouldMatch ( >= 0 )
+      //
+      // However, as WANDScorer uses more complex algorithm and data structure, we would like to
+      // still use DisjunctionSumScorer to handle exhaustive pure disjunctions, which may be faster
+      if (scoreMode == ScoreMode.TOP_SCORES || minShouldMatch > 1) {
+        return new WANDScorer(weight, optionalScorers, minShouldMatch, scoreMode);
       } else {
         return new DisjunctionSumScorer(weight, optionalScorers, scoreMode);
       }
diff --git a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
index 6ec17ba..b28124f 100644
--- a/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
@@ -63,7 +63,7 @@ final class BooleanScorer extends BulkScorer {
     }
   }
 
-  // See MinShouldMatchSumScorer for an explanation
+  // See WANDScorer for an explanation
   private static long cost(Collection<BulkScorer> scorers, int minShouldMatch) {
     final PriorityQueue<BulkScorer> pq = new PriorityQueue<BulkScorer>(scorers.size() - minShouldMatch + 1) {
       @Override
diff --git a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java b/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
deleted file mode 100644
index caef2c4..0000000
--- a/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/*
- * 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 java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-import static org.apache.lucene.search.DisiPriorityQueue.leftNode;
-import static org.apache.lucene.search.DisiPriorityQueue.parentNode;
-import static org.apache.lucene.search.DisiPriorityQueue.rightNode;
-
-/**
- * A {@link Scorer} for {@link BooleanQuery} when
- * {@link BooleanQuery.Builder#setMinimumNumberShouldMatch(int) minShouldMatch} is
- * between 2 and the total number of clauses.
- *
- * This implementation keeps sub scorers in 3 different places:
- *  - lead: a linked list of scorer that are positioned on the desired doc ID
- *  - tail: a heap that contains at most minShouldMatch - 1 scorers that are
- *    behind the desired doc ID. These scorers are ordered by cost so that we
- *    can advance the least costly ones first.
- *  - head: a heap that contains scorers which are beyond the desired doc ID,
- *    ordered by doc ID in order to move quickly to the next candidate.
- *
- * Finding the next match consists of first setting the desired doc ID to the
- * least entry in 'head' and then advance 'tail' until there is a match.
- */
-final class MinShouldMatchSumScorer extends Scorer {
-
-  final int minShouldMatch;
-
-  // list of scorers which 'lead' the iteration and are currently
-  // positioned on 'doc'
-  DisiWrapper lead;
-  int doc;  // current doc ID of the leads
-  int freq; // number of scorers on the desired doc ID
-
-  // 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 cost.
-  final DisiWrapper[] tail;
-  int tailSize;
-
-  final long cost;
-
-  MinShouldMatchSumScorer(Weight weight, Collection<Scorer> scorers, int minShouldMatch) {
-    super(weight);
-
-    if (minShouldMatch > scorers.size()) {
-      throw new IllegalArgumentException("minShouldMatch should be <= the number of scorers");
-    }
-    if (minShouldMatch < 1) {
-      throw new IllegalArgumentException("minShouldMatch should be >= 1");
-    }
-
-    this.minShouldMatch = minShouldMatch;
-    this.doc = -1;
-
-    head = new DisiPriorityQueue(scorers.size() - minShouldMatch + 1);
-    // there can be at most minShouldMatch - 1 scorers beyond the current position
-    // otherwise we might be skipping over matching documents
-    tail = new DisiWrapper[minShouldMatch - 1];
-
-    for (Scorer scorer : scorers) {
-      addLead(new DisiWrapper(scorer));
-    }
-
-    this.cost =
-        ScorerUtil.costWithMinShouldMatch(
-            scorers.stream().map(Scorer::iterator).mapToLong(DocIdSetIterator::cost),
-            scorers.size(),
-            minShouldMatch);
-  }
-
-  @Override
-  public final Collection<ChildScorable> getChildren() throws IOException {
-    List<ChildScorable> matchingChildren = new ArrayList<>();
-    updateFreq();
-    for (DisiWrapper s = lead; s != null; s = s.next) {
-      matchingChildren.add(new ChildScorable(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() {
-        assert doc == lead.doc;
-        return doc;
-      }
-
-      @Override
-      public int nextDoc() throws IOException {
-        // We are moving to the next doc ID, so scorers in 'lead' need to go in
-        // 'tail'. If there is not enough space in 'tail', then we take the least
-        // costly scorers and advance them.
-        for (DisiWrapper s = lead; s != null; s = s.next) {
-          final DisiWrapper evicted = insertTailWithOverFlow(s);
-          if (evicted != null) {
-            if (evicted.doc == doc) {
-              evicted.doc = evicted.iterator.nextDoc();
-            } else {
-              evicted.doc = evicted.iterator.advance(doc + 1);
-            }
-            head.add(evicted);
-          }
-        }
-
-        setDocAndFreq();
-        // It would be correct to return doNextCandidate() at this point but if you
-        // call nextDoc as opposed to advance, it probably means that you really
-        // need the next match. Returning 'doc' here would lead to a similar
-        // iteration over sub postings overall except that the decision making would
-        // happen at a higher level where more abstractions are involved and
-        // benchmarks suggested it causes a significant performance hit.
-        return doNext();
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        // Same logic as in nextDoc
-        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);
-          }
-        }
-
-        // But this time there might also be scorers in 'head' behind the desired
-        // target so we need to do the same thing that we did on 'lead' on 'head'
-        DisiWrapper headTop = head.top();
-        while (headTop.doc < target) {
-          final DisiWrapper evicted = insertTailWithOverFlow(headTop);
-          // We know that the tail is full since it contains at most
-          // minShouldMatch - 1 entries and we just moved at least minShouldMatch
-          // entries to it, so evicted is not null
-          evicted.doc = evicted.iterator.advance(target);
-          headTop = head.updateTop(evicted);
-        }
-
-        setDocAndFreq();
-        return doNextCandidate();
-      }
-
-      @Override
-      public long cost() {
-        return cost;
-      }
-    };
-    return new TwoPhaseIterator(approximation) {
-
-      @Override
-      public boolean matches() throws IOException {
-        while (freq < minShouldMatch) {
-          assert freq > 0;
-          if (freq + tailSize >= minShouldMatch) {
-            // 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;
-    freq += 1;
-  }
-
-  private void pushBackLeads() throws IOException {
-    for (DisiWrapper s = lead; s != null; s = s.next) {
-      addTail(s);
-    }
-  }
-
-  private void advanceTail(DisiWrapper top) throws IOException {
-    top.doc = top.iterator.advance(doc);
-    if (top.doc == doc) {
-      addLead(top);
-    } else {
-      head.add(top);
-    }
-  }
-
-  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;
-    freq = 1;
-    doc = lead.doc;
-    while (head.size() > 0 && head.top().doc == doc) {
-      addLead(head.pop());
-    }
-  }
-
-  /** Advance tail to the lead until there is a match. */
-  private int doNext() throws IOException {
-    while (freq < minShouldMatch) {
-      assert freq > 0;
-      if (freq + tailSize >= minShouldMatch) {
-        // a match on doc is still possible, try to
-        // advance scorers from the tail
-        advanceTail();
-      } else {
-        // no match on doc is possible anymore, move to the next potential match
-        pushBackLeads();
-        setDocAndFreq();
-      }
-    }
-
-    return doc;
-  }
-
-  /** Move iterators to the tail until the cumulated size of lead+tail is
-   *  greater than or equal to minShouldMath */
-  private int doNextCandidate() throws IOException {
-    while (freq + tailSize < minShouldMatch) {
-      // no match on doc is possible, move to the next potential match
-      pushBackLeads();
-      setDocAndFreq();
-    }
-
-    return doc;
-  }
-
-  /** Advance all entries from the tail to know about all matches on the
-   *  current doc. */
-  private void updateFreq() throws IOException {
-    assert freq >= minShouldMatch;
-    // we return the next doc when there are minShouldMatch matching clauses
-    // 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;
-  }
-
-  @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 getMaxScore(int upTo) throws IOException {
-    // TODO: implement but be careful about floating-point errors.
-    return Float.POSITIVE_INFINITY;
-  }
-
-  @Override
-  public int docID() {
-    assert doc == lead.doc;
-    return doc;
-  }
-
-  /** Insert an entry in 'tail' and evict the least-costly scorer if full. */
-  private DisiWrapper insertTailWithOverFlow(DisiWrapper s) {
-    if (tailSize < tail.length) {
-      addTail(s);
-      return null;
-    } else if (tail.length >= 1) {
-      final DisiWrapper top = tail[0];
-      if (top.cost < s.cost) {
-        tail[0] = s;
-        downHeapCost(tail, tailSize);
-        return top;
-      }
-    }
-    return s;
-  }
-
-  /** Add an entry to 'tail'. Fails if over capacity. */
-  private void addTail(DisiWrapper s) {
-    tail[tailSize] = s;
-    upHeapCost(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];
-    downHeapCost(tail, tailSize);
-    return result;
-  }
-
-  /** Heap helpers */
-
-  private static void upHeapCost(DisiWrapper[] heap, int i) {
-    final DisiWrapper node = heap[i];
-    final long nodeCost = node.cost;
-    int j = parentNode(i);
-    while (j >= 0 && nodeCost < heap[j].cost) {
-      heap[i] = heap[j];
-      i = j;
-      j = parentNode(j);
-    }
-    heap[i] = node;
-  }
-
-  private static void downHeapCost(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 && heap[k].cost < heap[j].cost) {
-        j = k;
-      }
-      if (heap[j].cost < node.cost) {
-        do {
-          heap[i] = heap[j];
-          i = j;
-          j = leftNode(i);
-          k = rightNode(j);
-          if (k < size && heap[k].cost < heap[j].cost) {
-            j = k;
-          }
-        } while (j < size && heap[j].cost < node.cost);
-        heap[i] = node;
-      }
-    }
-  }
-
-}
diff --git a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
index 1d9199d..2d31bbf 100644
--- a/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/WANDScorer.java
@@ -28,20 +28,28 @@ 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. Enhanced with
- * techniques described in "Faster Top-k Document Retrieval Using Block-Max
- * Indexes" by Ding and Suel.
- * 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#getMaxScore(int) 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}.
+ * 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.
+ * Enhanced with techniques described in "Faster Top-k Document Retrieval Using Block-Max Indexes"
+ * by Ding and Suel. For scoreMode == {@link ScoreMode#TOP_SCORES}, 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.
+ *
+ * <p>The implementation supports both minCompetitiveScore by enforce that {@code ∑ max_score >=
+ * minCompetitiveScore}, and minShouldMatch by enforcing {@code freq >= minShouldMatch}. It keeps
+ * sub scorers in 3 different places: - tail: a heap that contains scorers that are behind the
+ * desired doc ID. These scorers are ordered by cost so that we can advance the least costly ones
+ * first. - lead: a linked list of scorer that are positioned on the desired doc ID - head: a heap
+ * that contains scorers which are beyond the desired doc ID, ordered by doc ID in order to move
+ * quickly to the next candidate.
+ *
+ * <p>When scoreMode == {@link ScoreMode#TOP_SCORES}, it leverages the {@link
+ * Scorer#getMaxScore(int) 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. When scoreMode != {@link ScoreMode#TOP_SCORES}, block-max scoring related logic
+ * is skipped. Finding the next match consists of first setting the desired doc ID to the least
+ * entry in 'head', and then advance 'tail' until there is a match, by meeting the configured {@code
+ * freq >= minShouldMatch} and / or {@code ∑ max_score >= minCompetitiveScore} requirements.
  */
 final class WANDScorer extends Scorer {
 
@@ -134,7 +142,10 @@ final class WANDScorer extends Scorer {
   final int minShouldMatch;
   int freq;
 
-  WANDScorer(Weight weight, Collection<Scorer> scorers, int minShouldMatch) throws IOException {
+  final ScoreMode scoreMode;
+
+  WANDScorer(Weight weight, Collection<Scorer> scorers, int minShouldMatch, ScoreMode scoreMode)
+      throws IOException {
     super(weight);
 
     if (minShouldMatch >= scorers.size()) {
@@ -149,21 +160,32 @@ final class WANDScorer extends Scorer {
     this.doc = -1;
     this.upTo = -1; // will be computed on the first call to nextDoc/advance
 
+    this.scoreMode = scoreMode;
+
     head = new DisiPriorityQueue(scorers.size());
     // there can be at most num_scorers - 1 scorers beyond the current position
     tail = new DisiWrapper[scorers.size()];
 
-    OptionalInt scalingFactor = OptionalInt.empty();
-    for (Scorer scorer : scorers) {
-      scorer.advanceShallow(0);
-      float maxScore = scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS);
-      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)));
+    if (this.scoreMode == ScoreMode.TOP_SCORES) {
+      OptionalInt scalingFactor = OptionalInt.empty();
+      for (Scorer scorer : scorers) {
+        scorer.advanceShallow(0);
+        float maxScore = scorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS);
+        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);
+      this.maxScorePropagator = new MaxScoreSumPropagator(scorers);
+    } else {
+      this.scalingFactor = 0;
+      this.maxScorePropagator = null;
     }
-    // Use a scaling factor of 0 if all max scores are either 0 or +Infty
-    this.scalingFactor = scalingFactor.orElse(0);
 
     for (Scorer scorer : scorers) {
       addLead(new DisiWrapper(scorer));
@@ -174,33 +196,34 @@ final class WANDScorer extends Scorer {
             scorers.stream().map(Scorer::iterator).mapToLong(DocIdSetIterator::cost),
             scorers.size(),
             minShouldMatch);
-    this.maxScorePropagator = new MaxScoreSumPropagator(scorers);
   }
 
   // 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;
+    if (scoreMode == ScoreMode.TOP_SCORES) {
+      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;
 
-    maxScoreSum = 0;
-    for (DisiWrapper w = lead; w != null; w = w.next) {
-      assert w.doc == doc;
-      maxScoreSum = Math.addExact(maxScoreSum, w.maxScore);
+      assert minCompetitiveScore == 0 || tailMaxScore < minCompetitiveScore;
+      assert doc <= upTo;
     }
-    assert maxScoreSum == leadMaxScore : maxScoreSum + " " + leadMaxScore;
 
     for (DisiWrapper w : head) {
       assert w.doc > doc;
     }
 
-    assert minCompetitiveScore == 0 || tailMaxScore < minCompetitiveScore;
-    assert doc <= upTo;
-
     return true;
   }
 
@@ -208,6 +231,8 @@ final class WANDScorer extends Scorer {
   public void setMinCompetitiveScore(float minScore) throws IOException {
     // Let this disjunction know about the new min score so that it can skip
     // over clauses that produce low scores.
+    assert scoreMode == ScoreMode.TOP_SCORES
+        : "minCompetitiveScore can only be set for ScoreMode.TOP_SCORES, but got: " + scoreMode;
     assert minScore >= 0;
     long scaledMinScore = scaleMinScore(minScore, scalingFactor);
     assert scaledMinScore >= minCompetitiveScore;
@@ -419,22 +444,26 @@ final class WANDScorer extends Scorer {
       }
     }
 
-    assert upTo == DocIdSetIterator.NO_MORE_DOCS || (head.size() > 0 && head.top().doc <= upTo);
+    assert (head.size() == 0 && upTo == DocIdSetIterator.NO_MORE_DOCS)
+        || (head.size() > 0 && head.top().doc <= upTo);
+    assert upTo >= target;
   }
 
   /** Set 'doc' to the next potential match, and move all disis of 'head' that
    *  are on this doc into 'lead'. */
   private void moveToNextCandidate(int target) throws IOException {
-    // Update score bounds if necessary so
-    updateMaxScoresIfNecessary(target);
-    assert upTo >= target;
+    if (scoreMode == ScoreMode.TOP_SCORES) {
+      // Update score bounds if necessary so
+      updateMaxScoresIfNecessary(target);
+      assert upTo >= target;
 
-    // updateMaxScores tries to move forward until a block with matches is found
-    // so if the head is empty it means there are no matches at all anymore
-    if (head.size() == 0) {
-      assert upTo == DocIdSetIterator.NO_MORE_DOCS;
-      doc = DocIdSetIterator.NO_MORE_DOCS;
-      return;
+      // updateMaxScores tries to move forward until a block with matches is found
+      // so if the head is empty it means there are no matches at all anymore
+      if (head.size() == 0) {
+        assert upTo == DocIdSetIterator.NO_MORE_DOCS;
+        doc = DocIdSetIterator.NO_MORE_DOCS;
+        return;
+      }
     }
 
     // The top of `head` defines the next potential match
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
index ba1d8f9..35abbbc 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
@@ -238,7 +238,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
       assertEquals(
           "ConjunctionScorer\n" +
           "    MUST ConstantScoreScorer\n" +
-          "    MUST MinShouldMatchSumScorer\n" +
+          "    MUST WANDScorer\n" +
           "            SHOULD TermScorer body:crawler\n" +
           "            SHOULD TermScorer body:web\n" +
           "            SHOULD TermScorer body:nutch",
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestWANDScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestWANDScorer.java
index d2a3cec..0d0822d 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestWANDScorer.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestWANDScorer.java
@@ -289,6 +289,57 @@ public class TestWANDScorer extends LuceneTestCase {
     }
   }
 
+  public void testBasicsWithDisjunctionAndMinShouldMatchAndNonScoringMode() throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()))) {
+        for (String[] values :
+            Arrays.asList(
+                new String[] {"A", "B"}, // 0
+                new String[] {"A"}, // 1
+                new String[] {}, // 2
+                new String[] {"A", "B", "C"}, // 3
+                new String[] {"B"}, // 4
+                new String[] {"B", "C"} // 5
+                )) {
+          Document doc = new Document();
+          for (String value : values) {
+            doc.add(new StringField("foo", value, Store.NO));
+          }
+          w.addDocument(doc);
+        }
+
+        w.forceMerge(1);
+      }
+
+      try (IndexReader reader = DirectoryReader.open(dir)) {
+        IndexSearcher searcher = newSearcher(reader);
+
+        Query query =
+            new BooleanQuery.Builder()
+                .add(
+                    new BoostQuery(new ConstantScoreQuery(new TermQuery(new Term("foo", "A"))), 2),
+                    Occur.SHOULD)
+                .add(new ConstantScoreQuery(new TermQuery(new Term("foo", "B"))), Occur.SHOULD)
+                .add(
+                    new BoostQuery(new ConstantScoreQuery(new TermQuery(new Term("foo", "C"))), 3),
+                    Occur.SHOULD)
+                .setMinimumNumberShouldMatch(2)
+                .build();
+
+        Scorer scorer =
+            searcher
+                .createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1)
+                .scorer(searcher.getIndexReader().leaves().get(0));
+
+        assertEquals(0, scorer.iterator().nextDoc());
+        assertEquals(3, scorer.iterator().nextDoc());
+        assertEquals(5, scorer.iterator().nextDoc());
+        assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+      }
+    }
+  }
+
   public void testBasicsWithFilteredDisjunctionAndMinShouldMatch() throws Exception {
     try (Directory dir = newDirectory()) {
       try (IndexWriter w =
@@ -364,6 +415,66 @@ public class TestWANDScorer extends LuceneTestCase {
     }
   }
 
+  public void testBasicsWithFilteredDisjunctionAndMinShouldMatchAndNonScoringMode()
+      throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()))) {
+        for (String[] values :
+            Arrays.asList(
+                new String[] {"A", "B"}, // 0
+                new String[] {"A", "C", "D"}, // 1
+                new String[] {}, // 2
+                new String[] {"A", "B", "C", "D"}, // 3
+                new String[] {"B"}, // 4
+                new String[] {"C", "D"} // 5
+                )) {
+          Document doc = new Document();
+          for (String value : values) {
+            doc.add(new StringField("foo", value, Store.NO));
+          }
+          w.addDocument(doc);
+        }
+
+        w.forceMerge(1);
+      }
+
+      try (IndexReader reader = DirectoryReader.open(dir)) {
+        IndexSearcher searcher = newSearcher(reader);
+
+        Query query =
+            new BooleanQuery.Builder()
+                .add(
+                    new BooleanQuery.Builder()
+                        .add(
+                            new BoostQuery(
+                                new ConstantScoreQuery(new TermQuery(new Term("foo", "A"))), 2),
+                            Occur.SHOULD)
+                        .add(
+                            new ConstantScoreQuery(new TermQuery(new Term("foo", "B"))),
+                            Occur.SHOULD)
+                        .add(
+                            new BoostQuery(
+                                new ConstantScoreQuery(new TermQuery(new Term("foo", "D"))), 4),
+                            Occur.SHOULD)
+                        .setMinimumNumberShouldMatch(2)
+                        .build(),
+                    Occur.MUST)
+                .add(new TermQuery(new Term("foo", "C")), Occur.FILTER)
+                .build();
+
+        Scorer scorer =
+            searcher
+                .createWeight(searcher.rewrite(query), ScoreMode.TOP_DOCS, 1)
+                .scorer(searcher.getIndexReader().leaves().get(0));
+
+        assertEquals(1, scorer.iterator().nextDoc());
+        assertEquals(3, scorer.iterator().nextDoc());
+        assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+      }
+    }
+  }
+
   public void testBasicsWithFilteredDisjunctionAndMustNotAndMinShouldMatch() throws Exception {
     try (Directory dir = newDirectory()) {
       try (IndexWriter w =
@@ -431,6 +542,58 @@ public class TestWANDScorer extends LuceneTestCase {
     }
   }
 
+  public void testBasicsWithFilteredDisjunctionAndMustNotAndMinShouldMatchAndNonScoringMode()
+      throws Exception {
+    try (Directory dir = newDirectory()) {
+      try (IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()))) {
+        for (String[] values :
+            Arrays.asList(
+                new String[] {"A", "B"}, // 0
+                new String[] {"A", "C", "D"}, // 1
+                new String[] {}, // 2
+                new String[] {"A", "B", "C", "D"}, // 3
+                new String[] {"B", "D"}, // 4
+                new String[] {"C", "D"} // 5
+                )) {
+          Document doc = new Document();
+          for (String value : values) {
+            doc.add(new StringField("foo", value, Store.NO));
+          }
+          w.addDocument(doc);
+        }
+
+        w.forceMerge(1);
+      }
+
+      try (IndexReader reader = DirectoryReader.open(dir)) {
+        IndexSearcher searcher = newSearcher(reader);
+
+        Query query =
+            new BooleanQuery.Builder()
+                .add(
+                    new BoostQuery(new ConstantScoreQuery(new TermQuery(new Term("foo", "A"))), 2),
+                    Occur.SHOULD)
+                .add(new ConstantScoreQuery(new TermQuery(new Term("foo", "B"))), Occur.SHOULD)
+                .add(new TermQuery(new Term("foo", "C")), Occur.MUST_NOT)
+                .add(
+                    new BoostQuery(new ConstantScoreQuery(new TermQuery(new Term("foo", "D"))), 4),
+                    Occur.SHOULD)
+                .setMinimumNumberShouldMatch(2)
+                .build();
+
+        Scorer scorer =
+            searcher
+                .createWeight(searcher.rewrite(query), ScoreMode.COMPLETE_NO_SCORES, 1)
+                .scorer(searcher.getIndexReader().leaves().get(0));
+
+        assertEquals(0, scorer.iterator().nextDoc());
+        assertEquals(4, scorer.iterator().nextDoc());
+        assertEquals(DocIdSetIterator.NO_MORE_DOCS, scorer.iterator().nextDoc());
+      }
+    }
+  }
+
   public void testRandom() throws IOException {
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());