You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2015/02/05 13:34:30 UTC

svn commit: r1657554 [1/2] - in /lucene/dev/trunk: lucene/ lucene/core/src/java/org/apache/lucene/codecs/lucene50/ lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/java/org/apache/lucene/search/payloads/ lucene/core/src/java/org/apache/lu...

Author: rmuir
Date: Thu Feb  5 12:34:28 2015
New Revision: 1657554

URL: http://svn.apache.org/r1657554
Log:
LUCENE-6218: don't decode freqs or enumerate all positions when scoring is not needed

Added:
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/BooleanQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.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/ExactPhraseScorer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCollector.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/MultiCollector.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/QueryRescorer.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/SloppyPhraseScorer.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/TimeLimitingCollector.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.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/index/TestOmitTf.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/MultiCollectorTest.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.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/TestBooleanQueryVisitSubscorers.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/TestCachingCollector.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/TestDocBoost.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.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/TestMultiTermConstantScore.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSloppyPhraseQuery.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/TestTimeLimitingCollector.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestTopDocsCollector.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/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/AssertingSubDocsAtOnceCollector.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/BlockGroupingCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupHeadsCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionAllGroupsCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionDistinctValuesCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionFirstPassGroupingCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/function/FunctionSecondPassGroupingCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupHeadsCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermAllGroupsCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermDistinctValuesCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermFirstPassGroupingCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermGroupFacetCollector.java
    lucene/dev/trunk/lucene/grouping/src/java/org/apache/lucene/search/grouping/term/TermSecondPassGroupingCollector.java
    lucene/dev/trunk/lucene/highlighter/src/test/org/apache/lucene/search/highlight/HighlighterPhraseTest.java
    lucene/dev/trunk/lucene/join/src/java/org/apache/lucene/search/join/TermsCollector.java
    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/TermsWithScoreCollector.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/ToParentBlockJoinIndexSearcher.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/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    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/queryparser/src/test/org/apache/lucene/queryparser/surround/query/BooleanQueryTst.java
    lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
    lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.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/CheckHits.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/QueryUtils.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/search/SearchEquivalenceTestBase.java
    lucene/dev/trunk/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/BasicAccumulator.java
    lucene/dev/trunk/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/FieldFacetAccumulator.java
    lucene/dev/trunk/solr/contrib/analytics/src/java/org/apache/solr/analytics/accumulator/facet/QueryFacetAccumulator.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/handler/component/ExpandComponent.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/DelegatingCollector.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/DocSetCollector.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/ExportQParserPlugin.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/ReRankQParserPlugin.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/core/src/java/org/apache/solr/update/DeleteByQueryWrapper.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestRankQueryPlugin.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Feb  5 12:34:28 2015
@@ -60,6 +60,9 @@ Optimizations
   are greater than one and is used when queries produce dense result sets.
   (Adrien Grand)
 
+* LUCENE-6218: Don't decode frequencies or match all positions when scoring
+  is not needed. (Robert Muir)
+
 API Changes
 
 * LUCENE-6204, LUCENE-6208: Simplify CompoundFormat: remove files()
@@ -68,6 +71,9 @@ API Changes
 * LUCENE-6217: Add IndexWriter.isOpen and getTragicException.  (Simon
   Willnauer, Mike McCandless)
 
+* LUCENE-6218: Add Collector.needsScores() and needsScores parameter
+  to Weight.scorer(). (Robert Muir)
+
 Other
 
 * LUCENE-6193: Collapse identical catch branches in try-catch statements.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java Thu Feb  5 12:34:28 2015
@@ -320,7 +320,7 @@ public final class Lucene50PostingsReade
 
       doc = -1;
       this.needsFreq = (flags & DocsEnum.FLAG_FREQS) != 0;
-      if (!indexHasFreq) {
+      if (indexHasFreq == false || needsFreq == false) {
         Arrays.fill(freqBuffer, 1);
       }
       accum = 0;

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -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, context.reader().getLiveDocs()) == null) {
+        if (w.scorer(context, context.reader().getLiveDocs(), true) == null) {
           if (c.isRequired()) {
             fail = true;
             Explanation r = new Explanation(0.0f, "no match on required clause (" + c.getQuery().toString() + ")");
@@ -307,12 +307,12 @@ public class BooleanQuery extends Query
     /** 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 {
+    BooleanScorer booleanScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       List<BulkScorer> optional = new ArrayList<BulkScorer>();
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
         BooleanClause c =  cIter.next();
-        BulkScorer subScorer = w.bulkScorer(context, acceptDocs);
+        BulkScorer subScorer = w.bulkScorer(context, acceptDocs, needsScores);
         if (subScorer == null) {
           if (c.isRequired()) {
             return null;
@@ -342,8 +342,8 @@ public class BooleanQuery extends Query
     }
 
     @Override
-    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs);
+    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
+      final BooleanScorer bulkScorer = booleanScorer(context, acceptDocs, needsScores);
       if (bulkScorer != null) { // BooleanScorer is applicable
         // TODO: what is the right heuristic here?
         final long costThreshold;
@@ -366,12 +366,11 @@ public class BooleanQuery extends Query
           return bulkScorer;
         }
       }
-      return super.bulkScorer(context, acceptDocs);
+      return super.bulkScorer(context, acceptDocs, needsScores);
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs)
-        throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       // initially the user provided value,
       // but if minNrShouldMatch == optional.size(),
       // we will optimize and move these to required, making this 0
@@ -383,7 +382,7 @@ public class BooleanQuery extends Query
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
         BooleanClause c =  cIter.next();
-        Scorer subScorer = w.scorer(context, acceptDocs);
+        Scorer subScorer = w.scorer(context, acceptDocs, needsScores && c.isProhibited() == false);
         if (subScorer == null) {
           if (c.isRequired()) {
             return null;
@@ -416,6 +415,11 @@ public class BooleanQuery extends Query
         return null;
       }
       
+      // we don't need scores, so if we have required clauses, drop optional clauses completely
+      if (!needsScores && minShouldMatch == 0 && required.size() > 0) {
+        optional.clear();
+      }
+      
       // three cases: conjunction, disjunction, or mix
       
       // pure conjunction

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java Thu Feb  5 12:34:28 2015
@@ -297,6 +297,11 @@ public abstract class CachingCollector e
       @Override
       public void collect(int doc) {}
 
+      @Override
+      public boolean needsScores() {
+        return true;
+      }
+
     };
     return create(other, cacheScores, maxRAMMB);
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java Thu Feb  5 12:34:28 2015
@@ -73,4 +73,10 @@ public interface Collector {
    */
   LeafCollector getLeafCollector(LeafReaderContext context) throws IOException;
 
+  /**
+   * Indicates if document scores are needed by this collector.
+   * 
+   * @return {@code true} if scores are needed.
+   */
+  boolean needsScores();
 }

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -134,14 +134,14 @@ public class ConstantScoreQuery extends
     }
 
     @Override
-    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
-        return super.bulkScorer(context, acceptDocs);
+        return super.bulkScorer(context, acceptDocs, needsScores);
       } else {
         assert query != null && innerWeight != null;
-        BulkScorer bulkScorer = innerWeight.bulkScorer(context, acceptDocs);
+        BulkScorer bulkScorer = innerWeight.bulkScorer(context, acceptDocs, false);
         if (bulkScorer == null) {
           return null;
         }
@@ -150,7 +150,7 @@ public class ConstantScoreQuery extends
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
@@ -161,7 +161,7 @@ public class ConstantScoreQuery extends
         disi = dis.iterator();
       } else {
         assert query != null && innerWeight != null;
-        disi = innerWeight.scorer(context, acceptDocs);
+        disi = innerWeight.scorer(context, acceptDocs, false);
       }
 
       if (disi == null) {
@@ -172,7 +172,7 @@ public class ConstantScoreQuery extends
 
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      final Scorer cs = scorer(context, context.reader().getLiveDocs());
+      final Scorer cs = scorer(context, context.reader().getLiveDocs(), true);
       final boolean exists = (cs != null && cs.advance(doc) == doc);
 
       final ComplexExplanation result = new ComplexExplanation();

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -153,11 +153,11 @@ public class DisjunctionMaxQuery extends
 
     /** Create the scorer used to score our associated DisjunctionMaxQuery */
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       List<Scorer> scorers = new ArrayList<>();
       for (Weight w : weights) {
         // we will advance() subscorers
-        Scorer subScorer = w.scorer(context, acceptDocs);
+        Scorer subScorer = w.scorer(context, acceptDocs, needsScores);
         if (subScorer != null) {
           scorers.add(subScorer);
         }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Thu Feb  5 12:34:28 2015
@@ -55,11 +55,13 @@ final class ExactPhraseScorer extends Sc
   private int freq;
 
   private final Similarity.SimScorer docScorer;
+  private final boolean needsScores;
   
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-                    Similarity.SimScorer docScorer) throws IOException {
+                    Similarity.SimScorer docScorer, boolean needsScores) throws IOException {
     super(weight);
     this.docScorer = docScorer;
+    this.needsScores = needsScores;
 
     chunkStates = new ChunkState[postings.length];
 
@@ -233,6 +235,9 @@ final class ExactPhraseScorer extends Sc
             final int posIndex = cs.pos - chunkStart;
             if (posIndex >= 0 && gens[posIndex] == gen && counts[posIndex] == endMinus1) {
               freq++;
+              if (!needsScores) {
+                return freq; // we determined there was a match.
+              }
             }
           }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterCollector.java Thu Feb  5 12:34:28 2015
@@ -44,5 +44,9 @@ public class FilterCollector implements
   public String toString() {
     return getClass().getSimpleName() + "(" + in + ")";
   }
-  
+
+  @Override
+  public boolean needsScores() {
+    return in.needsScores();
+  }
 }

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -119,7 +119,7 @@ public class FilteredQuery extends Query
 
       // return a filtering scorer
       @Override
-      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
         assert filter != null;
 
         DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
@@ -128,12 +128,12 @@ public class FilteredQuery extends Query
           return null;
         }
 
-        return strategy.filteredScorer(context, weight, filterDocIdSet);
+        return strategy.filteredScorer(context, weight, filterDocIdSet, needsScores);
       }
 
       // return a filtering top scorer
       @Override
-      public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+      public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
         assert filter != null;
 
         DocIdSet filterDocIdSet = filter.getDocIdSet(context, acceptDocs);
@@ -142,7 +142,7 @@ public class FilteredQuery extends Query
           return null;
         }
 
-        return strategy.filteredBulkScorer(context, weight, filterDocIdSet);
+        return strategy.filteredBulkScorer(context, weight, filterDocIdSet, needsScores);
       }
     };
   }
@@ -465,7 +465,7 @@ public class FilteredQuery extends Query
      * @throws IOException if an {@link IOException} occurs
      */
     public abstract Scorer filteredScorer(LeafReaderContext context,
-        Weight weight, DocIdSet docIdSet) throws IOException;
+        Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException;
 
     /**
      * Returns a filtered {@link BulkScorer} based on this
@@ -480,8 +480,8 @@ public class FilteredQuery extends Query
      * @return a filtered top scorer
      */
     public BulkScorer filteredBulkScorer(LeafReaderContext context,
-        Weight weight, DocIdSet docIdSet) throws IOException {
-      Scorer scorer = filteredScorer(context, weight, docIdSet);
+        Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException {
+      Scorer scorer = filteredScorer(context, weight, docIdSet, needsScores);
       if (scorer == null) {
         return null;
       }
@@ -502,7 +502,7 @@ public class FilteredQuery extends Query
   public static class RandomAccessFilterStrategy extends FilterStrategy {
 
     @Override
-    public Scorer filteredScorer(LeafReaderContext context, Weight weight, DocIdSet docIdSet) throws IOException {
+    public Scorer filteredScorer(LeafReaderContext context, Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException {
       final DocIdSetIterator filterIter = docIdSet.iterator();
       if (filterIter == null) {
         // this means the filter does not accept any documents.
@@ -514,11 +514,11 @@ public class FilteredQuery extends Query
       final boolean useRandomAccess = filterAcceptDocs != null && useRandomAccess(filterAcceptDocs, filterIter.cost());
       if (useRandomAccess) {
         // if we are using random access, we return the inner scorer, just with other acceptDocs
-        return weight.scorer(context, filterAcceptDocs);
+        return weight.scorer(context, filterAcceptDocs, needsScores);
       } else {
         // 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, null);
+        final Scorer scorer = weight.scorer(context, null, needsScores);
         return (scorer == null) ? null : new LeapFrogScorer(weight, filterIter, scorer, scorer);
       }
     }
@@ -551,14 +551,14 @@ public class FilteredQuery extends Query
 
     @Override
     public Scorer filteredScorer(LeafReaderContext context,
-        Weight weight, DocIdSet docIdSet) throws IOException {
+        Weight weight, DocIdSet docIdSet, boolean needsScores) throws IOException {
       final DocIdSetIterator filterIter = docIdSet.iterator();
       if (filterIter == null) {
         // this means the filter does not accept any documents.
         return null;
       }
       // we pass null as acceptDocs, as our filter has already respected acceptDocs, no need to do twice
-      final Scorer scorer = weight.scorer(context, null);
+      final Scorer scorer = weight.scorer(context, null, needsScores);
       if (scorer == null) {
         return null;
       }
@@ -587,30 +587,28 @@ public class FilteredQuery extends Query
   private static final class QueryFirstFilterStrategy extends FilterStrategy {
     @Override
     public Scorer filteredScorer(final LeafReaderContext context,
-        Weight weight,
-        DocIdSet docIdSet) throws IOException {
+        Weight weight, DocIdSet docIdSet, boolean needsScores) 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.filteredScorer(context, weight, docIdSet);
+        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet, needsScores);
       }
-      final Scorer scorer = weight.scorer(context, null);
+      final Scorer scorer = weight.scorer(context, null, needsScores);
       return scorer == null ? null : new QueryFirstScorer(weight,
           filterAcceptDocs, scorer);
     }
 
     @Override
     public BulkScorer filteredBulkScorer(final LeafReaderContext context,
-        Weight weight,
-        DocIdSet docIdSet) throws IOException {
+        Weight weight, DocIdSet docIdSet, boolean needsScores) 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, docIdSet);
+        return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredBulkScorer(context, weight, docIdSet, needsScores);
       }
-      final Scorer scorer = weight.scorer(context, null);
+      final Scorer scorer = weight.scorer(context, null, needsScores);
       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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -586,7 +586,7 @@ public class IndexSearcher {
         // continue with the following leaf
         continue;
       }
-      BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs());
+      BulkScorer scorer = weight.bulkScorer(ctx, ctx.reader().getLiveDocs(), collector.needsScores());
       if (scorer != null) {
         try {
           scorer.score(leafCollector);

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -114,7 +114,7 @@ public class MatchAllDocsQuery extends Q
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java Thu Feb  5 12:34:28 2015
@@ -93,6 +93,16 @@ public class MultiCollector implements C
   }
 
   @Override
+  public boolean needsScores() {
+    for (Collector collector : collectors) {
+      if (collector.needsScores()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  @Override
   public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
     final LeafCollector[] leafCollectors = new LeafCollector[collectors.length];
     for (int i = 0; i < collectors.length; ++i) {

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -179,7 +179,7 @@ public class MultiPhraseQuery extends Qu
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       assert !termArrays.isEmpty();
       final LeafReader reader = context.reader();
       final Bits liveDocs = acceptDocs;
@@ -249,15 +249,15 @@ public class MultiPhraseQuery extends Qu
       }
 
       if (slop == 0) {
-        return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context));
+        return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context), needsScores);
       } else {
-        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context));
+        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context), needsScores);
       }
     }
 
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs(), true);
       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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -245,7 +245,7 @@ public class PhraseQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       assert !terms.isEmpty();
       final LeafReader reader = context.reader();
       final Bits liveDocs = acceptDocs;
@@ -285,9 +285,9 @@ public class PhraseQuery extends Query {
       }
 
       if (slop == 0) {  // optimize exact case
-        return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context));
+        return new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context), needsScores);
       } else {
-        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context));
+        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context), needsScores);
       }
     }
     
@@ -298,7 +298,7 @@ public class PhraseQuery extends Query {
 
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs(), true);
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java Thu Feb  5 12:34:28 2015
@@ -82,7 +82,7 @@ public abstract class QueryRescorer exte
       if (readerContext != null) {
         // We advanced to another segment:
         docBase = readerContext.docBase;
-        scorer = weight.scorer(readerContext, null);
+        scorer = weight.scorer(readerContext, null, true);
       }
 
       if(scorer != null) {

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -56,7 +56,7 @@ public class QueryWrapperFilter extends
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {
-        return weight.scorer(privateContext, acceptDocs);
+        return weight.scorer(privateContext, acceptDocs, false);
       }
 
       @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Thu Feb  5 12:34:28 2015
@@ -50,11 +50,13 @@ final class SloppyPhraseScorer extends S
   
   private int numMatches;
   private final long cost;
+  final boolean needsScores;
   
   SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-      int slop, Similarity.SimScorer docScorer) {
+      int slop, Similarity.SimScorer docScorer, boolean needsScores) {
     super(weight);
     this.docScorer = docScorer;
+    this.needsScores = needsScores;
     this.slop = slop;
     this.numPostings = postings==null ? 0 : postings.length;
     pq = new PhraseQueue(postings.length);
@@ -114,6 +116,9 @@ final class SloppyPhraseScorer extends S
         if (matchLength <= slop) {
           freq += docScorer.computeSlopFactor(matchLength); // score match
           numMatches++;
+          if (!needsScores) {
+            return freq;
+          }
         }      
         pq.add(pp);
         pp = pq.pop();

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -75,13 +75,13 @@ public class TermQuery extends Query {
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) 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) {
         return null;
       }
-      DocsEnum docs = termsEnum.docs(acceptDocs, null);
+      DocsEnum docs = termsEnum.docs(acceptDocs, null, needsScores ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
       assert docs != null;
       return new TermScorer(this, docs, similarity.simScorer(stats, context));
     }
@@ -110,7 +110,7 @@ public class TermQuery extends Query {
     
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, context.reader().getLiveDocs());
+      Scorer scorer = scorer(context, context.reader().getLiveDocs(), true);
       if (scorer != null) {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java Thu Feb  5 12:34:28 2015
@@ -157,6 +157,11 @@ public class TimeLimitingCollector imple
     };
   }
   
+  @Override
+  public boolean needsScores() {
+    return collector.needsScores();
+  }
+
   /**
    * This is so the same timer can be used with a multi-phase search process such as grouping. 
    * We don't want to create a new TimeLimitingCollector for each phase because that would 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Thu Feb  5 12:34:28 2015
@@ -129,8 +129,8 @@ public abstract class TopFieldCollector
 
     final FieldValueHitQueue<Entry> queue;
 
-    public NonScoringCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
+    public NonScoringCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
+      super(queue, numHits, fillFields, sort.needsScores());
       this.queue = queue;
     }
 
@@ -216,8 +216,8 @@ public abstract class TopFieldCollector
 
     final FieldValueHitQueue<Entry> queue;
 
-    public ScoringNoMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
+    public ScoringNoMaxScoreCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
+      super(queue, numHits, fillFields, true);
       this.queue = queue;
     }
 
@@ -315,8 +315,8 @@ public abstract class TopFieldCollector
 
     final FieldValueHitQueue<Entry> queue;
 
-    public ScoringMaxScoreCollector(FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
-      super(queue, numHits, fillFields);
+    public ScoringMaxScoreCollector(Sort sort, FieldValueHitQueue<Entry> queue, int numHits, boolean fillFields) {
+      super(queue, numHits, fillFields, true);
       this.queue = queue;
       maxScore = Float.MIN_NORMAL; // otherwise we would keep NaN
     }
@@ -414,9 +414,9 @@ public abstract class TopFieldCollector
     final boolean trackMaxScore;
     final FieldDoc after;
 
-    public PagingFieldCollector(FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
+    public PagingFieldCollector(Sort sort, FieldValueHitQueue<Entry> queue, FieldDoc after, int numHits, boolean fillFields,
                                 boolean trackDocScores, boolean trackMaxScore) {
-      super(queue, numHits, fillFields);
+      super(queue, numHits, fillFields, trackDocScores || trackMaxScore || sort.needsScores());
       this.queue = queue;
       this.trackDocScores = trackDocScores;
       this.trackMaxScore = trackMaxScore;
@@ -520,18 +520,25 @@ public abstract class TopFieldCollector
   FieldValueHitQueue.Entry bottom = null;
   boolean queueFull;
   int docBase;
+  final boolean needsScores;
 
   // Declaring the constructor private prevents extending this class by anyone
   // else. Note that the class cannot be final since it's extended by the
   // internal versions. If someone will define a constructor with any other
   // visibility, then anyone will be able to extend the class, which is not what
   // we want.
-  private TopFieldCollector(PriorityQueue<Entry> pq, int numHits, boolean fillFields) {
+  private TopFieldCollector(PriorityQueue<Entry> pq, int numHits, boolean fillFields, boolean needsScores) {
     super(pq);
+    this.needsScores = needsScores;
     this.numHits = numHits;
     this.fillFields = fillFields;
   }
 
+  @Override
+  public boolean needsScores() {
+    return needsScores;
+  }
+
   /**
    * Creates a new {@link TopFieldCollector} from the given
    * arguments.
@@ -622,11 +629,11 @@ public abstract class TopFieldCollector
 
     if (after == null) {
       if (trackMaxScore) {
-        return new ScoringMaxScoreCollector(queue, numHits, fillFields);
+        return new ScoringMaxScoreCollector(sort, queue, numHits, fillFields);
       } else if (trackDocScores) {
-        return new ScoringNoMaxScoreCollector(queue, numHits, fillFields);
+        return new ScoringNoMaxScoreCollector(sort, queue, numHits, fillFields);
       } else {
-        return new NonScoringCollector(queue, numHits, fillFields);
+        return new NonScoringCollector(sort, queue, numHits, fillFields);
       }
     } else {
       if (after.fields == null) {
@@ -637,7 +644,7 @@ public abstract class TopFieldCollector
         throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length);
       }
 
-      return new PagingFieldCollector(queue, after, numHits, fillFields, trackDocScores, trackMaxScore);
+      return new PagingFieldCollector(sort, queue, after, numHits, fillFields, trackDocScores, trackMaxScore);
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TopScoreDocCollector.java Thu Feb  5 12:34:28 2015
@@ -207,4 +207,9 @@ public abstract class TopScoreDocCollect
 
     return new TopDocs(totalHits, results, maxScore);
   }
+
+  @Override
+  public boolean needsScores() {
+    return true;
+  }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java Thu Feb  5 12:34:28 2015
@@ -34,4 +34,9 @@ public class TotalHitCountCollector exte
   public void collect(int doc) {
     totalHits++;
   }
+
+  @Override
+  public boolean needsScores() {
+    return false;
+  }
 }

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -34,7 +34,7 @@ import org.apache.lucene.util.Bits;
  * {@link org.apache.lucene.index.LeafReader} dependent state should reside in the {@link Scorer}.
  * <p>
  * Since {@link Weight} creates {@link Scorer} instances for a given
- * {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits)})
+ * {@link org.apache.lucene.index.LeafReaderContext} ({@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits, boolean)})
  * callers must maintain the relationship between the searcher's top-level
  * {@link IndexReaderContext} and the context used to create a {@link Scorer}. 
  * <p>
@@ -49,7 +49,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(org.apache.lucene.index.LeafReaderContext, Bits)}.
+ * {@link #scorer(org.apache.lucene.index.LeafReaderContext, Bits, boolean)}.
  * </ol>
  * 
  * @since 2.9
@@ -87,11 +87,13 @@ public abstract class Weight {
    * @param acceptDocs
    *          Bits that represent the allowable docs to match (typically deleted docs
    *          but possibly filtering other documents)
+   * @param needsScores
+   *          True if document scores ({@link Scorer#score}) or match frequencies ({@link Scorer#freq}) are needed.
    *          
    * @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(LeafReaderContext context, Bits acceptDocs) throws IOException;
+  public abstract Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException;
 
   /**
    * Optional method, to return a {@link BulkScorer} to
@@ -106,14 +108,16 @@ public abstract class Weight {
    * @param acceptDocs
    *          Bits that represent the allowable docs to match (typically deleted docs
    *          but possibly filtering other documents)
+   * @param needsScores
+   *          True if document scores are needed.
    *
    * @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 BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+  public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
 
-    Scorer scorer = scorer(context, acceptDocs);
+    Scorer scorer = scorer(context, acceptDocs, needsScores);
     if (scorer == null) {
       // No docs match
       return null;

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -436,15 +436,13 @@ 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.LeafReaderContext, org.apache.lucene.util.Bits)
-                  scorer(LeafReaderContext context, Bits acceptDocs)} &mdash;
+                {@link org.apache.lucene.search.Weight#scorer scorer()} &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.LeafReaderContext, org.apache.lucene.util.Bits)
-                  scorer(LeafReaderContext context, Bits acceptDocs)} &mdash;
+                {@link org.apache.lucene.search.Weight#bulkScorer bulkScorer()} &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>

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -148,14 +148,14 @@ public class PayloadNearQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this,
           similarity, similarity.simScorer(stats, context));
     }
     
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs());
+      PayloadNearSpanScorer scorer = (PayloadNearSpanScorer) scorer(context, context.reader().getLiveDocs(), true);
       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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -79,7 +79,7 @@ public class PayloadTermQuery extends Sp
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
       return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts),
           this, similarity.simScorer(stats, context));
     }
@@ -176,7 +176,7 @@ public class PayloadTermQuery extends Sp
     
     @Override
     public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs());
+      PayloadTermSpanScorer scorer = (PayloadTermSpanScorer) scorer(context, context.reader().getLiveDocs(), true);
       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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -81,7 +81,7 @@ public class SpanWeight extends Weight {
   }
 
   @Override
-  public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+  public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
     if (stats == null) {
       return null;
     } else {
@@ -91,7 +91,7 @@ public class SpanWeight extends Weight {
 
   @Override
   public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-    SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs());
+    SpanScorer scorer = (SpanScorer) scorer(context, context.reader().getLiveDocs(), true);
     if (scorer != null) {
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/index/TestOmitTf.java Thu Feb  5 12:34:28 2015
@@ -437,6 +437,11 @@ public class TestOmitTf extends LuceneTe
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
       docBase = context.docBase;
     }
+    
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
   }
   
   /** test that when freqs are omitted, that totalTermFreq and sumTotalTermFreq are -1 */

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -53,6 +53,10 @@ final class JustCompileSearch {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
+    @Override
+    public boolean needsScores() {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
   }
   
   static final class JustCompileDocIdSet extends DocIdSet {
@@ -253,7 +257,11 @@ final class JustCompileSearch {
     public TopDocs topDocs( int start, int end ) {
         throw new UnsupportedOperationException( UNSUPPORTED_MSG );
     }
-    
+
+    @Override
+    public boolean needsScores() {
+      throw new UnsupportedOperationException( UNSUPPORTED_MSG );
+    }
   }
 
   static final class JustCompileWeight extends Weight {
@@ -279,7 +287,7 @@ final class JustCompileSearch {
     }
 
     @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
     

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/MultiCollectorTest.java Thu Feb  5 12:34:28 2015
@@ -46,6 +46,10 @@ public class MultiCollectorTest extends
       setScorerCalled = true;
     }
 
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
   }
 
   @Test

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanCoord.java Thu Feb  5 12:34:28 2015
@@ -707,14 +707,14 @@ public class TestBooleanCoord extends Lu
   private void assertScore(final float expected, Query query) throws Exception {
     // test in-order
     Weight weight = searcher.createNormalizedWeight(query);
-    Scorer scorer = weight.scorer(reader.leaves().get(0), null);
+    Scorer scorer = weight.scorer(reader.leaves().get(0), null, true);
     assertTrue(scorer.docID() == -1 || scorer.docID() == DocIdSetIterator.NO_MORE_DOCS);
     assertEquals(0, scorer.nextDoc());
     assertEquals(expected, scorer.score(), 0.0001f);
     
     // test bulk scorer
     final AtomicBoolean seen = new AtomicBoolean(false);
-    BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), null);
+    BulkScorer bulkScorer = weight.bulkScorer(reader.leaves().get(0), null, true);
     assertNotNull(bulkScorer);
     bulkScorer.score(new LeafCollector() {
       Scorer scorer;

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -189,7 +189,7 @@ public class TestBooleanOr extends Lucen
     Weight w = s.createNormalizedWeight(bq);
 
     assertEquals(1, s.getIndexReader().leaves().size());
-    BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null);
+    BulkScorer scorer = w.bulkScorer(s.getIndexReader().leaves().get(0), null, true);
 
     final FixedBitSet hits = new FixedBitSet(docCount);
     final AtomicInteger end = new AtomicInteger();
@@ -200,6 +200,11 @@ public class TestBooleanOr extends Lucen
           assertTrue("collected doc=" + doc + " beyond max=" + end, doc < end.intValue());
           hits.set(doc);
         }
+        
+        @Override
+        public boolean needsScores() {
+          return false;
+        }
       };
 
     while (end.intValue() < docCount) {

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -235,7 +235,7 @@ public class TestBooleanQuery extends Lu
 
       Weight weight = s.createNormalizedWeight(q);
 
-      Scorer scorer = weight.scorer(s.leafContexts.get(0), null);
+      Scorer scorer = weight.scorer(s.leafContexts.get(0), null, true);
 
       // First pass: just use .nextDoc() to gather all hits
       final List<ScoreDoc> hits = new ArrayList<>();
@@ -252,7 +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), null);
+        scorer = weight.scorer(s.leafContexts.get(0), null, true);
 
         if (VERBOSE) {
           System.out.println("  iter2=" + iter2);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestBooleanQueryVisitSubscorers.java Thu Feb  5 12:34:28 2015
@@ -234,6 +234,11 @@ public class TestBooleanQueryVisitSubsco
     public List<String> getSummaries() {
       return summaries;
     }
+    
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
 
     @Override
     public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
@@ -282,8 +287,8 @@ public class TestBooleanQueryVisitSubsco
     public Weight createWeight(IndexSearcher searcher) throws IOException {
       return new BooleanWeight(searcher, false) {
         @Override
-        public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-          Scorer scorer = scorer(context, acceptDocs);
+        public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
+          Scorer scorer = scorer(context, acceptDocs, needsScores);
           if (scorer == null) {
             return null;
           }

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -92,12 +92,12 @@ public class TestBooleanScorer extends L
         }
 
         @Override
-        public Scorer scorer(LeafReaderContext context, Bits acceptDocs) {
+        public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) {
           throw new UnsupportedOperationException();
         }
 
         @Override
-        public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) {
+        public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) {
           return new BulkScorer() {
             @Override
             public int score(LeafCollector collector, int min, int max) throws IOException {

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestCachingCollector.java Thu Feb  5 12:34:28 2015
@@ -56,6 +56,11 @@ public class TestCachingCollector extend
 
     @Override
     public void collect(int doc) throws IOException {}
+    
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
 
   }
 
@@ -79,6 +84,11 @@ public class TestCachingCollector extend
           assertEquals(prevDocID + 1, doc);
           prevDocID = doc;
         }
+        
+        @Override
+        public boolean needsScores() {
+          return false;
+        }
       });
     }
   }

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -67,6 +67,11 @@ public class TestConstantScoreQuery exte
         assertEquals("Score differs from expected", expectedScore, this.scorer.score(), 0);
         count[0]++;
       }
+      
+      @Override
+      public boolean needsScores() {
+        return true;
+      }
     });
     assertEquals("invalid number of results", 1, count[0]);
   }

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=1657554&r1=1657553&r2=1657554&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 Thu Feb  5 12:34:28 2015
@@ -180,7 +180,7 @@ public class TestDisjunctionMaxQuery ext
     assertTrue(s.getTopReaderContext() instanceof LeafReaderContext);
     final Weight dw = s.createNormalizedWeight(dq);
     LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
-    final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
+    final Scorer ds = dw.scorer(context, context.reader().getLiveDocs(), true);
     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);
     LeafReaderContext context = (LeafReaderContext)s.getTopReaderContext();
-    final Scorer ds = dw.scorer(context, context.reader().getLiveDocs());
+    final Scorer ds = dw.scorer(context, context.reader().getLiveDocs(), true);
     assertTrue("firsttime skipTo found no match",
         ds.advance(3) != DocIdSetIterator.NO_MORE_DOCS);
     assertEquals("found wrong docid", "d4", r.document(ds.docID()).get("id"));

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestDocBoost.java Thu Feb  5 12:34:28 2015
@@ -74,6 +74,10 @@ public class TestDocBoost extends Lucene
          protected void doSetNextReader(LeafReaderContext context) throws IOException {
            base = context.docBase;
          }
+         @Override
+         public boolean needsScores() {
+           return true;
+         }
        });
 
     float lastScore = 0.0f;

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestEarlyTermination.java Thu Feb  5 12:34:28 2015
@@ -80,7 +80,11 @@ public class TestEarlyTermination extend
             collectionTerminated = false;
           }
         }
-
+        
+        @Override
+        public boolean needsScores() {
+          return false;
+        }
       };
 
       searcher.search(new MatchAllDocsQuery(), collector);

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMinShouldMatch2.java Thu Feb  5 12:34:28 2015
@@ -131,11 +131,11 @@ public class TestMinShouldMatch2 extends
     case DOC_VALUES:
       return new SlowMinShouldMatchScorer(weight, reader, searcher);
     case SCORER:
-      return weight.scorer(reader.getContext(), null);
+      return weight.scorer(reader.getContext(), null, true);
     case BULK_SCORER:
-      final BulkScorer bulkScorer = weight.booleanScorer(reader.getContext(), null);
+      final BulkScorer bulkScorer = weight.booleanScorer(reader.getContext(), null, true);
       if (bulkScorer == null) {
-        if (weight.scorer(reader.getContext(), null) != null) {
+        if (weight.scorer(reader.getContext(), null, true) != null) {
           throw new AssertionError("BooleanScorer should be applicable for this query");
         }
         return null;

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestMultiTermConstantScore.java Thu Feb  5 12:34:28 2015
@@ -241,6 +241,11 @@ public class TestMultiTermConstantScore
       protected void doSetNextReader(LeafReaderContext context) throws IOException {
         base = context.docBase;
       }
+      
+      @Override
+      public boolean needsScores() {
+        return true;
+      }
     });
 
     //

Added: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java?rev=1657554&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java (added)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestNeedsScores.java Thu Feb  5 12:34:28 2015
@@ -0,0 +1,185 @@
+package org.apache.lucene.search;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestNeedsScores extends LuceneTestCase {
+  Directory dir;
+  IndexReader reader;
+  IndexSearcher searcher;
+  
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    for (int i = 0; i < 5; i++) {
+      Document doc = new Document();
+      doc.add(new TextField("field", "this is document " + i, Field.Store.NO));
+      iw.addDocument(doc);
+    }
+    reader = iw.getReader();
+    searcher = newSearcher(reader);
+    iw.close();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    IOUtils.close(reader, dir);
+    super.tearDown();
+  }
+
+  /** prohibited clauses in booleanquery don't need scoring */
+  public void testProhibitedClause() throws Exception {
+    Query required = new TermQuery(new Term("field", "this"));
+    Query prohibited = new TermQuery(new Term("field", "3"));
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(new AssertNeedsScores(required, true), BooleanClause.Occur.MUST);
+    bq.add(new AssertNeedsScores(prohibited, false), BooleanClause.Occur.MUST_NOT);
+    assertEquals(4, searcher.search(bq, 5).totalHits); // we exclude 3
+  }
+  
+  /** nested inside constant score query */
+  public void testConstantScoreQuery() throws Exception {
+    Query term = new TermQuery(new Term("field", "this"));
+    Query constantScore = new ConstantScoreQuery(new AssertNeedsScores(term, false));
+    assertEquals(5, searcher.search(constantScore, 5).totalHits);
+  }
+  
+  /** when converted to a filter */
+  public void testQueryWrapperFilter() throws Exception {
+    Query query = new MatchAllDocsQuery();
+    Query term = new TermQuery(new Term("field", "this"));
+    Filter filter = new QueryWrapperFilter(new AssertNeedsScores(term, false));
+    assertEquals(5, searcher.search(query, filter, 5).totalHits);
+  }
+  
+  /** when not sorting by score */
+  public void testSortByField() throws Exception {
+    Query query = new AssertNeedsScores(new MatchAllDocsQuery(), false);
+    assertEquals(5, searcher.search(query, 5, Sort.INDEXORDER).totalHits);
+  }
+  
+  /** when sorting by score */
+  public void testSortByScore() throws Exception {
+    Query query = new AssertNeedsScores(new MatchAllDocsQuery(), true);
+    assertEquals(5, searcher.search(query, 5, Sort.RELEVANCE).totalHits);
+  }
+
+  /** 
+   * Wraps a query, checking that the needsScores param 
+   * passed to Weight.scorer is the expected value.
+   */
+  static class AssertNeedsScores extends Query {
+    final Query in;
+    final boolean value;
+    
+    AssertNeedsScores(Query in, boolean value) {
+      this.in = in;
+      this.value = value;
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher) throws IOException {
+      final Weight w = in.createWeight(searcher);
+      return new Weight() {
+        @Override
+        public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+          return w.explain(context, doc);
+        }
+
+        @Override
+        public Query getQuery() {
+          return AssertNeedsScores.this;
+        }
+
+        @Override
+        public float getValueForNormalization() throws IOException {
+          return w.getValueForNormalization();
+        }
+
+        @Override
+        public void normalize(float norm, float topLevelBoost) {
+          w.normalize(norm, topLevelBoost);
+        }
+
+        @Override
+        public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
+          assertEquals("query=" + in, value, needsScores);
+          return w.scorer(context, acceptDocs, needsScores);
+        }
+      };
+    }
+
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+      Query in2 = in.rewrite(reader);
+      if (in2 == in) {
+        return this;
+      } else {
+        return new AssertNeedsScores(in2, value);
+      }
+    }
+
+    @Override
+    public void extractTerms(Set<Term> terms) {
+      in.extractTerms(terms);
+    }
+
+    @Override
+    public int hashCode() {
+      final int prime = 31;
+      int result = super.hashCode();
+      result = prime * result + ((in == null) ? 0 : in.hashCode());
+      result = prime * result + (value ? 1231 : 1237);
+      return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (!super.equals(obj)) return false;
+      if (getClass() != obj.getClass()) return false;
+      AssertNeedsScores other = (AssertNeedsScores) obj;
+      if (in == null) {
+        if (other.in != null) return false;
+      } else if (!in.equals(other.in)) return false;
+      if (value != other.value) return false;
+      return true;
+    }
+
+    @Override
+    public String toString(String field) {
+      return "asserting(" + in.toString(field) + ")";
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestQueryRescorer.java Thu Feb  5 12:34:28 2015
@@ -443,7 +443,7 @@ public class TestQueryRescorer extends L
         }
 
         @Override
-        public Scorer scorer(final LeafReaderContext context, Bits acceptDocs) throws IOException {
+        public Scorer scorer(final LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
 
           return new Scorer(null) {
             int docID = -1;

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScoreCachingWrappingScorer.java Thu Feb  5 12:34:28 2015
@@ -90,6 +90,11 @@ public class TestScoreCachingWrappingSco
     @Override public void setScorer(Scorer scorer) {
       this.scorer = new ScoreCachingWrappingScorer(scorer);
     }
+    
+    @Override
+    public boolean needsScores() {
+      return true;
+    }
 
   }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java Thu Feb  5 12:34:28 2015
@@ -115,6 +115,11 @@ public class TestScorerPerf extends Luce
     protected void doSetNextReader(LeafReaderContext context) throws IOException {
       docBase = context.docBase;
     }
+    
+    @Override
+    public boolean needsScores() {
+      return false;
+    }
   }
 
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java?rev=1657554&r1=1657553&r2=1657554&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestSimilarity.java Thu Feb  5 12:34:28 2015
@@ -86,7 +86,8 @@ public class TestSimilarity extends Luce
         public final void collect(int doc) throws IOException {
            assertEquals(1.0f, scorer.score(), 0);
          }
-        public boolean acceptsDocsOutOfOrder() {
+         @Override
+         public boolean needsScores() {
            return true;
          }
        });
@@ -111,6 +112,10 @@ public class TestSimilarity extends Luce
          protected void doSetNextReader(LeafReaderContext context) throws IOException {
            base = context.docBase;
          }
+         @Override
+         public boolean needsScores() {
+           return true;
+         }
        });
 
     PhraseQuery pq = new PhraseQuery();
@@ -129,6 +134,10 @@ public class TestSimilarity extends Luce
            //System.out.println("Doc=" + doc + " score=" + score);
            assertEquals(1.0f, scorer.score(), 0);
          }
+         @Override
+         public boolean needsScores() {
+           return true;
+         }
        });
 
     pq.setSlop(2);
@@ -144,6 +153,10 @@ public class TestSimilarity extends Luce
         //System.out.println("Doc=" + doc + " score=" + score);
         assertEquals(2.0f, scorer.score(), 0);
       }
+      @Override
+      public boolean needsScores() {
+        return true;
+      }
     });
 
     reader.close();