You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2014/03/11 21:04:55 UTC

svn commit: r1576487 [1/2] - in /lucene/dev/trunk: ./ dev-tools/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/core/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/search/payloads/ lucene/core/src/java/o...

Author: mikemccand
Date: Tue Mar 11 20:04:53 2014
New Revision: 1576487

URL: http://svn.apache.org/r1576487
Log:
LUCENE-5487: separate Weight.bulkScorer and Weight.scorer

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java
      - copied unchanged from r1576481, lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/BulkScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java
      - copied unchanged from r1576481, lucene/dev/branches/lucene5487/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java
      - copied unchanged from r1576481, lucene/dev/branches/lucene5487/lucene/join/src/java/org/apache/lucene/search/join/FakeScorer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java
      - copied unchanged from r1576481, lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkOutOfOrderScorer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
      - copied unchanged from r1576481, lucene/dev/branches/lucene5487/lucene/test-framework/src/java/org/apache/lucene/search/AssertingBulkScorer.java
Modified:
    lucene/dev/trunk/   (props changed)
    lucene/dev/trunk/dev-tools/   (props changed)
    lucene/dev/trunk/lucene/   (props changed)
    lucene/dev/trunk/lucene/CHANGES.txt   (contents, props changed)
    lucene/dev/trunk/lucene/analysis/   (props changed)
    lucene/dev/trunk/lucene/analysis/common/   (props changed)
    lucene/dev/trunk/lucene/core/   (props changed)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Scorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Weight.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/package.html
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestFilteredQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTermScorer.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/spans/TestSpans.java
    lucene/dev/trunk/lucene/facet/   (props changed)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/TaxonomyFacetSumValueSource.java
    lucene/dev/trunk/lucene/grouping/   (props changed)
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
    lucene/dev/trunk/lucene/join/   (props changed)
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToChildBlockJoinQuery.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinCollector.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/ToParentBlockJoinQuery.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestBlockJoin.java
    lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
    lucene/dev/trunk/lucene/memory/   (props changed)
    lucene/dev/trunk/lucene/misc/   (props changed)
    lucene/dev/trunk/lucene/queries/   (props changed)
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java
    lucene/dev/trunk/lucene/suggest/   (props changed)
    lucene/dev/trunk/lucene/test-framework/   (props changed)
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingIndexSearcher.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingScorer.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/AssertingWeight.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
    lucene/dev/trunk/solr/   (props changed)
    lucene/dev/trunk/solr/CHANGES.txt   (props changed)
    lucene/dev/trunk/solr/core/   (props changed)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/schema/LatLonType.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/join/IgnoreAcceptDocsQuery.java
    lucene/dev/trunk/solr/example/   (props changed)
    lucene/dev/trunk/solr/solrj/   (props changed)
    lucene/dev/trunk/solr/test-framework/   (props changed)

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Tue Mar 11 20:04:53 2014
@@ -120,6 +120,12 @@ API Changes
   a boolean that indicates if a new merge was found in the caller thread before
   the scheduler was called. (Simon Willnauer)
 
+* LUCENE-5487: Separated bulk scorer (new Weight.bulkScorer method) from
+  normal scoring (Weight.scorer) for those queries that can do bulk
+  scoring more efficiently, e.g. BooleanQuery in some cases.  This
+  also simplified the Weight.scorer API by removing the two confusing
+  booleans.  (Robert Muir, Uwe Schindler, Mike McCandless)
+
 Optimizations
 
 * LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java Tue Mar 11 20:04:53 2014
@@ -242,7 +242,7 @@ public class BooleanQuery extends Query 
       for (Iterator<Weight> wIter = weights.iterator(); wIter.hasNext();) {
         Weight w = wIter.next();
         BooleanClause c = cIter.next();
-        if (w.scorer(context, true, true, context.reader().getLiveDocs()) == null) {
+        if (w.scorer(context, context.reader().getLiveDocs()) == null) {
           if (c.isRequired()) {
             fail = true;
             Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@@ -305,8 +305,43 @@ public class BooleanQuery extends Query 
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs)
+    public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder,
+                                 Bits acceptDocs) throws IOException {
+
+      if (scoreDocsInOrder || 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, scoreDocsInOrder, acceptDocs);
+      }
+
+      List<BulkScorer> prohibited = new ArrayList<BulkScorer>();
+      List<BulkScorer> optional = new ArrayList<BulkScorer>();
+      Iterator<BooleanClause> cIter = clauses.iterator();
+      for (Weight w  : weights) {
+        BooleanClause c =  cIter.next();
+        BulkScorer subScorer = w.bulkScorer(context, false, acceptDocs);
+        if (subScorer == null) {
+          if (c.isRequired()) {
+            return null;
+          }
+        } else if (c.isRequired()) {
+          // TODO: there are some cases where BooleanScorer
+          // would handle conjunctions faster than
+          // BooleanScorer2...
+          return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
+        } else if (c.isProhibited()) {
+          prohibited.add(subScorer);
+        } else {
+          optional.add(subScorer);
+        }
+      }
+
+      // Check if we can and should return a BooleanScorer
+      return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
+    }
+
+    @Override
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs)
         throws IOException {
       List<Scorer> required = new ArrayList<Scorer>();
       List<Scorer> prohibited = new ArrayList<Scorer>();
@@ -314,7 +349,7 @@ public class BooleanQuery extends Query 
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
         BooleanClause c =  cIter.next();
-        Scorer subScorer = w.scorer(context, true, false, acceptDocs);
+        Scorer subScorer = w.scorer(context, acceptDocs);
         if (subScorer == null) {
           if (c.isRequired()) {
             return null;
@@ -328,20 +363,6 @@ public class BooleanQuery extends Query 
         }
       }
 
-      // NOTE: we could also use BooleanScorer, if we knew
-      // this BooleanQuery was embedded in another
-      // BooleanQuery that was also using BooleanScorer (ie,
-      // BooleanScorer can nest).  But this is hard to
-      // detect and we never do so today... (ie, we only
-      // return BooleanScorer for topScorer):
-
-      // Check if we can and should return a BooleanScorer
-      // TODO: (LUCENE-4872) in some cases BooleanScorer may be faster for minNrShouldMatch
-      // but the same is even true of pure conjunctions...
-      if (!scoreDocsInOrder && topScorer && required.size() == 0 && minNrShouldMatch <= 1) {
-        return new BooleanScorer(this, disableCoord, minNrShouldMatch, optional, prohibited, maxCoord);
-      }
-      
       if (required.size() == 0 && optional.size() == 0) {
         // no required and optional clauses.
         return null;
@@ -373,9 +394,14 @@ public class BooleanQuery extends Query 
     
     @Override
     public boolean scoresDocsOutOfOrder() {
+      if (minNrShouldMatch > 1) {
+        // BS2 (in-order) will be used by scorer()
+        return false;
+      }
       for (BooleanClause c : clauses) {
         if (c.isRequired()) {
-          return false; // BS2 (in-order) will be used by scorer()
+          // BS2 (in-order) will be used by scorer()
+          return false;
         }
       }
       

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer.java Tue Mar 11 20:04:53 2014
@@ -23,6 +23,7 @@ import java.util.Collection;
 import java.util.List;
 
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 
 /* Description from Doug Cutting (excerpted from
@@ -58,7 +59,7 @@ import org.apache.lucene.search.BooleanQ
  * conjunction can reduce the number of priority queue
  * updates for the optional terms. */
 
-final class BooleanScorer extends Scorer {
+final class BooleanScorer extends BulkScorer {
   
   private static final class BooleanScorerCollector extends Collector {
     private BucketTable bucketTable;
@@ -108,38 +109,6 @@ final class BooleanScorer extends Scorer
 
   }
   
-  // An internal class which is used in score(Collector, int) for setting the
-  // current score. This is required since Collector exposes a setScorer method
-  // and implementations that need the score will call scorer.score().
-  // Therefore the only methods that are implemented are score() and doc().
-  private static final class BucketScorer extends Scorer {
-
-    double score;
-    int doc = NO_MORE_DOCS;
-    int freq;
-    
-    public BucketScorer(Weight weight) { super(weight); }
-    
-    @Override
-    public int advance(int target) { return NO_MORE_DOCS; }
-
-    @Override
-    public int docID() { return doc; }
-
-    @Override
-    public int freq() { return freq; }
-
-    @Override
-    public int nextDoc() { return NO_MORE_DOCS; }
-    
-    @Override
-    public float score() { return (float)score; }
-    
-    @Override
-    public long cost() { return 1; }
-
-  }
-
   static final class Bucket {
     int doc = -1;            // tells if bucket is valid
     double score;             // incremental score
@@ -175,19 +144,21 @@ final class BooleanScorer extends Scorer
   }
 
   static final class SubScorer {
-    public Scorer scorer;
+    public BulkScorer scorer;
     // TODO: re-enable this if BQ ever sends us required clauses
     //public boolean required = false;
     public boolean prohibited;
     public Collector collector;
     public SubScorer next;
+    public boolean more;
 
-    public SubScorer(Scorer scorer, boolean required, boolean prohibited,
+    public SubScorer(BulkScorer scorer, boolean required, boolean prohibited,
         Collector collector, SubScorer next) {
       if (required) {
         throw new IllegalArgumentException("this scorer cannot handle required=true");
       }
       this.scorer = scorer;
+      this.more = true;
       // TODO: re-enable this if BQ ever sends us required clauses
       //this.required = required;
       this.prohibited = prohibited;
@@ -206,26 +177,20 @@ final class BooleanScorer extends Scorer
   private Bucket current;
   // Any time a prohibited clause matches we set bit 0:
   private static final int PROHIBITED_MASK = 1;
-  
+
+  private final Weight weight;
+
   BooleanScorer(BooleanWeight weight, boolean disableCoord, int minNrShouldMatch,
-      List<Scorer> optionalScorers, List<Scorer> prohibitedScorers, int maxCoord) throws IOException {
-    super(weight);
+      List<BulkScorer> optionalScorers, List<BulkScorer> prohibitedScorers, int maxCoord) throws IOException {
     this.minNrShouldMatch = minNrShouldMatch;
+    this.weight = weight;
 
-    if (optionalScorers != null && optionalScorers.size() > 0) {
-      for (Scorer scorer : optionalScorers) {
-        if (scorer.nextDoc() != NO_MORE_DOCS) {
-          scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers);
-        }
-      }
+    for (BulkScorer scorer : optionalScorers) {
+      scorers = new SubScorer(scorer, false, false, bucketTable.newCollector(0), scorers);
     }
     
-    if (prohibitedScorers != null && prohibitedScorers.size() > 0) {
-      for (Scorer scorer : prohibitedScorers) {
-        if (scorer.nextDoc() != NO_MORE_DOCS) {
-          scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
-        }
-      }
+    for (BulkScorer scorer : prohibitedScorers) {
+      scorers = new SubScorer(scorer, false, true, bucketTable.newCollector(PROHIBITED_MASK), scorers);
     }
 
     coordFactors = new float[optionalScorers.size() + 1];
@@ -234,17 +199,15 @@ final class BooleanScorer extends Scorer
     }
   }
 
-  // firstDocID is ignored since nextDoc() initializes 'current'
   @Override
-  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
-    // Make sure it's only BooleanScorer that calls us:
-    assert firstDocID == -1;
+  public boolean score(Collector collector, int max) throws IOException {
+
     boolean more;
     Bucket tmp;
-    BucketScorer bs = new BucketScorer(weight);
+    FakeScorer fs = new FakeScorer();
 
     // The internal loop will set the score and doc before calling collect.
-    collector.setScorer(bs);
+    collector.setScorer(fs);
     do {
       bucketTable.first = null;
       
@@ -263,7 +226,7 @@ final class BooleanScorer extends Scorer
           // that should work)... but in theory an outside
           // app could pass a different max so we must check
           // it:
-          if (current.doc >= max){
+          if (current.doc >= max) {
             tmp = current;
             current = current.next;
             tmp.next = bucketTable.first;
@@ -272,9 +235,9 @@ final class BooleanScorer extends Scorer
           }
           
           if (current.coord >= minNrShouldMatch) {
-            bs.score = current.score * coordFactors[current.coord];
-            bs.doc = current.doc;
-            bs.freq = current.coord;
+            fs.score = (float) (current.score * coordFactors[current.coord]);
+            fs.doc = current.doc;
+            fs.freq = current.coord;
             collector.collect(current.doc);
           }
         }
@@ -292,9 +255,9 @@ final class BooleanScorer extends Scorer
       more = false;
       end += BucketTable.SIZE;
       for (SubScorer sub = scorers; sub != null; sub = sub.next) {
-        int subScorerDocID = sub.scorer.docID();
-        if (subScorerDocID != NO_MORE_DOCS) {
-          more |= sub.scorer.score(sub.collector, end, subScorerDocID);
+        if (sub.more) {
+          sub.more = sub.scorer.score(sub.collector, end);
+          more |= sub.more;
         }
       }
       current = bucketTable.first;
@@ -303,43 +266,8 @@ final class BooleanScorer extends Scorer
 
     return false;
   }
-  
-  @Override
-  public int advance(int target) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int docID() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public int nextDoc() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public float score() {
-    throw new UnsupportedOperationException();
-  }
 
   @Override
-  public int freq() throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public long cost() {
-    return Integer.MAX_VALUE;
-  }
-
-  @Override
-  public void score(Collector collector) throws IOException {
-    score(collector, Integer.MAX_VALUE, -1);
-  }
-  
-  @Override
   public String toString() {
     StringBuilder buffer = new StringBuilder();
     buffer.append("boolean(");
@@ -350,9 +278,4 @@ final class BooleanScorer extends Scorer
     buffer.append(")");
     return buffer.toString();
   }
-  
-  @Override
-  public Collection<ChildScorer> getChildren() {
-    throw new UnsupportedOperationException();
-  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanScorer2.java Tue Mar 11 20:04:53 2014
@@ -279,28 +279,6 @@ class BooleanScorer2 extends Scorer {
                                 : new MinShouldMatchSumScorer(weight, prohibitedScorers)));
   }
 
-  /** Scores and collects all matching documents.
-   * @param collector The collector to which all matching documents are passed through.
-   */
-  @Override
-  public void score(Collector collector) throws IOException {
-    collector.setScorer(this);
-    while ((doc = countingSumScorer.nextDoc()) != NO_MORE_DOCS) {
-      collector.collect(doc);
-    }
-  }
-  
-  @Override
-  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
-    doc = firstDocID;
-    collector.setScorer(this);
-    while (doc < max) {
-      collector.collect(doc);
-      doc = countingSumScorer.nextDoc();
-    }
-    return doc != NO_MORE_DOCS;
-  }
-
   @Override
   public int docID() {
     return doc;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Tue Mar 11 20:04:53 2014
@@ -134,8 +134,23 @@ public class ConstantScoreQuery extends 
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, final Bits acceptDocs) throws IOException {
+    public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+      final DocIdSetIterator disi;
+      if (filter != null) {
+        assert query == null;
+        return super.bulkScorer(context, scoreDocsInOrder, acceptDocs);
+      } else {
+        assert query != null && innerWeight != null;
+        BulkScorer bulkScorer = innerWeight.bulkScorer(context, scoreDocsInOrder, acceptDocs);
+        if (bulkScorer == null) {
+          return null;
+        }
+        return new ConstantBulkScorer(bulkScorer, this, queryWeight);
+      }
+    }
+
+    @Override
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
@@ -146,7 +161,7 @@ public class ConstantScoreQuery extends 
         disi = dis.iterator();
       } else {
         assert query != null && innerWeight != null;
-        disi = innerWeight.scorer(context, scoreDocsInOrder, topScorer, acceptDocs);
+        disi = innerWeight.scorer(context, acceptDocs);
       }
 
       if (disi == null) {
@@ -154,7 +169,7 @@ public class ConstantScoreQuery extends 
       }
       return new ConstantScorer(disi, this, queryWeight);
     }
-    
+
     @Override
     public boolean scoresDocsOutOfOrder() {
       return (innerWeight != null) ? innerWeight.scoresDocsOutOfOrder() : false;
@@ -162,7 +177,7 @@ public class ConstantScoreQuery extends 
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      final Scorer cs = scorer(context, true, false, context.reader().getLiveDocs());
+      final Scorer cs = scorer(context, context.reader().getLiveDocs());
       final boolean exists = (cs != null && cs.advance(doc) == doc);
 
       final ComplexExplanation result = new ComplexExplanation();
@@ -181,6 +196,52 @@ public class ConstantScoreQuery extends 
     }
   }
 
+  /** We return this as our {@link BulkScorer} so that if the CSQ
+   *  wraps a query with its own optimized top-level
+   *  scorer (e.g. BooleanScorer) we can use that
+   *  top-level scorer. */
+  protected class ConstantBulkScorer extends BulkScorer {
+    final BulkScorer bulkScorer;
+    final Weight weight;
+    final float theScore;
+
+    public ConstantBulkScorer(BulkScorer bulkScorer, Weight weight, float theScore) {
+      this.bulkScorer = bulkScorer;
+      this.weight = weight;
+      this.theScore = theScore;
+    }
+
+    @Override
+    public boolean score(Collector collector, int max) throws IOException {
+      return bulkScorer.score(wrapCollector(collector), max);
+    }
+
+    private Collector wrapCollector(final Collector collector) {
+      return new Collector() {
+        @Override
+        public void setScorer(Scorer scorer) throws IOException {
+          // we must wrap again here, but using the scorer passed in as parameter:
+          collector.setScorer(new ConstantScorer(scorer, weight, theScore));
+        }
+        
+        @Override
+        public void collect(int doc) throws IOException {
+          collector.collect(doc);
+        }
+        
+        @Override
+        public void setNextReader(AtomicReaderContext context) throws IOException {
+          collector.setNextReader(context);
+        }
+        
+        @Override
+        public boolean acceptsDocsOutOfOrder() {
+          return collector.acceptsDocsOutOfOrder();
+        }
+      };
+    }
+  }
+
   protected class ConstantScorer extends Scorer {
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
@@ -222,57 +283,13 @@ public class ConstantScoreQuery extends 
       return docIdSetIterator.cost();
     }
 
-    private Collector wrapCollector(final Collector collector) {
-      return new Collector() {
-        @Override
-        public void setScorer(Scorer scorer) throws IOException {
-          // we must wrap again here, but using the scorer passed in as parameter:
-          collector.setScorer(new ConstantScorer(scorer, ConstantScorer.this.weight, ConstantScorer.this.theScore));
-        }
-        
-        @Override
-        public void collect(int doc) throws IOException {
-          collector.collect(doc);
-        }
-        
-        @Override
-        public void setNextReader(AtomicReaderContext context) throws IOException {
-          collector.setNextReader(context);
-        }
-        
-        @Override
-        public boolean acceptsDocsOutOfOrder() {
-          return collector.acceptsDocsOutOfOrder();
-        }
-      };
-    }
-
-    // this optimization allows out of order scoring as top scorer!
-    @Override
-    public void score(Collector collector) throws IOException {
-      if (query != null) {
-        ((Scorer) docIdSetIterator).score(wrapCollector(collector));
-      } else {
-        super.score(collector);
-      }
-    }
-
-    // this optimization allows out of order scoring as top scorer,
-    @Override
-    public boolean score(Collector collector, int max, int firstDocID) throws IOException {
-      if (query != null) {
-        return ((Scorer) docIdSetIterator).score(wrapCollector(collector), max, firstDocID);
-      } else {
-        return super.score(collector, max, firstDocID);
-      }
-    }
-
     @Override
     public Collection<ChildScorer> getChildren() {
-      if (query != null)
+      if (query != null) {
         return Collections.singletonList(new ChildScorer((Scorer) docIdSetIterator, "constant"));
-      else
+      } else {
         return Collections.emptyList();
+      }
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionMaxQuery.java Tue Mar 11 20:04:53 2014
@@ -153,12 +153,11 @@ public class DisjunctionMaxQuery extends
 
     /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       List<Scorer> scorers = new ArrayList<Scorer>();
       for (Weight w : weights) {
         // we will advance() subscorers
-        Scorer subScorer = w.scorer(context, true, false, acceptDocs);
+        Scorer subScorer = w.scorer(context, acceptDocs);
         if (subScorer != null) {
           scorers.add(subScorer);
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Tue Mar 11 20:04:53 2014
@@ -93,12 +93,12 @@ public class FilteredQuery extends Query
       }
 
       @Override
-      public void normalize (float norm, float topLevelBoost) { 
+      public void normalize(float norm, float topLevelBoost) { 
         weight.normalize(norm, topLevelBoost * getBoost()); // incorporate boost
       }
 
       @Override
-      public Explanation explain (AtomicReaderContext ir, int i) throws IOException {
+      public Explanation explain(AtomicReaderContext ir, int i) throws IOException {
         Explanation inner = weight.explain (ir, i);
         Filter f = FilteredQuery.this.filter;
         DocIdSet docIdSet = f.getDocIdSet(ir, ir.reader().getLiveDocs());
@@ -124,16 +124,30 @@ public class FilteredQuery extends Query
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, final Bits acceptDocs) throws IOException {
+      public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
         assert filter != null;
 
-        final DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
+        DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
         if (filterDocIdSet == null) {
           // this means the filter does not accept any documents.
           return null;
         }
 
-        return strategy.filteredScorer(context, scoreDocsInOrder, topScorer, weight, filterDocIdSet);
+        return strategy.filteredScorer(context, weight, filterDocIdSet);
+      }
+
+      // return a filtering top scorer
+      @Override
+      public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+        assert filter != null;
+
+        DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
+        if (filterDocIdSet == null) {
+          // this means the filter does not accept any documents.
+          return null;
+        }
+
+        return strategy.filteredBulkScorer(context, weight, scoreDocsInOrder, filterDocIdSet);
       }
     };
   }
@@ -147,37 +161,20 @@ public class FilteredQuery extends Query
   private static final class QueryFirstScorer extends Scorer {
     private final Scorer scorer;
     private int scorerDoc = -1;
-    private Bits filterbits;
+    private final Bits filterBits;
 
     protected QueryFirstScorer(Weight weight, Bits filterBits, Scorer other) {
       super(weight);
       this.scorer = other;
-      this.filterbits = filterBits;
-    }
-    
-    // optimization: we are topScorer and collect directly
-    @Override
-    public void score(Collector collector) throws IOException {
-      // the normalization trick already applies the boost of this query,
-      // so we can use the wrapped scorer directly:
-      collector.setScorer(scorer);
-      for (;;) {
-        final int scorerDoc = scorer.nextDoc();
-        if (scorerDoc == DocIdSetIterator.NO_MORE_DOCS) {
-          break;
-        }
-        if (filterbits.get(scorerDoc)) {
-          collector.collect(scorerDoc);
-        }
-      }
+      this.filterBits = filterBits;
     }
-    
+
     @Override
     public int nextDoc() throws IOException {
       int doc;
       for(;;) {
         doc = scorer.nextDoc();
-        if (doc == Scorer.NO_MORE_DOCS || filterbits.get(doc)) {
+        if (doc == Scorer.NO_MORE_DOCS || filterBits.get(doc)) {
           return scorerDoc = doc;
         }
       } 
@@ -186,7 +183,7 @@ public class FilteredQuery extends Query
     @Override
     public int advance(int target) throws IOException {
       int doc = scorer.advance(target);
-      if (doc != Scorer.NO_MORE_DOCS && !filterbits.get(doc)) {
+      if (doc != Scorer.NO_MORE_DOCS && !filterBits.get(doc)) {
         return scorerDoc = nextDoc();
       } else {
         return scorerDoc = doc;
@@ -216,6 +213,40 @@ public class FilteredQuery extends Query
       return scorer.cost();
     }
   }
+
+  private static class QueryFirstBulkScorer extends BulkScorer {
+
+    private final Scorer scorer;
+    private final Bits filterBits;
+
+    public QueryFirstBulkScorer(Scorer scorer, Bits filterBits) {
+      this.scorer = scorer;
+      this.filterBits = filterBits;
+    }
+
+    @Override
+    public boolean score(Collector collector, int maxDoc) throws IOException {
+      // the normalization trick already applies the boost of this query,
+      // so we can use the wrapped scorer directly:
+      collector.setScorer(scorer);
+      if (scorer.docID() == -1) {
+        scorer.nextDoc();
+      }
+      while (true) {
+        final int scorerDoc = scorer.docID();
+        if (scorerDoc < maxDoc) {
+          if (filterBits.get(scorerDoc)) {
+            collector.collect(scorerDoc);
+          }
+          scorer.nextDoc();
+        } else {
+          break;
+        }
+      }
+
+      return scorer.docID() != Scorer.NO_MORE_DOCS;
+    }
+  }
   
   /**
    * A Scorer that uses a "leap-frog" approach (also called "zig-zag join"). The scorer and the filter
@@ -236,32 +267,7 @@ public class FilteredQuery extends Query
       this.secondary = secondary;
       this.scorer = scorer;
     }
-    
-    // optimization: we are topScorer and collect directly using short-circuited algo
-    @Override
-    public final void score(Collector collector) throws IOException {
-      // the normalization trick already applies the boost of this query,
-      // so we can use the wrapped scorer directly:
-      collector.setScorer(scorer);
-      int primDoc = primaryNext();
-      int secDoc = secondary.advance(primDoc);
-      for (;;) {
-        if (primDoc == secDoc) {
-          // Check if scorer has exhausted, only before collecting.
-          if (primDoc == DocIdSetIterator.NO_MORE_DOCS) {
-            break;
-          }
-          collector.collect(primDoc);
-          primDoc = primary.nextDoc();
-          secDoc = secondary.advance(primDoc);
-        } else if (secDoc > primDoc) {
-          primDoc = primary.advance(secDoc);
-        } else {
-          secDoc = secondary.advance(primDoc);
-        }
-      }
-    }
-    
+
     private final int advanceToNextCommonDoc() throws IOException {
       for (;;) {
         if (secondaryDoc < primaryDoc) {
@@ -472,18 +478,6 @@ public class FilteredQuery extends Query
      * 
      * @param context
      *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
-     * @param scoreDocsInOrder
-     *          specifies whether in-order scoring of documents is required. Note
-     *          that if set to false (i.e., out-of-order scoring is required),
-     *          this method can return whatever scoring mode it supports, as every
-     *          in-order scorer is also an out-of-order one. However, an
-     *          out-of-order scorer may not support {@link Scorer#nextDoc()}
-     *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
-     *          request an in-order scorer if use of these methods is required.
-     * @param topScorer
-     *          if true, {@link Scorer#score(Collector)} will be called; if false,
-     *          {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
-     *          be called.
      * @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
      * @param docIdSet the filter {@link DocIdSet} to apply
      * @return a filtered scorer
@@ -491,8 +485,30 @@ public class FilteredQuery extends Query
      * @throws IOException if an {@link IOException} occurs
      */
     public abstract Scorer filteredScorer(AtomicReaderContext context,
-        boolean scoreDocsInOrder, boolean topScorer, Weight weight,
-        DocIdSet docIdSet) throws IOException;
+        Weight weight, DocIdSet docIdSet) throws IOException;
+
+    /**
+     * Returns a filtered {@link BulkScorer} based on this
+     * strategy.  This is an optional method: the default
+     * implementation just calls {@link #filteredScorer} and
+     * wraps that into a BulkScorer.
+     *
+     * @param context
+     *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
+     * @param weight the {@link FilteredQuery} {@link Weight} to create the filtered scorer.
+     * @param docIdSet the filter {@link DocIdSet} to apply
+     * @return a filtered top scorer
+     */
+    public BulkScorer filteredBulkScorer(AtomicReaderContext context,
+        Weight weight, boolean scoreDocsInOrder, DocIdSet docIdSet) throws IOException {
+      Scorer scorer = filteredScorer(context, weight, docIdSet);
+      if (scorer == null) {
+        return null;
+      }
+      // This impl always scores docs in order, so we can
+      // ignore scoreDocsInOrder:
+      return new Weight.DefaultBulkScorer(scorer);
+    }
   }
   
   /**
@@ -506,7 +522,7 @@ public class FilteredQuery extends Query
   public static class RandomAccessFilterStrategy extends FilterStrategy {
 
     @Override
-    public Scorer filteredScorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer, Weight weight, DocIdSet docIdSet) throws IOException {
+    public Scorer filteredScorer(AtomicReaderContext context, Weight weight, DocIdSet docIdSet) throws IOException {
       final DocIdSetIterator filterIter = docIdSet.iterator();
       if (filterIter == null) {
         // this means the filter does not accept any documents.
@@ -523,12 +539,12 @@ public class FilteredQuery extends Query
       final boolean useRandomAccess = filterAcceptDocs != null && useRandomAccess(filterAcceptDocs, firstFilterDoc);
       if (useRandomAccess) {
         // if we are using random access, we return the inner scorer, just with other acceptDocs
-        return weight.scorer(context, scoreDocsInOrder, topScorer, filterAcceptDocs);
+        return weight.scorer(context, filterAcceptDocs);
       } else {
         assert firstFilterDoc > -1;
         // we are gonna advance() this scorer, so we set inorder=true/toplevel=false
         // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
-        final Scorer scorer = weight.scorer(context, true, false, null);
+        final Scorer scorer = weight.scorer(context, null);
         // TODO once we have way to figure out if we use RA or LeapFrog we can remove this scorer
         return (scorer == null) ? null : new PrimaryAdvancedLeapFrogScorer(weight, firstFilterDoc, filterIter, scorer);
       }
@@ -559,25 +575,27 @@ public class FilteredQuery extends Query
     private LeapFrogFilterStrategy(boolean scorerFirst) {
       this.scorerFirst = scorerFirst;
     }
+
     @Override
     public Scorer filteredScorer(AtomicReaderContext context,
-        boolean scoreDocsInOrder, boolean topScorer, Weight weight,
-        DocIdSet docIdSet) throws IOException {
+        Weight weight, DocIdSet docIdSet) throws IOException {
       final DocIdSetIterator filterIter = docIdSet.iterator();
       if (filterIter == null) {
         // this means the filter does not accept any documents.
         return null;
       }
-      // we are gonna advance() this scorer, so we set inorder=true/toplevel=false
       // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
-      final Scorer scorer = weight.scorer(context, true, false, null);
+      final Scorer scorer = weight.scorer(context, null);
+      if (scorer == null) {
+        return null;
+      }
+
       if (scorerFirst) {
-        return (scorer == null) ? null : new LeapFrogScorer(weight, scorer, filterIter, scorer);  
+        return new LeapFrogScorer(weight, scorer, filterIter, scorer);  
       } else {
-        return (scorer == null) ? null : new LeapFrogScorer(weight, filterIter, scorer, scorer);  
+        return new LeapFrogScorer(weight, filterIter, scorer, scorer);  
       }
     }
-    
   }
   
   /**
@@ -596,16 +614,33 @@ public class FilteredQuery extends Query
   private static final class QueryFirstFilterStrategy extends FilterStrategy {
     @Override
     public Scorer filteredScorer(final AtomicReaderContext context,
-        boolean scoreDocsInOrder, boolean topScorer, Weight weight,
+        Weight weight,
         DocIdSet docIdSet) throws IOException {
       Bits filterAcceptDocs = docIdSet.bits();
       if (filterAcceptDocs == null) {
-        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, scoreDocsInOrder, topScorer, weight, docIdSet);
+        // Filter does not provide random-access Bits; we
+        // must fallback to leapfrog:
+        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet);
       }
-      final Scorer scorer = weight.scorer(context, true, false, null);
+      final Scorer scorer = weight.scorer(context, null);
       return scorer == null ? null : new QueryFirstScorer(weight,
           filterAcceptDocs, scorer);
     }
+
+    @Override
+    public BulkScorer filteredBulkScorer(final AtomicReaderContext context,
+        Weight weight,
+        boolean scoreDocsInOrder, // ignored (we always top-score in order)
+        DocIdSet docIdSet) throws IOException {
+      Bits filterAcceptDocs = docIdSet.bits();
+      if (filterAcceptDocs == null) {
+        // Filter does not provide random-access Bits; we
+        // must fallback to leapfrog:
+        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, scoreDocsInOrder, docIdSet);
+      }
+      final Scorer scorer = weight.scorer(context, null);
+      return scorer == null ? null : new QueryFirstBulkScorer(scorer, filterAcceptDocs);
+    }
   }
   
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java Tue Mar 11 20:04:53 2014
@@ -607,7 +607,7 @@ public class IndexSearcher {
         // continue with the following leaf
         continue;
       }
-      Scorer scorer = weight.scorer(ctx, !collector.acceptsDocsOutOfOrder(), true, ctx.reader().getLiveDocs());
+      BulkScorer scorer = weight.bulkScorer(ctx, !collector.acceptsDocsOutOfOrder(), ctx.reader().getLiveDocs());
       if (scorer != null) {
         try {
           scorer.score(collector);
@@ -768,45 +768,6 @@ public class IndexSearcher {
       this.doMaxScore = doMaxScore;
     }
 
-    private final class FakeScorer extends Scorer {
-      float score;
-      int doc;
-
-      public FakeScorer() {
-        super(null);
-      }
-    
-      @Override
-      public int advance(int target) {
-        throw new UnsupportedOperationException("FakeScorer doesn't support advance(int)");
-      }
-
-      @Override
-      public int docID() {
-        return doc;
-      }
-
-      @Override
-      public int freq() {
-        throw new UnsupportedOperationException("FakeScorer doesn't support freq()");
-      }
-
-      @Override
-      public int nextDoc() {
-        throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
-      }
-    
-      @Override
-      public float score() {
-        return score;
-      }
-
-      @Override
-      public long cost() {
-        return 1;
-      }
-    }
-
     private final FakeScorer fakeScorer = new FakeScorer();
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Tue Mar 11 20:04:53 2014
@@ -114,8 +114,7 @@ public class MatchAllDocsQuery extends Q
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Tue Mar 11 20:04:53 2014
@@ -179,8 +179,7 @@ public class MultiPhraseQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       assert !termArrays.isEmpty();
       final AtomicReader reader = context.reader();
       final Bits liveDocs = acceptDocs;
@@ -263,7 +262,7 @@ public class MultiPhraseQuery extends Qu
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Tue Mar 11 20:04:53 2014
@@ -245,8 +245,7 @@ public class PhraseQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       assert !terms.isEmpty();
       final AtomicReader reader = context.reader();
       final Bits liveDocs = acceptDocs;
@@ -305,7 +304,7 @@ public class PhraseQuery extends Query {
 
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java Tue Mar 11 20:04:53 2014
@@ -56,7 +56,7 @@ public class QueryWrapperFilter extends 
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {
-        return weight.scorer(privateContext, true, false, acceptDocs);
+        return weight.scorer(privateContext, acceptDocs);
       }
       @Override
       public boolean isCacheable() { return false; }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ScoreCachingWrappingScorer.java Tue Mar 11 20:04:53 2014
@@ -45,11 +45,6 @@ public class ScoreCachingWrappingScorer 
   }
 
   @Override
-  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
-    return scorer.score(collector, max, firstDocID);
-  }
-  
-  @Override
   public float score() throws IOException {
     int doc = scorer.docID();
     if (doc != curDoc) {
@@ -76,11 +71,6 @@ public class ScoreCachingWrappingScorer 
   }
   
   @Override
-  public void score(Collector collector) throws IOException {
-    scorer.score(collector);
-  }
-  
-  @Override
   public int advance(int target) throws IOException {
     return scorer.advance(target);
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Scorer.java Tue Mar 11 20:04:53 2014
@@ -54,42 +54,6 @@ public abstract class Scorer extends Doc
     this.weight = weight;
   }
 
-  /** Scores and collects all matching documents.
-   * @param collector The collector to which all matching documents are passed.
-   */
-  public void score(Collector collector) throws IOException {
-    assert docID() == -1; // not started
-    collector.setScorer(this);
-    int doc;
-    while ((doc = nextDoc()) != NO_MORE_DOCS) {
-      collector.collect(doc);
-    }
-  }
-
-  /**
-   * Expert: Collects matching documents in a range. Hook for optimization.
-   * Note, <code>firstDocID</code> is added to ensure that {@link #nextDoc()}
-   * was called before this method.
-   * 
-   * @param collector
-   *          The collector to which all matching documents are passed.
-   * @param max
-   *          Do not score documents past this.
-   * @param firstDocID
-   *          The first document ID (ensures {@link #nextDoc()} is called before
-   *          this method.
-   * @return true if more matching documents may remain.
-   */
-  public boolean score(Collector collector, int max, int firstDocID) throws IOException {
-    assert docID() == firstDocID;
-    collector.setScorer(this);
-    int doc;
-    for (doc = firstDocID; doc < max; doc = nextDoc()) {
-      collector.collect(doc);
-    }
-    return doc != NO_MORE_DOCS;
-  }
-  
   /** Returns the score of the current document matching the query.
    * Initially invalid, until {@link #nextDoc()} or {@link #advance(int)}
    * is called the first time, or when called from within

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TermQuery.java Tue Mar 11 20:04:53 2014
@@ -75,8 +75,7 @@ public class TermQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termStates.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context);
       final TermsEnum termsEnum = getTermsEnum(context);
       if (termsEnum == null) {
@@ -111,7 +110,7 @@ public class TermQuery extends Query {
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, true, false, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Weight.java Tue Mar 11 20:04:53 2014
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.AtomicReader; // javadocs
 import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.IndexReaderContext; // javadocs
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Bits;
@@ -35,8 +36,7 @@ import org.apache.lucene.util.Bits;
  * {@link AtomicReader} dependent state should reside in the {@link Scorer}.
  * <p>
  * Since {@link Weight} creates {@link Scorer} instances for a given
- * {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, 
- * boolean, boolean, Bits)})
+ * {@link AtomicReaderContext} ({@link #scorer(AtomicReaderContext, Bits)})
  * callers must maintain the relationship between the searcher's top-level
  * {@link IndexReaderContext} and the context used to create a {@link Scorer}. 
  * <p>
@@ -51,7 +51,7 @@ import org.apache.lucene.util.Bits;
  * <li>The query normalization factor is passed to {@link #normalize(float, float)}. At
  * this point the weighting is complete.
  * <li>A <code>Scorer</code> is constructed by
- * {@link #scorer(AtomicReaderContext, boolean, boolean, Bits)}.
+ * {@link #scorer(AtomicReaderContext, Bits)}.
  * </ol>
  * 
  * @since 2.9
@@ -91,6 +91,25 @@ public abstract class Weight {
    * 
    * @param context
    *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
+   * @param acceptDocs
+   *          Bits that represent the allowable docs to match (typically deleted docs
+   *          but possibly filtering other documents)
+   *          
+   * @return a {@link Scorer} which scores documents in/out-of order.
+   * @throws IOException if there is a low-level I/O error
+   */
+  public abstract Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException;
+
+  /**
+   * Optional method, to return a {@link BulkScorer} to
+   * score the query and send hits to a {@link Collector}.
+   * Only queries that have a different top-level approach
+   * need to override this; the default implementation
+   * pulls a normal {@link Scorer} and iterates and
+   * collects the resulting hits.
+   *
+   * @param context
+   *          the {@link AtomicReaderContext} for which to return the {@link Scorer}.
    * @param scoreDocsInOrder
    *          specifies whether in-order scoring of documents is required. Note
    *          that if set to false (i.e., out-of-order scoring is required),
@@ -98,31 +117,70 @@ public abstract class Weight {
    *          in-order scorer is also an out-of-order one. However, an
    *          out-of-order scorer may not support {@link Scorer#nextDoc()}
    *          and/or {@link Scorer#advance(int)}, therefore it is recommended to
-   *          request an in-order scorer if use of these methods is required.
-   * @param topScorer
-   *          if true, {@link Scorer#score(Collector)} will be called; if false,
-   *          {@link Scorer#nextDoc()} and/or {@link Scorer#advance(int)} will
-   *          be called.
+   *          request an in-order scorer if use of these
+   *          methods is required.
    * @param acceptDocs
    *          Bits that represent the allowable docs to match (typically deleted docs
    *          but possibly filtering other documents)
-   *          
-   * @return a {@link Scorer} which scores documents in/out-of order.
+   *
+   * @return a {@link BulkScorer} which scores documents and
+   * passes them to a collector.
    * @throws IOException if there is a low-level I/O error
    */
-  public abstract Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-      boolean topScorer, Bits acceptDocs) throws IOException;
+  public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) throws IOException {
+
+    Scorer scorer = scorer(context, acceptDocs);
+    if (scorer == null) {
+      // No docs match
+      return null;
+    }
+
+    // This impl always scores docs in order, so we can
+    // ignore scoreDocsInOrder:
+    return new DefaultBulkScorer(scorer);
+  }
+
+  /** Just wraps a Scorer and performs top scoring using it. */
+  static class DefaultBulkScorer extends BulkScorer {
+    private final Scorer scorer;
+
+    public DefaultBulkScorer(Scorer scorer) {
+      assert scorer != null;
+      this.scorer = scorer;
+    }
+
+    @Override
+    public boolean score(Collector collector, int max) throws IOException {
+      // TODO: this may be sort of weird, when we are
+      // embedded in a BooleanScorer, because we are
+      // called for every chunk of 2048 documents.  But,
+      // then, scorer is a FakeScorer in that case, so any
+      // Collector doing something "interesting" in
+      // setScorer will be forced to use BS2 anyways:
+      collector.setScorer(scorer);
+      if (scorer.docID() == -1) {
+        scorer.nextDoc();
+      }
+      int doc;
+      for (doc = scorer.docID(); doc < max; doc = scorer.nextDoc()) {
+        collector.collect(doc);
+      }
+      return doc != DocsEnum.NO_MORE_DOCS;
+    }
+  }
 
   /**
    * Returns true iff this implementation scores docs only out of order. This
    * method is used in conjunction with {@link Collector}'s
    * {@link Collector#acceptsDocsOutOfOrder() acceptsDocsOutOfOrder} and
-   * {@link #scorer(AtomicReaderContext, boolean, boolean, Bits)} to
+   * {@link #bulkScorer(AtomicReaderContext, boolean, Bits)} to
    * create a matching {@link Scorer} instance for a given {@link Collector}, or
    * vice versa.
    * <p>
    * <b>NOTE:</b> the default implementation returns <code>false</code>, i.e.
    * the <code>Scorer</code> scores documents in-order.
    */
-  public boolean scoresDocsOutOfOrder() { return false; }
+  public boolean scoresDocsOutOfOrder() {
+    return false;
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/package.html Tue Mar 11 20:04:53 2014
@@ -53,8 +53,8 @@ org.apache.lucene.search.IndexSearcher#s
 <p>
 Once a Query has been created and submitted to the {@link org.apache.lucene.search.IndexSearcher IndexSearcher}, the scoring
 process begins. After some infrastructure setup, control finally passes to the {@link org.apache.lucene.search.Weight Weight}
-implementation and its {@link org.apache.lucene.search.Scorer Scorer} instances. See the <a href="#algorithm">Algorithm</a> 
-section for more notes on the process.
+implementation and its {@link org.apache.lucene.search.Scorer Scorer} or {@link org.apache.lucene.search.BulkScorer BulkScore}
+instances. See the <a href="#algorithm">Algorithm</a> section for more notes on the process.
 </p>
     <!-- FILL IN MORE HERE -->   
     <!-- TODO: this page over-links the same things too many times -->
@@ -370,6 +370,12 @@ on the built-in available scoring models
             {@link org.apache.lucene.search.Scorer Scorer} &mdash; An abstract class containing common
             functionality for scoring. Provides both scoring and
             explanation capabilities.  This is created per-segment.</li>
+        <li>
+            {@link org.apache.lucene.search.BulkScorer BulkScorer} &mdash; An abstract class that scores
+	    a range of documents.  A default implementation simply iterates through the hits from
+	    {@link org.apache.lucene.search.Scorer Scorer}, but some queries such as
+	    {@link org.apache.lucene.search.BooleanQuery BooleanQuery} have more efficient
+	    implementations.</li>
     </ol>
     Details on each of these classes, and their children, can be found in the subsections below.
 </p>
@@ -430,13 +436,19 @@ on the built-in available scoring models
                 that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will just defer to the Similarity's implementation:
                 {@link org.apache.lucene.search.similarities.Similarity.SimWeight#normalize SimWeight#normalize(float,float)}.</li>
             <li>
-                {@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, boolean, boolean, org.apache.lucene.util.Bits)
-                  scorer(AtomicReaderContext context, boolean scoresDocsInOrder, boolean topScorer, Bits acceptDocs)} &mdash;
+                {@link org.apache.lucene.search.Weight#scorer(org.apache.lucene.index.AtomicReaderContext, org.apache.lucene.util.Bits)
+                  scorer(AtomicReaderContext context, Bits acceptDocs)} &mdash;
                 Construct a new {@link org.apache.lucene.search.Scorer Scorer} for this Weight. See <a href="#scorerClass">The Scorer Class</a>
                 below for help defining a Scorer. As the name implies, the Scorer is responsible for doing the actual scoring of documents 
                 given the Query.
             </li>
             <li>
+                {@link org.apache.lucene.search.Weight#bulkScorer(org.apache.lucene.index.AtomicReaderContext, boolean, org.apache.lucene.util.Bits)
+                  scorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs)} &mdash;
+                Construct a new {@link org.apache.lucene.search.BulkScorer BulkScorer} for this Weight. See <a href="#bulkScorerClass">The BulkScorer Class</a>
+                below for help defining a BulkScorer. This is an optional method, and most queries do not implement it.
+            </li>
+            <li>
                 {@link org.apache.lucene.search.Weight#explain(org.apache.lucene.index.AtomicReaderContext, int) 
                   explain(AtomicReaderContext context, int doc)} &mdash; Provide a means for explaining why a given document was
                 scored the way it was.
@@ -489,6 +501,18 @@ on the built-in available scoring models
             </li>
         </ol>
     </p>
+<a name="bulkScorerClass"></a>
+<h4>The BulkScorer Class</h4>
+    <p>The
+        {@link org.apache.lucene.search.BulkScorer BulkScorer} scores a range of documents.  There is only one 
+        abstract method:
+        <ol>
+            <li>
+                {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector,int) score(Collector,int)} &mdash;
+		Score all documents up to but not including the specified max document.
+	    </li>
+        </ol>
+    </p>
 <h4>Why would I want to add my own Query?</h4>
 
     <p>In a nutshell, you want to add your own custom Query implementation when you think that Lucene's
@@ -539,7 +563,7 @@ on the built-in available scoring models
 <p>If a Filter is being used, some initial setup is done to determine which docs to include. 
    Otherwise, we ask the Weight for a {@link org.apache.lucene.search.Scorer Scorer} for each
    {@link org.apache.lucene.index.IndexReader IndexReader} segment and proceed by calling
-   {@link org.apache.lucene.search.Scorer#score(org.apache.lucene.search.Collector) Scorer.score()}.
+   {@link org.apache.lucene.search.BulkScorer#score(org.apache.lucene.search.Collector) BulkScorer.score(Collector)}.
 </p>
 <p>At last, we are actually going to score some documents. The score method takes in the Collector
    (most likely the TopScoreDocCollector or TopFieldCollector) and does its business.Of course, here 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Tue Mar 11 20:04:53 2014
@@ -148,15 +148,14 @@ public class PayloadNearQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this,
           similarity, similarity.simScorer(stats, context));
     }
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
+      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Tue Mar 11 20:04:53 2014
@@ -79,8 +79,7 @@ public class PayloadTermQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) throws IOException {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
       return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts),
           this, similarity.simScorer(stats, context));
     }
@@ -177,7 +176,7 @@ public class PayloadTermQuery extends Sp
     
     @Override
     public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
+      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs());
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Tue Mar 11 20:04:53 2014
@@ -81,8 +81,7 @@ public class SpanWeight extends Weight {
   }
 
   @Override
-  public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-      boolean topScorer, Bits acceptDocs) throws IOException {
+  public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) throws IOException {
     if (stats == null) {
       return null;
     } else {
@@ -92,7 +91,7 @@ public class SpanWeight extends Weight {
 
   @Override
   public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
-    SpanScorer scorer = (SpanScorer) scorer(context, true, false, context.reader().getLiveDocs());
+    SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs());
     if (scorer != null) {
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/JustCompileSearch.java Tue Mar 11 20:04:53 2014
@@ -17,8 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -232,11 +230,6 @@ final class JustCompileSearch {
     }
 
     @Override
-    public boolean score(Collector collector, int max, int firstDocID) {
-      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
-    }
-    
-    @Override
     public float score() {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
@@ -351,8 +344,7 @@ final class JustCompileSearch {
     }
 
     @Override
-    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
-        boolean topScorer, Bits acceptDocs) {
+    public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
     

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanOr.java Tue Mar 11 20:04:53 2014
@@ -183,7 +183,7 @@ public class TestBooleanOr extends Lucen
     Weight w = s.createNormalizedWeight(bq);
 
     assertEquals(1, s.getIndexReader().leaves().size());
-    Scorer scorer = w.scorer(s.getIndexReader().leaves().get(0), false, true, null);
+    BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), false, null);
 
     final FixedBitSet hits = new FixedBitSet(docCount);
     final AtomicInteger end = new AtomicInteger();
@@ -211,7 +211,7 @@ public class TestBooleanOr extends Lucen
     while (end.intValue() < docCount) {
       final int inc = TestUtil.nextInt(random(), 1, 1000);
       end.getAndAdd(inc);
-      scorer.score(c, end.intValue(), -1);
+      scorer.score(c, end.intValue());
     }
 
     assertEquals(docCount, hits.cardinality());

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQuery.java Tue Mar 11 20:04:53 2014
@@ -17,6 +17,7 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
@@ -29,6 +30,7 @@ import org.apache.lucene.analysis.MockAn
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -43,7 +45,6 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NamedThreadFactory;
 import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.TestUtil;
 
 public class TestBooleanQuery extends LuceneTestCase {
   
@@ -234,8 +235,7 @@ public class TestBooleanQuery extends Lu
 
       Weight weight = s.createNormalizedWeight(q);
 
-      Scorer scorer = weight.scorer(s.leafContexts.get(0),
-                                          true, false, null);
+      Scorer scorer = weight.scorer(s.leafContexts.get(0), null);
 
       // First pass: just use .nextDoc() to gather all hits
       final List<ScoreDoc> hits = new ArrayList<ScoreDoc>();
@@ -252,8 +252,7 @@ public class TestBooleanQuery extends Lu
       for(int iter2=0;iter2<10;iter2++) {
 
         weight = s.createNormalizedWeight(q);
-        scorer = weight.scorer(s.leafContexts.get(0),
-                               true, false, null);
+        scorer = weight.scorer(s.leafContexts.get(0), null);
 
         if (VERBOSE) {
           System.out.println("  iter2=" + iter2);
@@ -327,4 +326,30 @@ public class TestBooleanQuery extends Lu
     directory.close();
   }
 
+  // LUCENE-5487
+  public void testInOrderWithMinShouldMatch() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(newTextField("field", "some text here", Field.Store.NO));
+    w.addDocument(doc);
+    IndexReader r = w.getReader();
+    w.close();
+    IndexSearcher s = new IndexSearcher(r) {
+        @Override
+        protected void search(List<AtomicReaderContext> leaves, Weight weight, Collector collector) throws IOException {
+          assertEquals(-1, collector.getClass().getSimpleName().indexOf("OutOfOrder"));
+          super.search(leaves, weight, collector);
+        }
+      };
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new TermQuery(new Term("field", "some")), BooleanClause.Occur.SHOULD);
+    bq.add(new TermQuery(new Term("field", "text")), BooleanClause.Occur.SHOULD);
+    bq.add(new TermQuery(new Term("field", "here")), BooleanClause.Occur.SHOULD);
+    bq.setMinimumNumberShouldMatch(2);
+    s.search(bq, 10);
+    r.close();
+    dir.close();
+  }
+
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanScorer.java Tue Mar 11 20:04:53 2014
@@ -17,8 +17,10 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.document.Document;
@@ -30,10 +32,10 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanQuery.BooleanWeight;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.LuceneTestCase;
 
-public class TestBooleanScorer extends LuceneTestCase
-{
+public class TestBooleanScorer extends LuceneTestCase {
   private static final String FIELD = "category";
   
   public void testMethod() throws Exception {
@@ -78,27 +80,23 @@ public class TestBooleanScorer extends L
     writer.close();
     IndexSearcher searcher = newSearcher(ir);
     BooleanWeight weight = (BooleanWeight) new BooleanQuery().createWeight(searcher);
-    Scorer[] scorers = new Scorer[] {new Scorer(weight) {
+    BulkScorer[] scorers = new BulkScorer[] {new BulkScorer() {
       private int doc = -1;
-      @Override public float score() { return 0; }
-      @Override public int freq()  { return 0; }
-      @Override public int docID() { return doc; }
-      
-      @Override public int nextDoc() {
-        return doc = doc == -1 ? 3000 : NO_MORE_DOCS;
-      }
 
-      @Override public int advance(int target) {
-        return doc = target <= 3000 ? 3000 : NO_MORE_DOCS;
-      }
-      
       @Override
-      public long cost() {
-        return 1;
+      public boolean score(Collector c, int maxDoc) throws IOException {
+        assert doc == -1;
+        doc = 3000;
+        FakeScorer fs = new FakeScorer();
+        fs.doc = doc;
+        fs.score = 1.0f;
+        c.setScorer(fs);
+        c.collect(3000);
+        return false;
       }
     }};
     
-    BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), null, scorers.length);
+    BooleanScorer bs = new BooleanScorer(weight, false, 1, Arrays.asList(scorers), Collections.<BulkScorer>emptyList(), scorers.length);
 
     final List<Integer> hits = new ArrayList<Integer>();
     bs.score(new Collector() {
@@ -157,7 +155,7 @@ public class TestBooleanScorer extends L
       public void setScorer(Scorer scorer) {
         // Make sure we got BooleanScorer:
         final Class<?> clazz = scorer instanceof AssertingScorer ? ((AssertingScorer) scorer).getIn().getClass() : scorer.getClass();
-        assertEquals("Scorer is implemented by wrong class", BooleanScorer.class.getName() + "$BucketScorer", clazz.getName());
+        assertEquals("Scorer is implemented by wrong class", FakeScorer.class.getName(), clazz.getName());
       }
       
       @Override
@@ -180,4 +178,80 @@ public class TestBooleanScorer extends L
     r.close();
     d.close();
   }
+
+  /** Throws UOE if Weight.scorer is called */
+  private static class CrazyMustUseBulkScorerQuery extends Query {
+
+    @Override
+    public String toString(String field) {
+      return "MustUseBulkScorerQuery";
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher) throws IOException {
+      return new Weight() {
+        @Override
+        public Explanation explain(AtomicReaderContext context, int doc) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public Query getQuery() {
+          return CrazyMustUseBulkScorerQuery.this;
+        }
+
+        @Override
+        public float getValueForNormalization() {
+          return 1.0f;
+        }
+
+        @Override
+        public void normalize(float norm, float topLevelBoost) {
+        }
+
+        @Override
+        public Scorer scorer(AtomicReaderContext context, Bits acceptDocs) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public BulkScorer bulkScorer(AtomicReaderContext context, boolean scoreDocsInOrder, Bits acceptDocs) {
+          return new BulkScorer() {
+
+            @Override
+            public boolean score(Collector collector, int max) throws IOException {
+              collector.setScorer(new FakeScorer());
+              collector.collect(0);
+              return false;
+            }
+          };
+        }
+      };
+    }
+  }
+
+  /** Make sure BooleanScorer can embed another
+   *  BooleanScorer. */
+  public void testEmbeddedBooleanScorer() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(newTextField("field", "doctors are people who prescribe medicines of which they know little, to cure diseases of which they know less, in human beings of whom they know nothing", Field.Store.NO));
+    w.addDocument(doc);
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    BooleanQuery q1 = new BooleanQuery();
+    q1.add(new TermQuery(new Term("field", "little")), BooleanClause.Occur.SHOULD);
+    q1.add(new TermQuery(new Term("field", "diseases")), BooleanClause.Occur.SHOULD);
+
+    BooleanQuery q2 = new BooleanQuery();
+    q2.add(q1, BooleanClause.Occur.SHOULD);
+    q2.add(new CrazyMustUseBulkScorerQuery(), BooleanClause.Occur.SHOULD);
+
+    assertEquals(1, s.search(q2, 10).totalHits);
+    r.close();
+    dir.close();
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreQuery.java Tue Mar 11 20:04:53 2014
@@ -121,7 +121,7 @@ public class TestConstantScoreQuery exte
       checkHits(searcher, csq2, csq2.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), ConstantScoreQuery.ConstantScorer.class.getName());
       
       // for the combined BQ, the scorer should always be BooleanScorer's BucketScorer, because our scorer supports out-of order collection!
-      final String bucketScorerClass = BooleanScorer.class.getName() + "$BucketScorer";
+      final String bucketScorerClass = FakeScorer.class.getName();
       checkHits(searcher, bq, csq1.getBoost() + csq2.getBoost(), bucketScorerClass, null);
       checkHits(searcher, csqbq, csqbq.getBoost(), ConstantScoreQuery.ConstantScorer.class.getName(), bucketScorerClass);
     } finally {
@@ -158,7 +158,7 @@ public class TestConstantScoreQuery exte
   }
 
   // LUCENE-5307
-  // don't reuse the scorer of filters since they have been created with topScorer=false
+  // don't reuse the scorer of filters since they have been created with bulkScorer=false
   public void testQueryWrapperFilter() throws IOException {
     Directory d = newDirectory();
     RandomIndexWriter w = new RandomIndexWriter(random(), d);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java?rev=1576487&r1=1576486&r2=1576487&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDisjunctionMaxQuery.java Tue Mar 11 20:04:53 2014
@@ -180,7 +180,7 @@ public class TestDisjunctionMaxQuery ext
     assertTrue(s.getTopReaderContext() instanceof AtomicReaderContext);
     final Weight dw = s.createNormalizedWeight(dq);
     AtomicReaderContext context = (AtomicReaderContext)s.getTopReaderContext();
-    final Scorer ds = dw.scorer(context, true, false, context.reader().getLiveDocs());
+    final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
     final boolean skipOk = ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS;
     if (skipOk) {
       fail("firsttime skipTo found a match? ... "
@@ -196,7 +196,7 @@ public class TestDisjunctionMaxQuery ext
     QueryUtils.check(random(), dq, s);
     final Weight dw = s.createNormalizedWeight(dq);
     AtomicReaderContext context = (AtomicReaderContext)s.getTopReaderContext();
-    final Scorer ds = dw.scorer(context, true, false, context.reader().getLiveDocs());
+    final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
     assertTrue("firsttime skipTo found no match",
         ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals("found wrong docid", "d4", r.document(ds.docID()).get("id"));