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 2015/02/04 16:42:07 UTC

svn commit: r1657290 - in /lucene/dev/branches/branch_5x: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/util/ lucene/core/src/test/org/apache/lucene/search/ lucene/facet/ lucene/facet/src/...

Author: jpountz
Date: Wed Feb  4 15:42:06 2015
New Revision: 1657290

URL: http://svn.apache.org/r1657290
Log:
LUCENE-6201: Improvements to minShouldMatch handling.

Added:
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java
      - copied unchanged from r1657286, lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScorerPriorityQueue.java
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/BulkScorerWrapperScorer.java
      - copied unchanged from r1657286, lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/BulkScorerWrapperScorer.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/lucene/   (props changed)
    lucene/dev/branches/branch_5x/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/lucene/core/   (props changed)
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java
    lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
    lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
    lucene/dev/branches/branch_5x/lucene/facet/   (props changed)
    lucene/dev/branches/branch_5x/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
    lucene/dev/branches/branch_5x/lucene/test-framework/   (props changed)
    lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java

Modified: lucene/dev/branches/branch_5x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/CHANGES.txt?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/lucene/CHANGES.txt Wed Feb  4 15:42:06 2015
@@ -25,6 +25,10 @@ Optimizations
   instead of a separate merge sort for each segment.  In delete-heavy
   use cases this can be a sizable speedup. (Mike McCandless)
 
+* LUCENE-6201: BooleanScorer can now deal with values of minShouldMatch that
+  are greater than one and is used when queries produce dense result sets.
+  (Adrien Grand)
+
 API Changes
 
 * LUCENE-6204, LUCENE-6208: Simplify CompoundFormat: remove files()

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Wed Feb  4 15:42:06 2015
@@ -304,15 +304,10 @@ public class BooleanQuery extends Query
       }
     }
 
-    @Override
-    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-
-      if (minNrShouldMatch > 1) {
-        // TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
-        // but the same is even true of pure conjunctions...
-        return super.bulkScorer(context, acceptDocs);
-      }
-
+    /** Try to build a boolean scorer for this weight. Returns null if {@link BooleanScorer}
+     *  cannot be used. */
+    // pkg-private for forcing use of BooleanScorer in tests
+    BooleanScorer booleanScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
       List<BulkScorer> optional = new ArrayList<BulkScorer>();
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
@@ -326,10 +321,10 @@ public class BooleanQuery extends Query
           // TODO: there are some cases where BooleanScorer
           // would handle conjunctions faster than
           // BooleanScorer2...
-          return super.bulkScorer(context, acceptDocs);
+          return null;
         } else if (c.isProhibited()) {
           // TODO: there are some cases where BooleanScorer could do this faster
-          return super.bulkScorer(context, acceptDocs);
+          return null;
         } else {
           optional.add(subScorer);
         }
@@ -339,7 +334,39 @@ public class BooleanQuery extends Query
         return null;
       }
 
-      return new BooleanScorer(this, disableCoord, maxCoord, optional);
+      if (minNrShouldMatch > optional.size()) {
+        return null;
+      }
+
+      return new BooleanScorer(this, disableCoord, maxCoord, optional, Math.max(1, minNrShouldMatch));
+    }
+
+    @Override
+    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs);
+      if (bulkScorer != null) { // BooleanScorer is applicable
+        // TODO: what is the right heuristic here?
+        final long costThreshold;
+        if (minNrShouldMatch <= 1) {
+          // when all clauses are optional, use BooleanScorer aggressively
+          // TODO: is there actually a threshold under which we should rather
+          // use the regular scorer?
+          costThreshold = -1;
+        } else {
+          // when a minimum number of clauses should match, BooleanScorer is
+          // going to score all windows that have at least minNrShouldMatch
+          // matches in the window. But there is no way to know if there is
+          // an intersection (all clauses might match a different doc ID and
+          // there will be no matches in the end) so we should only use
+          // BooleanScorer if matches are very dense
+          costThreshold = context.reader().maxDoc() / 3;
+        }
+
+        if (bulkScorer.cost() > costThreshold) {
+          return bulkScorer;
+        }
+      }
+      return super.bulkScorer(context, acceptDocs);
     }
 
     @Override

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Wed Feb  4 15:42:06 2015
@@ -41,14 +41,75 @@ final class BooleanScorer extends BulkSc
     int freq;
   }
 
-  static class BulkScorerAndDoc {
+  private class BulkScorerAndDoc {
     final BulkScorer scorer;
+    final long cost;
     int next;
 
     BulkScorerAndDoc(BulkScorer scorer) {
       this.scorer = scorer;
-      this.next = 0;
+      this.cost = scorer.cost();
+      this.next = -1;
     }
+
+    void advance(int min) throws IOException {
+      score(min, min);
+    }
+
+    void score(int min, int max) throws IOException {
+      next = scorer.score(orCollector, min, max);
+    }
+  }
+
+  // See MinShouldMatchSumScorer for an explanation
+  private static long cost(Collection<BulkScorer> scorers, int minShouldMatch) {
+    final PriorityQueue<BulkScorer> pq = new PriorityQueue<BulkScorer>(scorers.size() - minShouldMatch + 1) {
+      @Override
+      protected boolean lessThan(BulkScorer a, BulkScorer b) {
+        return a.cost() > b.cost();
+      }
+    };
+    for (BulkScorer scorer : scorers) {
+      pq.insertWithOverflow(scorer);
+    }
+    long cost = 0;
+    for (BulkScorer scorer = pq.pop(); scorer != null; scorer = pq.pop()) {
+      cost += scorer.cost();
+    }
+    return cost;
+  }
+
+  static final class HeadPriorityQueue extends PriorityQueue<BulkScorerAndDoc> {
+
+    public HeadPriorityQueue(int maxSize) {
+      super(maxSize);
+    }
+
+    @Override
+    protected boolean lessThan(BulkScorerAndDoc a, BulkScorerAndDoc b) {
+      return a.next < b.next;
+    }
+
+  }
+
+  static final class TailPriorityQueue extends PriorityQueue<BulkScorerAndDoc> {
+
+    public TailPriorityQueue(int maxSize) {
+      super(maxSize);
+    }
+
+    @Override
+    protected boolean lessThan(BulkScorerAndDoc a, BulkScorerAndDoc b) {
+      return a.cost < b.cost;
+    }
+
+    public BulkScorerAndDoc get(int i) {
+      if (i < 0 || i >= size()) {
+        throw new IndexOutOfBoundsException();
+      }
+      return (BulkScorerAndDoc) getHeapArray()[1 + i];
+    }
+
   }
 
   final Bucket[] buckets = new Bucket[SIZE];
@@ -56,8 +117,12 @@ final class BooleanScorer extends BulkSc
   final long[] matching = new long[SET_SIZE];
 
   final float[] coordFactors;
-  final PriorityQueue<BulkScorerAndDoc> optionalScorers;
+  final BulkScorerAndDoc[] leads;
+  final HeadPriorityQueue head;
+  final TailPriorityQueue tail;
   final FakeScorer fakeScorer = new FakeScorer();
+  final int minShouldMatch;
+  final long cost;
 
   final class OrCollector implements LeafCollector {
     Scorer scorer;
@@ -80,34 +145,46 @@ final class BooleanScorer extends BulkSc
 
   final OrCollector orCollector = new OrCollector();
 
-  BooleanScorer(BooleanWeight weight, boolean disableCoord, int maxCoord, Collection<BulkScorer> optionalScorers) {
+  BooleanScorer(BooleanWeight weight, boolean disableCoord, int maxCoord, Collection<BulkScorer> scorers, int minShouldMatch) {
+    if (minShouldMatch < 1 || minShouldMatch > scorers.size()) {
+      throw new IllegalArgumentException("minShouldMatch should be within 1..num_scorers. Got " + minShouldMatch);
+    }
     for (int i = 0; i < buckets.length; i++) {
       buckets[i] = new Bucket();
     }
-    this.optionalScorers = new PriorityQueue<BulkScorerAndDoc>(optionalScorers.size()) {
-      @Override
-      protected boolean lessThan(BulkScorerAndDoc a, BulkScorerAndDoc b) {
-        return a.next < b.next;
+    this.leads = new BulkScorerAndDoc[scorers.size()];
+    this.head = new HeadPriorityQueue(scorers.size() - minShouldMatch + 1);
+    this.tail = new TailPriorityQueue(minShouldMatch - 1);
+    this.minShouldMatch = minShouldMatch;
+    for (BulkScorer scorer : scorers) {
+      final BulkScorerAndDoc evicted = tail.insertWithOverflow(new BulkScorerAndDoc(scorer));
+      if (evicted != null) {
+        head.add(evicted);
       }
-    };
-    for (BulkScorer scorer : optionalScorers) {
-      this.optionalScorers.add(new BulkScorerAndDoc(scorer));
     }
+    this.cost = cost(scorers, minShouldMatch);
 
-    coordFactors = new float[optionalScorers.size() + 1];
+    coordFactors = new float[scorers.size() + 1];
     for (int i = 0; i < coordFactors.length; i++) {
       coordFactors[i] = disableCoord ? 1.0f : weight.coord(i, maxCoord);
     }
   }
 
+  @Override
+  public long cost() {
+    return cost;
+  }
+
   private void scoreDocument(LeafCollector collector, int base, int i) throws IOException {
     final FakeScorer fakeScorer = this.fakeScorer;
     final Bucket bucket = buckets[i];
-    fakeScorer.freq = bucket.freq;
-    fakeScorer.score = (float) bucket.score * coordFactors[bucket.freq];
-    final int doc = base | i;
-    fakeScorer.doc = doc;
-    collector.collect(doc);
+    if (bucket.freq >= minShouldMatch) {
+      fakeScorer.freq = bucket.freq;
+      fakeScorer.score = (float) bucket.score * coordFactors[bucket.freq];
+      final int doc = base | i;
+      fakeScorer.doc = doc;
+      collector.collect(doc);
+    }
     bucket.freq = 0;
     bucket.score = 0;
   }
@@ -125,32 +202,95 @@ final class BooleanScorer extends BulkSc
     }
   }
 
-  private BulkScorerAndDoc scoreWindow(LeafCollector collector, int base, int min, int max,
-      PriorityQueue<BulkScorerAndDoc> optionalScorers, BulkScorerAndDoc top) throws IOException {
-    assert top.next < max;
-    do {
-      top.next = top.scorer.score(orCollector, min, max);
-      top = optionalScorers.updateTop();
-    } while (top.next < max);
+  private void scoreWindow(LeafCollector collector, int base, int min, int max,
+      BulkScorerAndDoc[] scorers, int numScorers) throws IOException {
+    for (int i = 0; i < numScorers; ++i) {
+      final BulkScorerAndDoc scorer = scorers[i];
+      assert scorer.next < max;
+      scorer.score(min, max);
+    }
 
     scoreMatches(collector, base);
     Arrays.fill(matching, 0L);
-    return top;
+  }
+
+  private BulkScorerAndDoc advance(int min) throws IOException {
+    assert tail.size() == minShouldMatch - 1;
+    final HeadPriorityQueue head = this.head;
+    final TailPriorityQueue tail = this.tail;
+    BulkScorerAndDoc headTop = head.top();
+    BulkScorerAndDoc tailTop = tail.top();
+    while (headTop.next < min) {
+      if (tailTop == null || headTop.cost <= tailTop.cost) {
+        headTop.advance(min);
+        headTop = head.updateTop();
+      } else {
+        // swap the top of head and tail
+        final BulkScorerAndDoc previousHeadTop = headTop;
+        tailTop.advance(min);
+        headTop = head.updateTop(tailTop);
+        tailTop = tail.updateTop(previousHeadTop);
+      }
+    }
+    return headTop;
+  }
+
+  private void scoreWindow(LeafCollector collector, int windowBase, int windowMin, int windowMax) throws IOException {
+    // Fill 'leads' with all scorers from 'head' that are in the right window
+    leads[0] = head.pop();
+    int maxFreq = 1;
+    while (head.size() > 0 && head.top().next < windowMax) {
+      leads[maxFreq++] = head.pop();
+    }
+
+    while (maxFreq < minShouldMatch && maxFreq + tail.size() >= minShouldMatch) {
+      // a match is still possible
+      final BulkScorerAndDoc candidate = tail.pop();
+      candidate.advance(windowMin);
+      if (candidate.next < windowMax) {
+        leads[maxFreq++] = candidate;
+      } else {
+        head.add(candidate);
+      }
+    }
+
+    if (maxFreq >= minShouldMatch) {
+      // There might be matches in other scorers from the tail too
+      for (int i = 0; i < tail.size(); ++i) {
+        leads[maxFreq++] = tail.get(i);
+      }
+      tail.clear();
+
+      scoreWindow(collector, windowBase, windowMin, windowMax, leads, maxFreq);
+    }
+
+    // Push back scorers into head and tail
+    for (int i = 0; i < maxFreq; ++i) {
+      final BulkScorerAndDoc evicted = head.insertWithOverflow(leads[i]);
+      if (evicted != null) {
+        tail.add(evicted);
+      }
+    }
   }
 
   @Override
   public int score(LeafCollector collector, int min, int max) throws IOException {
     fakeScorer.doc = -1;
     collector.setScorer(fakeScorer);
-    final PriorityQueue<BulkScorerAndDoc> optionalScorers = this.optionalScorers;
 
-    BulkScorerAndDoc top = optionalScorers.top();
-    for (int windowMin = Math.max(min, top.next); windowMin < max; windowMin = top.next) {
-      final int windowBase = windowMin & ~MASK; // find the window that windowMin belongs to
+    BulkScorerAndDoc top = advance(min);
+    while (top.next < max) {
+
+      final int windowBase = top.next & ~MASK; // find the window that the next match belongs to
+      final int windowMin = Math.max(min, windowBase);
       final int windowMax = Math.min(max, windowBase + SIZE);
-      top = scoreWindow(collector, windowBase, windowMin, windowMax, optionalScorers, top);
-      assert top.next >= windowMax;
+
+      // general case
+      scoreWindow(collector, windowBase, windowMin, windowMax);
+      top = head.top();
     }
+
     return top.next;
   }
+
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java Wed Feb  4 15:42:06 2015
@@ -74,4 +74,9 @@ public abstract class BulkScorer {
    * @return an under-estimation of the next matching doc after max
    */
   public abstract int score(LeafCollector collector, int min, int max) throws IOException;
+
+  /**
+   * Same as {@link Scorer#cost()} for bulk scorers.
+   */
+  public abstract long cost();
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Wed Feb  4 15:42:06 2015
@@ -220,6 +220,11 @@ public class ConstantScoreQuery extends
         }
       };
     }
+
+    @Override
+    public long cost() {
+      return bulkScorer.cost();
+    }
   }
 
   protected class ConstantScorer extends Scorer {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxScorer.java Wed Feb  4 15:42:06 2015
@@ -18,6 +18,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
+
 /**
  * The Scorer for DisjunctionMaxQuery.  The union of all documents generated by the the subquery scorers
  * is generated in document number order.  The score for each document is the maximum of the scores computed
@@ -28,10 +30,6 @@ final class DisjunctionMaxScorer extends
   /* Multiplier applied to non-maximum-scoring subqueries for a document as they are summed into the result. */
   private final float tieBreakerMultiplier;
 
-  /* Used when scoring currently matching doc. */
-  private float scoreSum;
-  private float scoreMax;
-
   /**
    * Creates a new instance of DisjunctionMaxScorer
    * 
@@ -47,23 +45,18 @@ final class DisjunctionMaxScorer extends
     super(weight, subScorers);
     this.tieBreakerMultiplier = tieBreakerMultiplier;
   }
-  
-  @Override
-  protected void reset() {
-    scoreSum = scoreMax = 0;
-  }
-  
+
   @Override
-  protected void accum(Scorer subScorer) throws IOException {
-    float subScore = subScorer.score();
-    scoreSum += subScore;
-    if (subScore > scoreMax) {
-      scoreMax = subScore;
+  protected float score(ScorerWrapper topList, int freq) throws IOException {
+    float scoreSum = 0;
+    float scoreMax = 0;
+    for (ScorerWrapper w = topList; w != null; w = w.next) {
+      final float subScore = w.scorer.score();
+      scoreSum += subScore;
+      if (subScore > scoreMax) {
+        scoreMax = subScore;
+      }
     }
-  }
-  
-  @Override
-  protected float getFinal() {
     return scoreMax + (scoreSum - scoreMax) * tieBreakerMultiplier; 
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Wed Feb  4 15:42:06 2015
@@ -21,95 +21,37 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
+
 /**
  * Base class for Scorers that score disjunctions.
  */
 abstract class DisjunctionScorer extends Scorer {
-  private final Scorer subScorers[];
-  private int numScorers;
+  private final ScorerPriorityQueue subScorers;
 
   /** The document number of the current match. */
   protected int doc = -1;
   /** Number of matching scorers for the current match. */
-  protected int freq = -1;
-  
+  private int freq = -1;
+  /** Linked list of scorers which are on the current doc */
+  private ScorerWrapper topScorers;
+
   protected DisjunctionScorer(Weight weight, Scorer subScorers[]) {
     super(weight);
-    this.subScorers = subScorers;
-    this.numScorers = subScorers.length;
-    if (numScorers <= 1) {
+    if (subScorers.length <= 1) {
       throw new IllegalArgumentException("There must be at least 2 subScorers");
     }
-    heapify();
-  }
-  
-  /** 
-   * Organize subScorers into a min heap with scorers generating the earliest document on top.
-   */
-  private void heapify() {
-    for (int i = (numScorers >>> 1) - 1; i >= 0; i--) {
-      heapAdjust(i);
-    }
-  }
-  
-  /** 
-   * The subtree of subScorers at root is a min heap except possibly for its root element.
-   * Bubble the root down as required to make the subtree a heap.
-   */
-  private void heapAdjust(int root) {
-    Scorer scorer = subScorers[root];
-    int doc = scorer.docID();
-    int i = root;
-    while (i <= (numScorers >>> 1) - 1) {
-      int lchild = (i << 1) + 1;
-      Scorer lscorer = subScorers[lchild];
-      int ldoc = lscorer.docID();
-      int rdoc = Integer.MAX_VALUE, rchild = (i << 1) + 2;
-      Scorer rscorer = null;
-      if (rchild < numScorers) {
-        rscorer = subScorers[rchild];
-        rdoc = rscorer.docID();
-      }
-      if (ldoc < doc) {
-        if (rdoc < ldoc) {
-          subScorers[i] = rscorer;
-          subScorers[rchild] = scorer;
-          i = rchild;
-        } else {
-          subScorers[i] = lscorer;
-          subScorers[lchild] = scorer;
-          i = lchild;
-        }
-      } else if (rdoc < doc) {
-        subScorers[i] = rscorer;
-        subScorers[rchild] = scorer;
-        i = rchild;
-      } else {
-        return;
-      }
-    }
-  }
-
-  /** 
-   * Remove the root Scorer from subScorers and re-establish it as a heap
-   */
-  private void heapRemoveRoot() {
-    if (numScorers == 1) {
-      subScorers[0] = null;
-      numScorers = 0;
-    } else {
-      subScorers[0] = subScorers[numScorers - 1];
-      subScorers[numScorers - 1] = null;
-      --numScorers;
-      heapAdjust(0);
+    this.subScorers = new ScorerPriorityQueue(subScorers.length);
+    for (Scorer scorer : subScorers) {
+      this.subScorers.add(new ScorerWrapper(scorer));
     }
   }
   
   @Override
   public final Collection<ChildScorer> getChildren() {
-    ArrayList<ChildScorer> children = new ArrayList<>(numScorers);
-    for (int i = 0; i < numScorers; i++) {
-      children.add(new ChildScorer(subScorers[i], "SHOULD"));
+    ArrayList<ChildScorer> children = new ArrayList<>();
+    for (ScorerWrapper scorer : subScorers) {
+      children.add(new ChildScorer(scorer.scorer, "SHOULD"));
     }
     return children;
   }
@@ -117,8 +59,8 @@ abstract class DisjunctionScorer extends
   @Override
   public final long cost() {
     long sum = 0;
-    for (int i = 0; i < numScorers; i++) {
-      sum += subScorers[i].cost();
+    for (ScorerWrapper scorer : subScorers) {
+      sum += scorer.cost;
     }
     return sum;
   } 
@@ -131,82 +73,68 @@ abstract class DisjunctionScorer extends
   @Override
   public final int nextDoc() throws IOException {
     assert doc != NO_MORE_DOCS;
-    while(true) {
-      if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
-        heapAdjust(0);
-      } else {
-        heapRemoveRoot();
-        if (numScorers == 0) {
-          return doc = NO_MORE_DOCS;
+
+    ScorerWrapper top = subScorers.top();
+    final int doc = this.doc;
+    while (top.doc == doc) {
+      top.doc = top.scorer.nextDoc();
+      if (top.doc == NO_MORE_DOCS) {
+        subScorers.pop();
+        if (subScorers.size() == 0) {
+          return this.doc = NO_MORE_DOCS;
         }
-      }
-      int docID = subScorers[0].docID();
-      if (docID != doc) {
-        freq = -1;
-        return doc = docID;
+        top = subScorers.top();
+      } else {
+        top = subScorers.updateTop();
       }
     }
+
+    freq = -1;
+    return this.doc = top.doc;
   }
   
   @Override
   public final int advance(int target) throws IOException {
     assert doc != NO_MORE_DOCS;
-    while(true) {
-      if (subScorers[0].advance(target) != NO_MORE_DOCS) {
-        heapAdjust(0);
-      } else {
-        heapRemoveRoot();
-        if (numScorers == 0) {
-          return doc = NO_MORE_DOCS;
+
+    ScorerWrapper top = subScorers.top();
+    while (top.doc < target) {
+      top.doc = top.scorer.advance(target);
+      if (top.doc == NO_MORE_DOCS) {
+        subScorers.pop();
+        if (subScorers.size() == 0) {
+          return this.doc = NO_MORE_DOCS;
         }
-      }
-      int docID = subScorers[0].docID();
-      if (docID >= target) {
-        freq = -1;
-        return doc = docID;
+        top = subScorers.top();
+      } else {
+        top = subScorers.updateTop();
       }
     }
-  }
-  
-  // if we haven't already computed freq + score, do so
-  private void visitScorers() throws IOException {
-    reset();
-    freq = 1;
-    accum(subScorers[0]);
-    visit(1);
-    visit(2);
-  }
-  
-  // TODO: remove recursion.
-  private void visit(int root) throws IOException {
-    if (root < numScorers && subScorers[root].docID() == doc) {
-      freq++;
-      accum(subScorers[root]);
-      visit((root<<1)+1);
-      visit((root<<1)+2);
-    }
-  }
-  
-  @Override
-  public final float score() throws IOException {
-    visitScorers();
-    return getFinal();
+
+    freq = -1;
+    return this.doc = top.doc;
   }
 
   @Override
   public final int freq() throws IOException {
     if (freq < 0) {
-      visitScorers();
+      topScorers = subScorers.topList();
+      int freq = 1;
+      for (ScorerWrapper w = topScorers.next; w != null; w = w.next) {
+        freq += 1;
+      }
+      this.freq = freq;
     }
     return freq;
   }
-  
-  /** Reset score state for a new match */
-  protected abstract void reset();
-  
-  /** Factor in sub-scorer match */
-  protected abstract void accum(Scorer subScorer) throws IOException;
-  
-  /** Return final score */
-  protected abstract float getFinal();
+
+  @Override
+  public final float score() throws IOException {
+    final int freq = freq(); // compute the top scorers if necessary
+    return score(topScorers, freq);
+  }
+
+  /** Compute the score for the given linked list of scorers. */
+  protected abstract float score(ScorerWrapper topList, int freq) throws IOException;
+
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionSumScorer.java Wed Feb  4 15:42:06 2015
@@ -19,11 +19,12 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
+
 /** A Scorer for OR like queries, counterpart of <code>ConjunctionScorer</code>.
  * This Scorer implements {@link Scorer#advance(int)} and uses advance() on the given Scorers. 
  */
 final class DisjunctionSumScorer extends DisjunctionScorer { 
-  private double score;
   private final float[] coord;
   
   /** Construct a <code>DisjunctionScorer</code>.
@@ -35,19 +36,13 @@ final class DisjunctionSumScorer extends
     super(weight, subScorers);
     this.coord = coord;
   }
-  
-  @Override
-  protected void reset() {
-    score = 0;
-  }
-  
-  @Override
-  protected void accum(Scorer subScorer) throws IOException {
-    score += subScorer.score();
-  }
-  
+
   @Override
-  protected float getFinal() {
-    return (float)score * coord[freq]; 
+  protected float score(ScorerWrapper topList, int freq) throws IOException {
+    double score = 0;
+    for (ScorerWrapper w = topList; w != null; w = w.next) {
+      score += w.scorer.score();
+    }
+    return (float)score * coord[freq];
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Wed Feb  4 15:42:06 2015
@@ -220,6 +220,11 @@ public class FilteredQuery extends Query
     }
 
     @Override
+    public long cost() {
+      return scorer.cost();
+    }
+
+    @Override
     public int score(LeafCollector collector, int min, int maxDoc) throws IOException {
       // the normalization trick already applies the boost of this query,
       // so we can use the wrapped scorer directly:

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java Wed Feb  4 15:42:06 2015
@@ -17,384 +17,345 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import static org.apache.lucene.search.ScorerPriorityQueue.leftNode;
+import static org.apache.lucene.search.ScorerPriorityQueue.parentNode;
+import static org.apache.lucene.search.ScorerPriorityQueue.rightNode;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Comparator;
+import java.util.Collections;
 import java.util.List;
 
-import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
+import org.apache.lucene.util.PriorityQueue;
 
 /**
- * A Scorer for OR like queries, counterpart of <code>ConjunctionScorer</code>.
- * This Scorer implements {@link Scorer#advance(int)} and uses advance() on the given Scorers.
- * 
- * This implementation uses the minimumMatch constraint actively to efficiently
- * prune the number of candidates, it is hence a mixture between a pure DisjunctionScorer
- * and a ConjunctionScorer.
+ * A {@link Scorer} for {@link BooleanQuery} when
+ * {@link BooleanQuery#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.
  */
-class MinShouldMatchSumScorer extends Scorer {
+final class MinShouldMatchSumScorer extends Scorer {
+
+  private static long cost(Collection<Scorer> scorers, int minShouldMatch) {
+    // the idea here is the following: a boolean query c1,c2,...cn with minShouldMatch=m
+    // could be rewritten to:
+    // (c1 AND (c2..cn|msm=m-1)) OR (!c1 AND (c2..cn|msm=m))
+    // if we assume that clauses come in ascending cost, then
+    // the cost of the first part is the cost of c1 (because the cost of a conjunction is
+    // the cost of the least costly clause)
+    // the cost of the second part is the cost of finding m matches among the c2...cn
+    // remaining clauses
+    // since it is a disjunction overall, the total cost is the sum of the costs of these
+    // two parts
+
+    // If we recurse infinitely, we find out that the cost of a msm query is the sum of the
+    // costs of the num_scorers - minShouldMatch + 1 least costly scorers
+    final PriorityQueue<Scorer> pq = new PriorityQueue<Scorer>(scorers.size() - minShouldMatch + 1) {
+      @Override
+      protected boolean lessThan(Scorer a, Scorer b) {
+        return a.cost() > b.cost();
+      }
+    };
+    for (Scorer scorer : scorers) {
+      pq.insertWithOverflow(scorer);
+    }
+    long cost = 0;
+    for (Scorer scorer = pq.pop(); scorer != null; scorer = pq.pop()) {
+      cost += scorer.cost();
+    }
+    return cost;
+  }
+
+  final int minShouldMatch;
+  final float[] coord;
+
+  // list of scorers which 'lead' the iteration and are currently
+  // positioned on 'doc'
+  ScorerWrapper 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 ScorerPriorityQueue head;
+
+  // priority queue of scorers which are behind the current doc.
+  // Ordered by cost.
+  final ScorerWrapper[] tail;
+  int tailSize;
 
-  /** The overall number of non-finalized scorers */
-  private int numScorers;
-  /** The minimum number of scorers that should match */
-  private final int mm;
-
-  /** A static array of all subscorers sorted by decreasing cost */
-  private final Scorer sortedSubScorers[];
-  /** A monotonically increasing index into the array pointing to the next subscorer that is to be excluded */
-  private int sortedSubScorersIdx = 0;
-
-  private final Scorer subScorers[]; // the first numScorers-(mm-1) entries are valid
-  private int nrInHeap; // 0..(numScorers-(mm-1)-1)
-
-  /** mmStack is supposed to contain the most costly subScorers that still did
-   *  not run out of docs, sorted by increasing sparsity of docs returned by that subScorer.
-   *  For now, the cost of subscorers is assumed to be inversely correlated with sparsity.
-   */
-  private final Scorer mmStack[]; // of size mm-1: 0..mm-2, always full
-
-  /** The document number of the current match. */
-  private int doc = -1;
-  /** The number of subscorers that provide the current match. */
-  protected int nrMatchers = -1;
-  private double score = Float.NaN;
-  
-  private final float coord[];
-
-  /**
-   * Construct a <code>MinShouldMatchSumScorer</code>.
-   * 
-   * @param weight The weight to be used.
-   * @param subScorers A collection of at least two subscorers.
-   * @param minimumNrMatchers The positive minimum number of subscorers that should
-   * match to match this query.
-   * <br>When <code>minimumNrMatchers</code> is bigger than
-   * the number of <code>subScorers</code>, no matches will be produced.
-   * <br>When minimumNrMatchers equals the number of subScorers,
-   * it is more efficient to use <code>ConjunctionScorer</code>.
-   */
-  public MinShouldMatchSumScorer(Weight weight, List<Scorer> subScorers, int minimumNrMatchers, float coord[]) throws IOException {
+  final Collection<ChildScorer> childScorers;
+  final long cost;
+
+  MinShouldMatchSumScorer(Weight weight, Collection<Scorer> scorers, int minShouldMatch, float[] coord) {
     super(weight);
-    this.nrInHeap = this.numScorers = subScorers.size();
 
-    if (minimumNrMatchers <= 0) {
-      throw new IllegalArgumentException("Minimum nr of matchers must be positive");
+    if (minShouldMatch > scorers.size()) {
+      throw new IllegalArgumentException("minShouldMatch should be <= the number of scorers");
     }
-    if (numScorers <= 1) {
-      throw new IllegalArgumentException("There must be at least 2 subScorers");
+    if (minShouldMatch < 1) {
+      throw new IllegalArgumentException("minShouldMatch should be >= 1");
     }
 
-    this.mm = minimumNrMatchers;
-    this.sortedSubScorers = subScorers.toArray(new Scorer[this.numScorers]);
-    // sorting by decreasing subscorer cost should be inversely correlated with
-    // next docid (assuming costs are due to generating many postings)
-    ArrayUtil.timSort(sortedSubScorers, new Comparator<Scorer>() {
-      @Override
-      public int compare(Scorer o1, Scorer o2) {
-        return Long.signum(o2.cost() - o1.cost());
-      }
-    });
-    // take mm-1 most costly subscorers aside
-    this.mmStack = new Scorer[mm-1];
-    for (int i = 0; i < mm-1; i++) {
-      mmStack[i] = sortedSubScorers[i];
-    }
-    nrInHeap -= mm-1;
-    this.sortedSubScorersIdx = mm-1;
-    // take remaining into heap, if any, and heapify
-    this.subScorers = new Scorer[nrInHeap];
-    for (int i = 0; i < nrInHeap; i++) {
-      this.subScorers[i] = this.sortedSubScorers[mm-1+i];
-    }
+    this.minShouldMatch = minShouldMatch;
     this.coord = coord;
-    minheapHeapify();
-    assert minheapCheck();
+    this.doc = -1;
+
+    head = new ScorerPriorityQueue(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 ScorerWrapper[minShouldMatch - 1];
+
+    for (Scorer scorer : scorers) {
+      addLead(new ScorerWrapper(scorer));
+    }
+
+    List<ChildScorer> children = new ArrayList<>();
+    for (Scorer scorer : scorers) {
+      children.add(new ChildScorer(scorer, "SHOULD"));
+    }
+    this.childScorers = Collections.unmodifiableCollection(children);
+    this.cost = cost(scorers, minShouldMatch);
+  }
+
+  @Override
+  public long cost() {
+    return cost;
   }
 
   @Override
   public final Collection<ChildScorer> getChildren() {
-    ArrayList<ChildScorer> children = new ArrayList<>(sortedSubScorers.length);
-    for (int i = 0; i < sortedSubScorers.length; i++) {
-      children.add(new ChildScorer(sortedSubScorers[i], "SHOULD"));
-    }
-    return children;
+    return childScorers;
   }
-  
+
   @Override
   public int nextDoc() throws IOException {
-    assert doc != NO_MORE_DOCS;
-    while (true) {
-      // to remove current doc, call next() on all subScorers on current doc within heap
-      while (subScorers[0].docID() == doc) {
-        if (subScorers[0].nextDoc() != NO_MORE_DOCS) {
-          minheapSiftDown(0);
+    // 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 (ScorerWrapper s = lead; s != null; s = s.next) {
+      final ScorerWrapper evicted = insertTailWithOverFlow(s);
+      if (evicted != null) {
+        if (evicted.doc == doc) {
+          evicted.doc = evicted.scorer.nextDoc();
         } else {
-          minheapRemoveRoot();
-          numScorers--;
-          if (numScorers < mm) {
-            return doc = NO_MORE_DOCS;
-          }
+          evicted.doc = evicted.scorer.advance(doc + 1);
         }
-        //assert minheapCheck();
+        head.add(evicted);
       }
+    }
 
-      evaluateSmallestDocInHeap();
+    setDocAndFreq();
+    return doNext();
+  }
 
-      if (nrMatchers >= mm) { // doc satisfies mm constraint
-        break;
+  @Override
+  public int advance(int target) throws IOException {
+    // Same logic as in nextDoc
+    for (ScorerWrapper s = lead; s != null; s = s.next) {
+      final ScorerWrapper evicted = insertTailWithOverFlow(s);
+      if (evicted != null) {
+        evicted.doc = evicted.scorer.advance(target);
+        head.add(evicted);
       }
     }
-    return doc;
-  }
-  
-  private void evaluateSmallestDocInHeap() throws IOException {
-    // within heap, subScorer[0] now contains the next candidate doc
-    doc = subScorers[0].docID();
-    if (doc == NO_MORE_DOCS) {
-      nrMatchers = Integer.MAX_VALUE; // stop looping
-      return;
-    }
-    // 1. score and count number of matching subScorers within heap
-    score = subScorers[0].score();
-    nrMatchers = 1;
-    countMatches(1);
-    countMatches(2);
-    // 2. score and count number of matching subScorers within stack,
-    // short-circuit: stop when mm can't be reached for current doc, then perform on heap next()
-    // TODO instead advance() might be possible, but complicates things
-    for (int i = mm-2; i >= 0; i--) { // first advance sparsest subScorer
-      if (mmStack[i].docID() >= doc || mmStack[i].advance(doc) != NO_MORE_DOCS) {
-        if (mmStack[i].docID() == doc) { // either it was already on doc, or got there via advance()
-          nrMatchers++;
-          score += mmStack[i].score();
-        } else { // scorer advanced to next after doc, check if enough scorers left for current doc
-          if (nrMatchers + i < mm) { // too few subScorers left, abort advancing
-            return; // continue looping TODO consider advance() here
-          }
-        }
-      } else { // subScorer exhausted
-        numScorers--;
-        if (numScorers < mm) { // too few subScorers left
-          doc = NO_MORE_DOCS;
-          nrMatchers = Integer.MAX_VALUE; // stop looping
-          return;
-        }
-        if (mm-2-i > 0) {
-          // shift RHS of array left
-          System.arraycopy(mmStack, i+1, mmStack, i, mm-2-i);
-        }
-        // find next most costly subScorer within heap TODO can this be done better?
-        while (!minheapRemove(sortedSubScorers[sortedSubScorersIdx++])) {
-          //assert minheapCheck();
-        }
-        // add the subScorer removed from heap to stack
-        mmStack[mm-2] = sortedSubScorers[sortedSubScorersIdx-1];
-        
-        if (nrMatchers + i < mm) { // too few subScorers left, abort advancing
-          return; // continue looping TODO consider advance() here
-        }
-      }
+
+    // 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'
+    ScorerWrapper headTop = head.top();
+    while (headTop.doc < target) {
+      final ScorerWrapper 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.scorer.advance(target);
+      headTop = head.updateTop(evicted);
     }
+
+    setDocAndFreq();
+    return doNext();
   }
 
-  // TODO: this currently scores, but so did the previous impl
-  // TODO: remove recursion.
-  // TODO: consider separating scoring out of here, then modify this
-  // and afterNext() to terminate when nrMatchers == minimumNrMatchers
-  // then also change freq() to just always compute it from scratch
-  private void countMatches(int root) throws IOException {
-    if (root < nrInHeap && subScorers[root].docID() == doc) {
-      nrMatchers++;
-      score += subScorers[root].score();
-      countMatches((root<<1)+1);
-      countMatches((root<<1)+2);
+  private void addLead(ScorerWrapper lead) {
+    lead.next = this.lead;
+    this.lead = lead;
+    freq += 1;
+  }
+
+  private void pushBackLeads() throws IOException {
+    for (ScorerWrapper s = lead; s != null; s = s.next) {
+      addTail(s);
     }
   }
 
-  /**
-   * Returns the score of the current document matching the query. Initially
-   * invalid, until {@link #nextDoc()} is called the first time.
-   */
-  @Override
-  public float score() throws IOException {
-    return coord[nrMatchers] * (float) score;
+  private void advanceTail(ScorerWrapper top) throws IOException {
+    top.doc = top.scorer.advance(doc);
+    if (top.doc == doc) {
+      addLead(top);
+    } else {
+      head.add(top);
+    }
   }
 
-  @Override
-  public int docID() {
-    return doc;
+  private void advanceTail() throws IOException {
+    final ScorerWrapper top = popTail();
+    advanceTail(top);
   }
 
-  @Override
-  public int freq() throws IOException {
-    return nrMatchers;
+  /** 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());
+    }
   }
 
-  /**
-   * Advances to the first match beyond the current whose document number is
-   * greater than or equal to a given target. <br>
-   * The implementation uses the advance() method on the subscorers.
-   * 
-   * @param target the target document number.
-   * @return the document whose number is greater than or equal to the given
-   *         target, or -1 if none exist.
-   */
-  @Override
-  public int advance(int target) throws IOException {
-    if (numScorers < mm)
-      return doc = NO_MORE_DOCS;
-    // advance all Scorers in heap at smaller docs to at least target
-    while (subScorers[0].docID() < target) {
-      if (subScorers[0].advance(target) != NO_MORE_DOCS) {
-        minheapSiftDown(0);
+  /** 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 {
-        minheapRemoveRoot();
-        numScorers--;
-        if (numScorers < mm) {
-          return doc = NO_MORE_DOCS;
-        }
+        // no match on doc is possible anymore, move to the next potential match
+        pushBackLeads();
+        setDocAndFreq();
       }
-      //assert minheapCheck();
     }
 
-    evaluateSmallestDocInHeap();
+    return doc;
+  }
 
-    if (nrMatchers >= mm) {
-      return doc;
-    } else {
-      return nextDoc();
+  /** 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 long cost() {
-    // cost for merging of lists analog to DisjunctionSumScorer
-    long costCandidateGeneration = 0;
-    for (int i = 0; i < nrInHeap; i++)
-      costCandidateGeneration += subScorers[i].cost();
-    // TODO is cost for advance() different to cost for iteration + heap merge
-    //      and how do they compare overall to pure disjunctions? 
-    final float c1 = 1.0f,
-                c2 = 1.0f; // maybe a constant, maybe a proportion between costCandidateGeneration and sum(subScorer_to_be_advanced.cost())?
-    return (long) (
-           c1 * costCandidateGeneration +        // heap-merge cost
-           c2 * costCandidateGeneration * (mm-1) // advance() cost
-           );
-  }
-  
-  /**
-   * Organize subScorers into a min heap with scorers generating the earliest document on top.
-   */
-  protected final void minheapHeapify() {
-    for (int i = (nrInHeap >> 1) - 1; i >= 0; i--) {
-      minheapSiftDown(i);
-    }
-  }
-  
-  /**
-   * The subtree of subScorers at root is a min heap except possibly for its root element.
-   * Bubble the root down as required to make the subtree a heap.
-   */
-  protected final void minheapSiftDown(int root) {
-    // TODO could this implementation also move rather than swapping neighbours?
-    Scorer scorer = subScorers[root];
-    int doc = scorer.docID();
-    int i = root;
-    while (i <= (nrInHeap >> 1) - 1) {
-      int lchild = (i << 1) + 1;
-      Scorer lscorer = subScorers[lchild];
-      int ldoc = lscorer.docID();
-      int rdoc = Integer.MAX_VALUE, rchild = (i << 1) + 2;
-      Scorer rscorer = null;
-      if (rchild < nrInHeap) {
-        rscorer = subScorers[rchild];
-        rdoc = rscorer.docID();
-      }
-      if (ldoc < doc) {
-        if (rdoc < ldoc) {
-          subScorers[i] = rscorer;
-          subScorers[rchild] = scorer;
-          i = rchild;
-        } else {
-          subScorers[i] = lscorer;
-          subScorers[lchild] = scorer;
-          i = lchild;
-        }
-      } else if (rdoc < doc) {
-        subScorers[i] = rscorer;
-        subScorers[rchild] = scorer;
-        i = rchild;
-      } else {
-        return;
-      }
+  public int freq() throws IOException {
+    // we need to know about all matches
+    updateFreq();
+    return freq;
+  }
+
+  @Override
+  public float score() throws IOException {
+    // we need to know about all matches
+    updateFreq();
+    double score = 0;
+    for (ScorerWrapper s = lead; s != null; s = s.next) {
+      score += s.scorer.score();
     }
+    return coord[freq] * (float) score;
   }
 
-  protected final void minheapSiftUp(int i) {
-    Scorer scorer = subScorers[i];
-    final int doc = scorer.docID();
-    // find right place for scorer
-    while (i > 0) {
-      int parent = (i - 1) >> 1;
-      Scorer pscorer = subScorers[parent];
-      int pdoc = pscorer.docID();
-      if (pdoc > doc) { // move root down, make space
-        subScorers[i] = subScorers[parent];
-        i = parent;
-      } else { // done, found right place
-        break;
+  @Override
+  public int docID() {
+    assert doc == lead.doc;
+    return doc;
+  }
+
+  /** Insert an entry in 'tail' and evict the least-costly scorer if full. */
+  private ScorerWrapper insertTailWithOverFlow(ScorerWrapper s) {
+    if (tailSize < tail.length) {
+      addTail(s);
+      return null;
+    } else if (tail.length >= 1) {
+      final ScorerWrapper top = tail[0];
+      if (top.cost < s.cost) {
+        tail[0] = s;
+        downHeapCost(tail, tailSize);
+        return top;
       }
     }
-    subScorers[i] = scorer;
+    return s;
   }
 
-  /**
-   * Remove the root Scorer from subScorers and re-establish it as a heap
-   */
-  protected final void minheapRemoveRoot() {
-    if (nrInHeap == 1) {
-      //subScorers[0] = null; // not necessary
-      nrInHeap = 0;
-    } else {
-      nrInHeap--;
-      subScorers[0] = subScorers[nrInHeap];
-      //subScorers[nrInHeap] = null; // not necessary
-      minheapSiftDown(0);
-    }
-  }
-  
-  /**
-   * Removes a given Scorer from the heap by placing end of heap at that
-   * position and bubbling it either up or down
-   */
-  protected final boolean minheapRemove(Scorer scorer) {
-    // find scorer: O(nrInHeap)
-    for (int i = 0; i < nrInHeap; i++) {
-      if (subScorers[i] == scorer) { // remove scorer
-        subScorers[i] = subScorers[--nrInHeap];
-        //if (i != nrInHeap) subScorers[nrInHeap] = null; // not necessary
-        minheapSiftUp(i);
-        minheapSiftDown(i);
-        return true;
+  /** Add an entry to 'tail'. Fails if over capacity. */
+  private void addTail(ScorerWrapper s) {
+    tail[tailSize] = s;
+    upHeapCost(tail, tailSize);
+    tailSize += 1;
+  }
+
+  /** Pop the least-costly scorer from 'tail'. */
+  private ScorerWrapper popTail() {
+    assert tailSize > 0;
+    final ScorerWrapper result = tail[0];
+    tail[0] = tail[--tailSize];
+    downHeapCost(tail, tailSize);
+    return result;
+  }
+
+  /** Heap helpers */
+
+  private static void upHeapCost(ScorerWrapper[] heap, int i) {
+    final ScorerWrapper 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(ScorerWrapper[] heap, int size) {
+    int i = 0;
+    final ScorerWrapper 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;
       }
     }
-    return false; // scorer already exhausted
   }
-  
-  boolean minheapCheck() {
-    return minheapCheck(0);
-  }
-  private boolean minheapCheck(int root) {
-    if (root >= nrInHeap)
-      return true;
-    int lchild = (root << 1) + 1;
-    int rchild = (root << 1) + 2;
-    if (lchild < nrInHeap && subScorers[root].docID() > subScorers[lchild].docID())
-      return false;
-    if (rchild < nrInHeap && subScorers[root].docID() > subScorers[rchild].docID())
-      return false;
-    return minheapCheck(lchild) && minheapCheck(rchild);
-  }
-  
-}
\ No newline at end of file
+
+}

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java Wed Feb  4 15:42:06 2015
@@ -136,6 +136,11 @@ public abstract class Weight {
     }
 
     @Override
+    public long cost() {
+      return scorer.cost();
+    }
+
+    @Override
     public int score(LeafCollector collector, int min, int max) throws IOException {
       // TODO: this may be sort of weird, when we are
       // embedded in a BooleanScorer, because we are

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/util/PriorityQueue.java Wed Feb  4 15:42:06 2015
@@ -205,6 +205,14 @@ public abstract class PriorityQueue<T> {
     return heap[1];
   }
 
+  /**
+   * Replace the top of the pq with {@code newTop} and run {@link #updateTop()}.
+   */
+  public final T updateTop(T newTop) {
+    heap[1] = newTop;
+    return updateTop();
+  }
+
   /** Returns the number of elements currently stored in the PriorityQueue. */
   public final int size() {
     return size;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java Wed Feb  4 15:42:06 2015
@@ -234,6 +234,10 @@ public class TestBooleanOr extends Lucen
         }
         return RandomInts.randomIntBetween(random(), max, matches[i]);
       }
+      @Override
+      public long cost() {
+        return matches.length;
+      }
     };
   }
 
@@ -246,7 +250,7 @@ public class TestBooleanOr extends Lucen
         scorer(5000, 100000, 9999998, 9999999)
     );
     Collections.shuffle(optionalScorers, random());
-    BooleanScorer scorer = new BooleanScorer(null, true, 0, optionalScorers);
+    BooleanScorer scorer = new BooleanScorer(null, true, 0, optionalScorers, 1);
     final List<Integer> matches = new ArrayList<>();
     scorer.score(new LeafCollector() {
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java Wed Feb  4 15:42:06 2015
@@ -192,16 +192,19 @@ public class TestBooleanQueryVisitSubsco
     query.add(new TermQuery(new Term(F2, "web")), Occur.SHOULD);
     query.add(new TermQuery(new Term(F2, "crawler")), Occur.SHOULD);
     query.setMinimumNumberShouldMatch(2);
+    query.add(new MatchAllDocsQuery(), Occur.MUST);
     ScorerSummarizingCollector collector = new ScorerSummarizingCollector();
     searcher.search(query, collector);
     assertEquals(1, collector.getNumHits());
     assertFalse(collector.getSummaries().isEmpty());
     for (String summary : collector.getSummaries()) {
       assertEquals(
-          "MinShouldMatchSumScorer\n" +
-          "    SHOULD TermScorer body:nutch\n" +
-          "    SHOULD TermScorer body:web\n" +
-          "    SHOULD TermScorer body:crawler", summary);
+          "CoordinatingConjunctionScorer\n" +
+          "    MUST MinShouldMatchSumScorer\n" +
+          "            SHOULD TermScorer body:nutch\n" +
+          "            SHOULD TermScorer body:web\n" +
+          "            SHOULD TermScorer body:crawler\n" +
+          "    MUST MatchAllScorer", summary);
     }
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java Wed Feb  4 15:42:06 2015
@@ -99,7 +99,6 @@ public class TestBooleanScorer extends L
         @Override
         public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) {
           return new BulkScorer() {
-
             @Override
             public int score(LeafCollector collector, int min, int max) throws IOException {
               assert min == 0;
@@ -107,6 +106,10 @@ public class TestBooleanScorer extends L
               collector.collect(0);
               return DocIdSetIterator.NO_MORE_DOCS;
             }
+            @Override
+            public long cost() {
+              return 1;
+            }
           };
         }
       };

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java Wed Feb  4 15:42:06 2015
@@ -58,6 +58,12 @@ public class TestMinShouldMatch2 extends
   static final String mediumTerms[] = { "e", "f", "g" };
   static final String rareTerms[]   = { "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r", "s", "t", "u", "v", "w", "x", "y", "z" };
   
+  enum Mode {
+    SCORER,
+    BULK_SCORER,
+    DOC_VALUES
+  }
+  
   @BeforeClass
   public static void beforeClass() throws Exception {
     dir = newDirectory();
@@ -112,7 +118,7 @@ public class TestMinShouldMatch2 extends
     }
   }
   
-  private Scorer scorer(String values[], int minShouldMatch, boolean slow) throws Exception {
+  private Scorer scorer(String values[], int minShouldMatch, Mode mode) throws Exception {
     BooleanQuery bq = new BooleanQuery();
     for (String value : values) {
       bq.add(new TermQuery(new Term("field", value)), BooleanClause.Occur.SHOULD);
@@ -121,10 +127,22 @@ public class TestMinShouldMatch2 extends
 
     BooleanWeight weight = (BooleanWeight) searcher.createNormalizedWeight(bq);
     
-    if (slow) {
+    switch (mode) {
+    case DOC_VALUES:
       return new SlowMinShouldMatchScorer(weight, reader, searcher);
-    } else {
+    case SCORER:
       return weight.scorer(reader.getContext(), null);
+    case BULK_SCORER:
+      final BulkScorer bulkScorer = weight.booleanScorer(reader.getContext(), null);
+      if (bulkScorer == null) {
+        if (weight.scorer(reader.getContext(), null) != null) {
+          throw new AssertionError("BooleanScorer should be applicable for this query");
+        }
+        return null;
+      }
+      return new BulkScorerWrapperScorer(weight, bulkScorer, TestUtil.nextInt(random(), 1, 100));
+    default:
+      throw new AssertionError();
     }
   }
   
@@ -167,8 +185,12 @@ public class TestMinShouldMatch2 extends
     for (int common = 0; common < commonTerms.length; common++) {
       for (int medium = 0; medium < mediumTerms.length; medium++) {
         for (int rare = 0; rare < rareTerms.length; rare++) {
-          Scorer expected = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, true);
-          Scorer actual = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, false);
+          Scorer expected = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.DOC_VALUES);
+          Scorer actual = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.SCORER);
+          assertNext(expected, actual);
+
+          expected = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.DOC_VALUES);
+          actual = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.BULK_SCORER);
           assertNext(expected, actual);
         }
       }
@@ -181,8 +203,12 @@ public class TestMinShouldMatch2 extends
       for (int common = 0; common < commonTerms.length; common++) {
         for (int medium = 0; medium < mediumTerms.length; medium++) {
           for (int rare = 0; rare < rareTerms.length; rare++) {
-            Scorer expected = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, true);
-            Scorer actual = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, false);
+            Scorer expected = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.DOC_VALUES);
+            Scorer actual = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.SCORER);
+            assertAdvance(expected, actual, amount);
+
+            expected = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.DOC_VALUES);
+            actual = scorer(new String[] { commonTerms[common], mediumTerms[medium], rareTerms[rare] }, 2, Mode.BULK_SCORER);
             assertAdvance(expected, actual, amount);
           }
         }
@@ -199,8 +225,12 @@ public class TestMinShouldMatch2 extends
     String terms[] = termsList.toArray(new String[0]);
     
     for (int minNrShouldMatch = 1; minNrShouldMatch <= terms.length; minNrShouldMatch++) {
-      Scorer expected = scorer(terms, minNrShouldMatch, true);
-      Scorer actual = scorer(terms, minNrShouldMatch, false);
+      Scorer expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+      Scorer actual = scorer(terms, minNrShouldMatch, Mode.SCORER);
+      assertNext(expected, actual);
+
+      expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+      actual = scorer(terms, minNrShouldMatch, Mode.BULK_SCORER);
       assertNext(expected, actual);
     }
   }
@@ -215,8 +245,12 @@ public class TestMinShouldMatch2 extends
     
     for (int amount = 25; amount < 200; amount += 25) {
       for (int minNrShouldMatch = 1; minNrShouldMatch <= terms.length; minNrShouldMatch++) {
-        Scorer expected = scorer(terms, minNrShouldMatch, true);
-        Scorer actual = scorer(terms, minNrShouldMatch, false);
+        Scorer expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+        Scorer actual = scorer(terms, minNrShouldMatch, Mode.SCORER);
+        assertAdvance(expected, actual, amount);
+
+        expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+        actual = scorer(terms, minNrShouldMatch, Mode.BULK_SCORER);
         assertAdvance(expected, actual, amount);
       }
     }
@@ -232,8 +266,12 @@ public class TestMinShouldMatch2 extends
     for (int numTerms = 2; numTerms <= termsList.size(); numTerms++) {
       String terms[] = termsList.subList(0, numTerms).toArray(new String[0]);
       for (int minNrShouldMatch = 1; minNrShouldMatch <= terms.length; minNrShouldMatch++) {
-        Scorer expected = scorer(terms, minNrShouldMatch, true);
-        Scorer actual = scorer(terms, minNrShouldMatch, false);
+        Scorer expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+        Scorer actual = scorer(terms, minNrShouldMatch, Mode.SCORER);
+        assertNext(expected, actual);
+
+        expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+        actual = scorer(terms, minNrShouldMatch, Mode.BULK_SCORER);
         assertNext(expected, actual);
       }
     }
@@ -251,8 +289,12 @@ public class TestMinShouldMatch2 extends
       for (int numTerms = 2; numTerms <= termsList.size(); numTerms++) {
         String terms[] = termsList.subList(0, numTerms).toArray(new String[0]);
         for (int minNrShouldMatch = 1; minNrShouldMatch <= terms.length; minNrShouldMatch++) {
-          Scorer expected = scorer(terms, minNrShouldMatch, true);
-          Scorer actual = scorer(terms, minNrShouldMatch, false);
+          Scorer expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+          Scorer actual = scorer(terms, minNrShouldMatch, Mode.SCORER);
+          assertAdvance(expected, actual, amount);
+
+          expected = scorer(terms, minNrShouldMatch, Mode.DOC_VALUES);
+          actual = scorer(terms, minNrShouldMatch, Mode.SCORER);
           assertAdvance(expected, actual, amount);
         }
       }

Modified: lucene/dev/branches/branch_5x/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java Wed Feb  4 15:42:06 2015
@@ -64,6 +64,11 @@ class DrillSidewaysScorer extends BulkSc
   }
 
   @Override
+  public long cost() {
+    return baseScorer.cost();
+  }
+
+  @Override
   public int score(LeafCollector collector, int min, int maxDoc) throws IOException {
     if (min != 0) {
       throw new IllegalArgumentException("min must be 0, got " + min);

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java?rev=1657290&r1=1657289&r2=1657290&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java Wed Feb  4 15:42:06 2015
@@ -50,6 +50,11 @@ final class AssertingBulkScorer extends
   }
 
   @Override
+  public long cost() {
+    return in.cost();
+  }
+
+  @Override
   public void score(LeafCollector collector) throws IOException {
     assert max == 0;
     collector = new AssertingLeafCollector(random, collector, 0, DocsEnum.NO_MORE_DOCS);
@@ -68,7 +73,7 @@ final class AssertingBulkScorer extends
   @Override
   public int score(LeafCollector collector, int min, final int max) throws IOException {
     assert min >= this.max: "Scoring backward: min=" + min + " while previous max was max=" + this.max;
-    assert min < max : "max must be greater than min, got min=" + min + ", and max=" + max;
+    assert min <= max : "max must be greater than min, got min=" + min + ", and max=" + max;
     this.max = max;
     collector = new AssertingLeafCollector(random, collector, min, max);
     final int next = in.score(collector, min, max);