You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by da...@apache.org on 2018/11/20 15:16:24 UTC

[14/31] lucene-solr:jira/http2: LUCENE-8464: ConstantScoreScorer now implements setMinCompetitveScore in order to early terminate the iterator if the minimum score is greater than the constant score.

LUCENE-8464: ConstantScoreScorer now implements setMinCompetitveScore in order to early terminate the iterator if the minimum score is greater than the constant score.

Signed-off-by: Jim Ferenczi <ji...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/b4449c73
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/b4449c73
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/b4449c73

Branch: refs/heads/jira/http2
Commit: b4449c73e4c1ed34bc155ae5a818ac1a870ea7f8
Parents: 5aa6782
Author: Christophe Bismuth <ch...@gmail.com>
Authored: Thu Nov 8 16:06:32 2018 +0100
Committer: Jim Ferenczi <ji...@apache.org>
Committed: Fri Nov 16 15:14:11 2018 +0100

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |   4 +
 .../document/LatLonDocValuesBoxQuery.java       |   2 +-
 .../document/LatLonDocValuesDistanceQuery.java  |   2 +-
 .../document/LatLonPointDistanceQuery.java      |   4 +-
 .../document/LatLonPointInPolygonQuery.java     |   2 +-
 .../apache/lucene/document/RangeFieldQuery.java |   4 +-
 .../SortedNumericDocValuesRangeQuery.java       |   2 +-
 .../document/SortedSetDocValuesRangeQuery.java  |   2 +-
 .../lucene/search/ConstantScoreScorer.java      |  70 +++++-
 .../search/DocValuesFieldExistsQuery.java       |   2 +-
 .../lucene/search/DocValuesRewriteMethod.java   |   2 +-
 .../org/apache/lucene/search/LRUQueryCache.java |   4 +-
 .../apache/lucene/search/MatchAllDocsQuery.java |   2 +-
 .../MultiTermQueryConstantScoreWrapper.java     |   2 +-
 .../lucene/search/NormsFieldExistsQuery.java    |   2 +-
 .../apache/lucene/search/PointInSetQuery.java   |   2 +-
 .../apache/lucene/search/PointRangeQuery.java   |   7 +-
 .../apache/lucene/search/TermInSetQuery.java    |   2 +-
 .../lucene/search/TestConjunctionDISI.java      |  16 +-
 .../lucene/search/TestConstantScoreScorer.java  | 219 +++++++++++++++++++
 .../apache/lucene/search/TestLRUQueryCache.java |   4 +-
 .../apache/lucene/search/TestScorerPerf.java    |   2 +-
 .../apache/lucene/search/TestSortRandom.java    |   2 +-
 .../TestUsageTrackingFilterCachingPolicy.java   |   2 +-
 .../apache/lucene/facet/DrillSidewaysQuery.java |   2 +-
 .../apache/lucene/facet/range/DoubleRange.java  |   2 +-
 .../apache/lucene/facet/range/LongRange.java    |   2 +-
 .../apache/lucene/facet/TestDrillSideways.java  |   2 +-
 .../queries/function/FunctionMatchQuery.java    |   2 +-
 .../lucene/document/LatLonShapeQuery.java       |  25 +--
 .../lucene/search/DocValuesNumbersQuery.java    |   2 +-
 .../lucene/search/DocValuesTermsQuery.java      |   2 +-
 .../lucene/search/TestTermAutomatonQuery.java   |   2 +-
 .../spatial/composite/CompositeVerifyQuery.java |   2 +-
 .../composite/IntersectsRPTVerifyQuery.java     |   4 +-
 .../spatial/prefix/AbstractPrefixTreeQuery.java |   2 +-
 .../serialized/SerializedDVStrategy.java        |   2 +-
 .../spatial/vector/PointVectorStrategy.java     |   2 +-
 .../spatial3d/PointInGeo3DShapeQuery.java       |   2 +-
 .../org/apache/solr/query/SolrRangeQuery.java   |  12 +-
 .../src/java/org/apache/solr/search/Filter.java |   4 +-
 .../solr/search/GraphTermsQParserPlugin.java    |   4 +-
 .../apache/solr/search/JoinQParserPlugin.java   |  10 +-
 .../solr/search/SolrConstantScoreQuery.java     |   8 +-
 .../apache/solr/update/SolrIndexSplitter.java   |   2 +-
 .../uninverting/TestFieldCacheSortRandom.java   |   2 +-
 46 files changed, 370 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 4bb7b5b..611cb20 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -192,6 +192,10 @@ Optimizations
 * LUCENE-8507: TopFieldCollector can now update the minimum competitive score if the primary sort
   is by relevancy and the total hit count is not required. (Jim Ferenczi)
 
+* LUCENE-8464: ConstantScoreScorer now implements setMinCompetitveScore in order
+  to early terminate the iterator if the minimum score is greater than the constant
+  score. (Christophe Bismuth via Jim Ferenczi)
+
 ======================= Lucene 7.7.0 =======================
 
 Build

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesBoxQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesBoxQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesBoxQuery.java
index 31037f9..f34089ef 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesBoxQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesBoxQuery.java
@@ -139,7 +139,7 @@ final class LatLonDocValuesBoxQuery extends Query {
             return 5; // 5 comparisons
           }
         };
-        return new ConstantScoreScorer(this, boost, iterator);
+        return new ConstantScoreScorer(this, boost, scoreMode, iterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesDistanceQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesDistanceQuery.java
index df350e6..5fd1924 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesDistanceQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonDocValuesDistanceQuery.java
@@ -126,7 +126,7 @@ final class LatLonDocValuesDistanceQuery extends Query {
           }
 
         };
-        return new ConstantScoreScorer(this, boost, iterator);
+        return new ConstantScoreScorer(this, boost, scoreMode, iterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
index bebff6d..ca9ff22 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPointDistanceQuery.java
@@ -160,10 +160,10 @@ final class LatLonPointDistanceQuery extends Query {
               int[] cost = new int[]{reader.maxDoc()};
               values.intersect(getInverseIntersectVisitor(result, cost));
               final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
-              return new ConstantScoreScorer(weight, score(), iterator);
+              return new ConstantScoreScorer(weight, score(), scoreMode, iterator);
             }
             values.intersect(visitor);
-            return new ConstantScoreScorer(weight, score(), result.build().iterator());
+            return new ConstantScoreScorer(weight, score(), scoreMode, result.build().iterator());
           }
 
           @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java b/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
index c9ef44a..2e4d98d 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LatLonPointInPolygonQuery.java
@@ -158,7 +158,7 @@ final class LatLonPointInPolygonQuery extends Query {
                            }
                          });
 
-        return new ConstantScoreScorer(this, score(), result.build().iterator());
+        return new ConstantScoreScorer(this, score(), scoreMode, result.build().iterator());
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
index f623701..6b235fe 100644
--- a/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/RangeFieldQuery.java
@@ -314,7 +314,7 @@ abstract class RangeFieldQuery extends Query {
           return new ScorerSupplier() {
             @Override
             public Scorer get(long leadCost) {
-              return new ConstantScoreScorer(weight, score(), DocIdSetIterator.all(reader.maxDoc()));
+              return new ConstantScoreScorer(weight, score(), scoreMode, DocIdSetIterator.all(reader.maxDoc()));
             }
 
             @Override
@@ -333,7 +333,7 @@ abstract class RangeFieldQuery extends Query {
             public Scorer get(long leadCost) throws IOException {
               values.intersect(visitor);
               DocIdSetIterator iterator = result.build().iterator();
-              return new ConstantScoreScorer(weight, score(), iterator);
+              return new ConstantScoreScorer(weight, score(), scoreMode, iterator);
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesRangeQuery.java
index 246b50f..771af5c 100644
--- a/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/SortedNumericDocValuesRangeQuery.java
@@ -143,7 +143,7 @@ abstract class SortedNumericDocValuesRangeQuery extends Query {
             }
           };
         }
-        return new ConstantScoreScorer(this, score(), iterator);
+        return new ConstantScoreScorer(this, score(), scoreMode, iterator);
       }
 
     };

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesRangeQuery.java b/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesRangeQuery.java
index de7c11b..e5d365f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/document/SortedSetDocValuesRangeQuery.java
@@ -180,7 +180,7 @@ abstract class SortedSetDocValuesRangeQuery extends Query {
             }
           };
         }
-        return new ConstantScoreScorer(this, score(), iterator);
+        return new ConstantScoreScorer(this, score(), scoreMode, iterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java
index 45a6bdb..17c8362 100644
--- a/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java
+++ b/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java
@@ -25,7 +25,38 @@ import java.io.IOException;
  */
 public final class ConstantScoreScorer extends Scorer {
 
+  private class DocIdSetIteratorWrapper extends DocIdSetIterator {
+    int doc = -1;
+    DocIdSetIterator delegate;
+
+    DocIdSetIteratorWrapper(DocIdSetIterator delegate) {
+      this.delegate = delegate;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      return doc = delegate.nextDoc();
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      return doc = delegate.advance(target);
+    }
+
+    @Override
+    public long cost() {
+      return delegate.cost();
+    }
+  }
+
   private final float score;
+  private final ScoreMode scoreMode;
+  private final DocIdSetIterator approximation;
   private final TwoPhaseIterator twoPhaseIterator;
   private final DocIdSetIterator disi;
 
@@ -33,24 +64,45 @@ public final class ConstantScoreScorer extends Scorer {
    *  drive iteration. Two phase iteration will not be supported.
    *  @param weight the parent weight
    *  @param score the score to return on each document
+   *  @param scoreMode the score mode
    *  @param disi the iterator that defines matching documents */
-  public ConstantScoreScorer(Weight weight, float score, DocIdSetIterator disi) {
+  public ConstantScoreScorer(Weight weight, float score, ScoreMode scoreMode, DocIdSetIterator disi) {
     super(weight);
     this.score = score;
+    this.scoreMode = scoreMode;
+    this.approximation = scoreMode == ScoreMode.TOP_SCORES ? new DocIdSetIteratorWrapper(disi) : disi;
     this.twoPhaseIterator = null;
-    this.disi = disi;
+    this.disi = this.approximation;
   }
 
   /** Constructor based on a {@link TwoPhaseIterator}. In that case the
    *  {@link Scorer} will support two-phase iteration.
    *  @param weight the parent weight
    *  @param score the score to return on each document
+   *  @param scoreMode the score mode
    *  @param twoPhaseIterator the iterator that defines matching documents */
-  public ConstantScoreScorer(Weight weight, float score, TwoPhaseIterator twoPhaseIterator) {
+  public ConstantScoreScorer(Weight weight, float score, ScoreMode scoreMode, TwoPhaseIterator twoPhaseIterator) {
     super(weight);
     this.score = score;
-    this.twoPhaseIterator = twoPhaseIterator;
-    this.disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator);
+    this.scoreMode = scoreMode;
+    if (scoreMode == ScoreMode.TOP_SCORES) {
+      this.approximation = new DocIdSetIteratorWrapper(twoPhaseIterator.approximation());
+      this.twoPhaseIterator = new TwoPhaseIterator(this.approximation) {
+        @Override
+        public boolean matches() throws IOException {
+          return twoPhaseIterator.matches();
+        }
+
+        @Override
+        public float matchCost() {
+          return twoPhaseIterator.matchCost();
+        }
+      };
+    } else {
+      this.approximation = twoPhaseIterator.approximation();
+      this.twoPhaseIterator = twoPhaseIterator;
+    }
+    this.disi = TwoPhaseIterator.asDocIdSetIterator(this.twoPhaseIterator);
   }
 
   @Override
@@ -59,6 +111,13 @@ public final class ConstantScoreScorer extends Scorer {
   }
 
   @Override
+  public void setMinCompetitiveScore(float minScore) throws IOException {
+    if (scoreMode == ScoreMode.TOP_SCORES && minScore > score) {
+      ((DocIdSetIteratorWrapper) approximation).delegate = DocIdSetIterator.empty();
+    }
+  }
+
+  @Override
   public DocIdSetIterator iterator() {
     return disi;
   }
@@ -79,4 +138,3 @@ public final class ConstantScoreScorer extends Scorer {
   }
 
 }
-

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java b/lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java
index 54c8512..4b9c97d 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DocValuesFieldExistsQuery.java
@@ -68,7 +68,7 @@ public final class DocValuesFieldExistsQuery extends Query {
         if (iterator == null) {
           return null;
         }
-        return new ConstantScoreScorer(this, score(), iterator);
+        return new ConstantScoreScorer(this, score(), scoreMode, iterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
index 6e2ec37..d328b42 100644
--- a/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
+++ b/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
@@ -150,7 +150,7 @@ public final class DocValuesRewriteMethod extends MultiTermQuery.RewriteMethod {
             }
           } while (termsEnum.next() != null);
 
-          return new ConstantScoreScorer(this, score(), new TwoPhaseIterator(fcsi) {
+          return new ConstantScoreScorer(this, score(), scoreMode, new TwoPhaseIterator(fcsi) {
 
             @Override
             public boolean matches() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
index 511de47..c2448ed 100644
--- a/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
+++ b/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
@@ -766,7 +766,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
       return new ScorerSupplier() {
         @Override
         public Scorer get(long LeadCost) throws IOException {
-          return new ConstantScoreScorer(CachingWrapperWeight.this, 0f, disi);
+          return new ConstantScoreScorer(CachingWrapperWeight.this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi);
         }
         
         @Override
@@ -844,7 +844,7 @@ public class LRUQueryCache implements QueryCache, Accountable {
         return null;
       }
 
-      return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, disi));
+      return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, ScoreMode.COMPLETE_NO_SCORES, disi));
     }
 
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
index 7094bdf..826494e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
@@ -37,7 +37,7 @@ public final class MatchAllDocsQuery extends Query {
       }
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
-        return new ConstantScoreScorer(this, score(), DocIdSetIterator.all(context.reader().maxDoc()));
+        return new ConstantScoreScorer(this, score(), scoreMode, DocIdSetIterator.all(context.reader().maxDoc()));
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
index 17bdb56..9c37211 100644
--- a/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
+++ b/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
@@ -185,7 +185,7 @@ final class MultiTermQueryConstantScoreWrapper<Q extends MultiTermQuery> extends
         if (disi == null) {
           return null;
         }
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), scoreMode, disi);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/NormsFieldExistsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/NormsFieldExistsQuery.java b/lucene/core/src/java/org/apache/lucene/search/NormsFieldExistsQuery.java
index 74218b4..8c86885 100644
--- a/lucene/core/src/java/org/apache/lucene/search/NormsFieldExistsQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/NormsFieldExistsQuery.java
@@ -73,7 +73,7 @@ public final class NormsFieldExistsQuery extends Query {
         }
         LeafReader reader = context.reader();
         DocIdSetIterator iterator = reader.getNormValues(field);
-        return new ConstantScoreScorer(this, score(), iterator);
+        return new ConstantScoreScorer(this, score(), scoreMode, iterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
index 487bdfb..b6d047e 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointInSetQuery.java
@@ -149,7 +149,7 @@ public abstract class PointInSetQuery extends Query {
           }
         }
 
-        return new ConstantScoreScorer(this, score(), result.build().iterator());
+        return new ConstantScoreScorer(this, score(), scoreMode, result.build().iterator());
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
index 688a31f..ca61d51 100644
--- a/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/PointRangeQuery.java
@@ -264,8 +264,7 @@ public abstract class PointRangeQuery extends Query {
           return new ScorerSupplier() {
             @Override
             public Scorer get(long leadCost) {
-              return new ConstantScoreScorer(weight, score(),
-                  DocIdSetIterator.all(reader.maxDoc()));
+              return new ConstantScoreScorer(weight, score(), scoreMode, DocIdSetIterator.all(reader.maxDoc()));
             }
             
             @Override
@@ -293,12 +292,12 @@ public abstract class PointRangeQuery extends Query {
                 int[] cost = new int[] { reader.maxDoc() };
                 values.intersect(getInverseIntersectVisitor(result, cost));
                 final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
-                return new ConstantScoreScorer(weight, score(), iterator);
+                return new ConstantScoreScorer(weight, score(), scoreMode, iterator);
               }
 
               values.intersect(visitor);
               DocIdSetIterator iterator = result.build().iterator();
-              return new ConstantScoreScorer(weight, score(), iterator);
+              return new ConstantScoreScorer(weight, score(), scoreMode, iterator);
             }
             
             @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
index 9181668..814bea2 100644
--- a/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
+++ b/lucene/core/src/java/org/apache/lucene/search/TermInSetQuery.java
@@ -298,7 +298,7 @@ public class TermInSetQuery extends Query implements Accountable {
         if (disi == null) {
           return null;
         }
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), scoreMode, disi);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java b/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
index c10b78e..a9fd287 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestConjunctionDISI.java
@@ -234,12 +234,12 @@ public class TestConjunctionDISI extends LuceneTestCase {
           case 0:
             // simple iterator
             sets[i] = set;
-            iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, anonymizeIterator(new BitDocIdSet(set).iterator()));
+            iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, anonymizeIterator(new BitDocIdSet(set).iterator()));
             break;
           case 1:
             // bitSet iterator
             sets[i] = set;
-            iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator());
+            iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, new BitDocIdSet(set).iterator());
             break;
           default:
             // scorer with approximation
@@ -270,7 +270,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
         if (random().nextBoolean()) {
           // simple iterator
           sets[i] = set;
-          iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator());
+          iterators[i] = new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.COMPLETE_NO_SCORES, new BitDocIdSet(set).iterator());
         } else {
           // scorer with approximation
           final FixedBitSet confirmed = clearRandomBits(set);
@@ -306,12 +306,12 @@ public class TestConjunctionDISI extends LuceneTestCase {
           case 0:
             // simple iterator
             sets[i] = set;
-            newIterator = new ConstantScoreScorer(new FakeWeight(), 0f, anonymizeIterator(new BitDocIdSet(set).iterator()));
+            newIterator = new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, anonymizeIterator(new BitDocIdSet(set).iterator()));
             break;
           case 1:
             // bitSet iterator
             sets[i] = set;
-            newIterator = new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator());
+            newIterator = new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, new BitDocIdSet(set).iterator());
             break;
           default:
             // scorer with approximation
@@ -352,7 +352,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
         if (random().nextBoolean()) {
           // simple iterator
           sets[i] = set;
-          scorers.add(new ConstantScoreScorer(new FakeWeight(), 0f, new BitDocIdSet(set).iterator()));
+          scorers.add(new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, new BitDocIdSet(set).iterator()));
         } else {
           // scorer with approximation
           final FixedBitSet confirmed = clearRandomBits(set);
@@ -372,7 +372,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
         if (wrapWithScorer) {
           subConjunction = new ConjunctionScorer(new FakeWeight(), subIterators, Collections.emptyList());
         } else {
-          subConjunction = new ConstantScoreScorer(new FakeWeight(), 0f, ConjunctionDISI.intersectScorers(subIterators));
+          subConjunction = new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, ConjunctionDISI.intersectScorers(subIterators));
         }
         scorers.set(subSeqStart, subConjunction);
         int toRemove = subSeqEnd - subSeqStart - 1;
@@ -382,7 +382,7 @@ public class TestConjunctionDISI extends LuceneTestCase {
       }
       if (scorers.size() == 1) {
         // ConjunctionDISI needs two iterators
-        scorers.add(new ConstantScoreScorer(new FakeWeight(), 0f, DocIdSetIterator.all(maxDoc)));
+        scorers.add(new ConstantScoreScorer(new FakeWeight(), 0f, ScoreMode.TOP_SCORES, DocIdSetIterator.all(maxDoc)));
       }
 
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreScorer.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreScorer.java b/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreScorer.java
new file mode 100644
index 0000000..7230231
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/search/TestConstantScoreScorer.java
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+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.LuceneTestCase;
+
+import static org.apache.lucene.search.BooleanClause.Occur;
+import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
+import static org.hamcrest.CoreMatchers.equalTo;
+
+public class TestConstantScoreScorer extends LuceneTestCase {
+  private static final String FIELD = "f";
+  private static final String[] VALUES = new String[]{
+      "foo",
+      "bar",
+      "foo bar",
+      "bar foo",
+      "foo not bar",
+      "bar foo bar",
+      "azerty"
+  };
+
+  private static final Query TERM_QUERY = new BooleanQuery.Builder()
+      .add(new TermQuery(new Term(FIELD, "foo")), Occur.MUST)
+      .add(new TermQuery(new Term(FIELD, "bar")), Occur.MUST)
+      .build();
+  private static final Query PHRASE_QUERY = new PhraseQuery(FIELD, "foo", "bar");
+
+  public void testMatching_ScoreMode_COMPLETE() throws Exception {
+    testMatching(ScoreMode.COMPLETE);
+  }
+
+  public void testMatching_ScoreMode_COMPLETE_NO_SCORES() throws Exception {
+    testMatching(ScoreMode.COMPLETE_NO_SCORES);
+  }
+
+  private void testMatching(ScoreMode scoreMode) throws Exception {
+
+    try (TestConstantScoreScorerIndex index = new TestConstantScoreScorerIndex()) {
+      int doc;
+      ConstantScoreScorer scorer = index.constantScoreScorer(TERM_QUERY, 1f, scoreMode);
+
+      // "foo bar" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(2));
+      assertThat(scorer.score(), equalTo(1f));
+
+      // should not reset iterator
+      scorer.setMinCompetitiveScore(2f);
+      assertThat(scorer.docID(), equalTo(doc));
+      assertThat(scorer.iterator().docID(), equalTo(doc));
+      assertThat(scorer.score(), equalTo(1f));
+      
+      // "bar foo" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(3));
+      assertThat(scorer.score(), equalTo(1f));
+
+      // "foo not bar" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(4));
+      assertThat(scorer.score(), equalTo(1f));
+
+      // "foo bar foo" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(5));
+      assertThat(scorer.score(), equalTo(1f));
+      
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(NO_MORE_DOCS));
+    }
+  }
+
+  public void testMatching_ScoreMode_TOP_SCORES() throws Exception {
+    try (TestConstantScoreScorerIndex index = new TestConstantScoreScorerIndex()) {
+      int doc;
+      ConstantScoreScorer scorer = index.constantScoreScorer(TERM_QUERY, 1f, ScoreMode.TOP_SCORES);
+
+      // "foo bar" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(2));
+      assertThat(scorer.score(), equalTo(1f));
+
+      scorer.setMinCompetitiveScore(2f);
+      assertThat(scorer.docID(), equalTo(doc));
+      assertThat(scorer.iterator().docID(), equalTo(doc));
+      assertThat(scorer.score(), equalTo(1f));
+
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(NO_MORE_DOCS));
+    }
+  }
+
+  public void testTwoPhaseMatching_ScoreMode_COMPLETE() throws Exception {
+    testTwoPhaseMatching(ScoreMode.COMPLETE);
+  }
+
+  public void testTwoPhaseMatching_ScoreMode_COMPLETE_NO_SCORES() throws Exception {
+    testTwoPhaseMatching(ScoreMode.COMPLETE_NO_SCORES);
+  }
+
+  private void testTwoPhaseMatching(ScoreMode scoreMode) throws Exception {
+    try (TestConstantScoreScorerIndex index = new TestConstantScoreScorerIndex()) {
+      int doc;
+      ConstantScoreScorer scorer = index.constantScoreScorer(PHRASE_QUERY, 1f, scoreMode);
+
+      // "foo bar" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(2));
+      assertThat(scorer.score(), equalTo(1f));
+
+      // should not reset iterator
+      scorer.setMinCompetitiveScore(2f);
+      assertThat(scorer.docID(), equalTo(doc));
+      assertThat(scorer.iterator().docID(), equalTo(doc));
+      assertThat(scorer.score(), equalTo(1f));
+      
+      // "foo not bar" will match the approximation but not the two phase iterator
+
+      // "foo bar foo" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(5));
+      assertThat(scorer.score(), equalTo(1f));
+
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(NO_MORE_DOCS));
+    }
+  }
+
+  public void testTwoPhaseMatching_ScoreMode_TOP_SCORES() throws Exception {
+    try (TestConstantScoreScorerIndex index = new TestConstantScoreScorerIndex()) {
+      int doc;
+      ConstantScoreScorer scorer = index.constantScoreScorer(PHRASE_QUERY, 1f, ScoreMode.TOP_SCORES);
+
+      // "foo bar" match
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(2));
+      assertThat(scorer.score(), equalTo(1f));
+
+      scorer.setMinCompetitiveScore(2f);
+      assertThat(scorer.docID(), equalTo(doc));
+      assertThat(scorer.iterator().docID(), equalTo(doc));
+      assertThat(scorer.score(), equalTo(1f));
+
+      doc = scorer.iterator().nextDoc();
+      assertThat(doc, equalTo(NO_MORE_DOCS));
+    }
+  }
+
+  static class TestConstantScoreScorerIndex implements AutoCloseable {
+    private final Directory directory;
+    private final RandomIndexWriter writer;
+    private final IndexReader reader;
+
+    TestConstantScoreScorerIndex() throws IOException {
+      directory = newDirectory();
+
+      writer = new RandomIndexWriter(random(), directory,
+          newIndexWriterConfig().setMergePolicy(newLogMergePolicy(random().nextBoolean())));
+
+      for (String VALUE : VALUES) {
+        Document doc = new Document();
+        doc.add(newTextField(FIELD, VALUE, Field.Store.YES));
+        writer.addDocument(doc);
+      }
+      writer.forceMerge(1);
+
+      reader = writer.getReader();
+      writer.close();
+    }
+
+    ConstantScoreScorer constantScoreScorer(Query query, float score, ScoreMode scoreMode) throws IOException {
+      IndexSearcher searcher = newSearcher(reader);
+      Weight weight = searcher.createWeight(new ConstantScoreQuery(query), scoreMode, 1);
+      List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
+
+      assertThat(leaves.size(), equalTo(1));
+
+      LeafReaderContext context = leaves.get(0);
+      Scorer scorer = weight.scorer(context);
+
+      if (scorer.twoPhaseIterator() == null) {
+        return new ConstantScoreScorer(scorer.getWeight(), score, scoreMode, scorer.iterator());
+      } else {
+        return new ConstantScoreScorer(scorer.getWeight(), score, scoreMode, scorer.twoPhaseIterator());
+      }
+    }
+
+    @Override
+    public void close() throws Exception {
+      reader.close();
+      directory.close();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 5633607..2e3a4f1 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -1398,7 +1398,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
             @Override
             public Scorer get(long leadCost) throws IOException {
               scorerCreated.set(true);
-              return new ConstantScoreScorer(weight, boost, DocIdSetIterator.all(1));
+              return new ConstantScoreScorer(weight, boost, scoreMode, DocIdSetIterator.all(1));
             }
 
             @Override
@@ -1484,7 +1484,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
         @Override
         public Scorer scorer(LeafReaderContext context) throws IOException {
           scorerCreatedCount.incrementAndGet();
-          return new ConstantScoreScorer(this, 1, DocIdSetIterator.all(context.reader().maxDoc()));
+          return new ConstantScoreScorer(this, 1, scoreMode, DocIdSetIterator.all(context.reader().maxDoc()));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
index 59a246c..60363d2 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestScorerPerf.java
@@ -153,7 +153,7 @@ public class TestScorerPerf extends LuceneTestCase {
       return new ConstantScoreWeight(this, boost) {
         @Override
         public Scorer scorer(LeafReaderContext context) throws IOException {
-          return new ConstantScoreScorer(this, score(), new BitSetIterator(docs, docs.approximateCardinality()));
+          return new ConstantScoreScorer(this, score(), scoreMode, new BitSetIterator(docs, docs.approximateCardinality()));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java b/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java
index d1a852f..136026e 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortRandom.java
@@ -247,7 +247,7 @@ public class TestSortRandom extends LuceneTestCase {
             }
           }
 
-          return new ConstantScoreScorer(this, score(), new BitSetIterator(bits, bits.approximateCardinality()));
+          return new ConstantScoreScorer(this, score(), scoreMode, new BitSetIterator(bits, bits.approximateCardinality()));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
index 187accf..c817623 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestUsageTrackingFilterCachingPolicy.java
@@ -122,7 +122,7 @@ public class TestUsageTrackingFilterCachingPolicy extends LuceneTestCase {
       return new ConstantScoreWeight(DummyQuery.this, boost) {
         @Override
         public Scorer scorer(LeafReaderContext context) throws IOException {
-          return new ConstantScoreScorer(this, score(), DocIdSetIterator.all(1));
+          return new ConstantScoreScorer(this, score(), scoreMode, DocIdSetIterator.all(1));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
index 8630b7e..82c642a 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
@@ -123,7 +123,7 @@ class DrillSidewaysQuery extends Query {
           Scorer scorer = drillDowns[dim].scorer(context);
           if (scorer == null) {
             nullCount++;
-            scorer = new ConstantScoreScorer(drillDowns[dim], 0f, DocIdSetIterator.empty());
+            scorer = new ConstantScoreScorer(drillDowns[dim], 0f, scoreMode, DocIdSetIterator.empty());
           }
 
           dims[dim] = new DrillSidewaysScorer.DocsAndCost(scorer, drillSidewaysCollectors[dim]);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
index 3c51595..afc4c4f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
@@ -186,7 +186,7 @@ public final class DoubleRange extends Range {
               return 100; // TODO: use cost of range.accept()
             }
           };
-          return new ConstantScoreScorer(this, score(), twoPhase);
+          return new ConstantScoreScorer(this, score(), scoreMode, twoPhase);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
index 6b711aa..eed293f 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
@@ -178,7 +178,7 @@ public final class LongRange extends Range {
               return 100; // TODO: use cost of range.accept()
             }
           };
-          return new ConstantScoreScorer(this, score(), twoPhase);
+          return new ConstantScoreScorer(this, score(), scoreMode, twoPhase);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
----------------------------------------------------------------------
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java b/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
index ad7754f..288f6cc 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestDrillSideways.java
@@ -726,7 +726,7 @@ public class TestDrillSideways extends FacetTestCase {
               @Override
               public Scorer scorer(LeafReaderContext context) throws IOException {
                 DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
-                return new ConstantScoreScorer(this, score(), new TwoPhaseIterator(approximation) {
+                return new ConstantScoreScorer(this, score(), scoreMode, new TwoPhaseIterator(approximation) {
 
                   @Override
                   public boolean matches() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionMatchQuery.java
----------------------------------------------------------------------
diff --git a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionMatchQuery.java b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionMatchQuery.java
index 7a83734..38cf3f6 100644
--- a/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionMatchQuery.java
+++ b/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionMatchQuery.java
@@ -80,7 +80,7 @@ public final class FunctionMatchQuery extends Query {
             return 100; // TODO maybe DoubleValuesSource should have a matchCost?
           }
         };
-        return new ConstantScoreScorer(this, score(), twoPhase);
+        return new ConstantScoreScorer(this, score(), scoreMode, twoPhase);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
index 454b2b8..8aa932f 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/document/LatLonShapeQuery.java
@@ -151,21 +151,21 @@ abstract class LatLonShapeQuery extends Query {
       }
 
       /** get a scorer supplier for INTERSECT queries */
-      protected ScorerSupplier getIntersectScorerSupplier(LeafReader reader, PointValues values, Weight weight) throws IOException {
+      protected ScorerSupplier getIntersectScorerSupplier(LeafReader reader, PointValues values, Weight weight, ScoreMode scoreMode) throws IOException {
         DocIdSetBuilder result = new DocIdSetBuilder(reader.maxDoc(), values, field);
         IntersectVisitor visitor = getSparseIntersectVisitor(result);
         return new RelationScorerSupplier(values, visitor) {
           @Override
           public Scorer get(long leadCost) throws IOException {
-            return getIntersectsScorer(LatLonShapeQuery.this, reader, weight, result, score());
+            return getIntersectsScorer(LatLonShapeQuery.this, reader, weight, result, score(), scoreMode);
           }
         };
       }
 
       /** get a scorer supplier for all other queries (DISJOINT, WITHIN) */
-      protected ScorerSupplier getScorerSupplier(LeafReader reader, PointValues values, Weight weight) throws IOException {
+      protected ScorerSupplier getScorerSupplier(LeafReader reader, PointValues values, Weight weight, ScoreMode scoreMode) throws IOException {
         if (queryRelation == QueryRelation.INTERSECTS) {
-          return getIntersectScorerSupplier(reader, values, weight);
+          return getIntersectScorerSupplier(reader, values, weight, scoreMode);
         }
 
         FixedBitSet intersect = new FixedBitSet(reader.maxDoc());
@@ -174,7 +174,7 @@ abstract class LatLonShapeQuery extends Query {
         return new RelationScorerSupplier(values, visitor) {
           @Override
           public Scorer get(long leadCost) throws IOException {
-            return getScorer(LatLonShapeQuery.this, weight, intersect, disjoint, score());
+            return getScorer(LatLonShapeQuery.this, weight, intersect, disjoint, score(), scoreMode);
           }
         };
       }
@@ -204,8 +204,7 @@ abstract class LatLonShapeQuery extends Query {
           return new ScorerSupplier() {
             @Override
             public Scorer get(long leadCost) throws IOException {
-              return new ConstantScoreScorer(weight, score(),
-                  DocIdSetIterator.all(reader.maxDoc()));
+              return new ConstantScoreScorer(weight, score(), scoreMode, DocIdSetIterator.all(reader.maxDoc()));
             }
 
             @Override
@@ -214,7 +213,7 @@ abstract class LatLonShapeQuery extends Query {
             }
           };
         } else {
-          return getScorerSupplier(reader, values, weight);
+          return getScorerSupplier(reader, values, weight, scoreMode);
         }
       }
 
@@ -309,7 +308,7 @@ abstract class LatLonShapeQuery extends Query {
 
     /** returns a Scorer for INTERSECT queries that uses a sparse bitset */
     protected Scorer getIntersectsScorer(LatLonShapeQuery query, LeafReader reader, Weight weight,
-                                         DocIdSetBuilder docIdSetBuilder, final float boost) throws IOException {
+                                         DocIdSetBuilder docIdSetBuilder, final float boost, ScoreMode scoreMode) throws IOException {
       if (values.getDocCount() == reader.maxDoc()
           && values.getDocCount() == values.size()
           && cost() > reader.maxDoc() / 2) {
@@ -321,17 +320,17 @@ abstract class LatLonShapeQuery extends Query {
         int[] cost = new int[]{reader.maxDoc()};
         values.intersect(getInverseIntersectVisitor(query, result, cost));
         final DocIdSetIterator iterator = new BitSetIterator(result, cost[0]);
-        return new ConstantScoreScorer(weight, boost, iterator);
+        return new ConstantScoreScorer(weight, boost, scoreMode, iterator);
       }
 
       values.intersect(visitor);
       DocIdSetIterator iterator = docIdSetBuilder.build().iterator();
-      return new ConstantScoreScorer(weight, boost, iterator);
+      return new ConstantScoreScorer(weight, boost, scoreMode, iterator);
     }
 
     /** returns a Scorer for all other (non INTERSECT) queries */
     protected Scorer getScorer(LatLonShapeQuery query, Weight weight,
-                               FixedBitSet intersect, FixedBitSet disjoint, final float boost) throws IOException {
+                               FixedBitSet intersect, FixedBitSet disjoint, final float boost, ScoreMode scoreMode) throws IOException {
       values.intersect(visitor);
       DocIdSetIterator iterator;
       if (query.queryRelation == QueryRelation.DISJOINT) {
@@ -343,7 +342,7 @@ abstract class LatLonShapeQuery extends Query {
       } else {
         iterator = new BitSetIterator(intersect, cost());
       }
-      return new ConstantScoreScorer(weight, boost, iterator);
+      return new ConstantScoreScorer(weight, boost, scoreMode, iterator);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
index e72e992..f018df4 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesNumbersQuery.java
@@ -102,7 +102,7 @@ public class DocValuesNumbersQuery extends Query {
       @Override
       public Scorer scorer(LeafReaderContext context) throws IOException {
         final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), field);
-        return new ConstantScoreScorer(this, score(), new TwoPhaseIterator(values) {
+        return new ConstantScoreScorer(this, score(), scoreMode, new TwoPhaseIterator(values) {
 
           @Override
           public boolean matches() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
index 0e615b4..8ca53b7 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
@@ -184,7 +184,7 @@ public class DocValuesTermsQuery extends Query {
         if (matchesAtLeastOneTerm == false) {
           return null;
         }
-        return new ConstantScoreScorer(this, score(), new TwoPhaseIterator(values) {
+        return new ConstantScoreScorer(this, score(), scoreMode, new TwoPhaseIterator(values) {
 
           @Override
           public boolean matches() throws IOException {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
----------------------------------------------------------------------
diff --git a/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java b/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
index dab78f9..0d7ce58 100644
--- a/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
+++ b/lucene/sandbox/src/test/org/apache/lucene/search/TestTermAutomatonQuery.java
@@ -595,7 +595,7 @@ public class TestTermAutomatonQuery extends LuceneTestCase {
               //System.out.println("  acc id=" + idSource.getInt(docID) + " docID=" + docID);
             }
           }
-          return new ConstantScoreScorer(this, score(), new BitSetIterator(bits, bits.approximateCardinality()));
+          return new ConstantScoreScorer(this, score(), scoreMode, new BitSetIterator(bits, bits.approximateCardinality()));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
index 2bfa4d5..cadacdc 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
@@ -95,7 +95,7 @@ public class CompositeVerifyQuery extends Query {
         }
 
         final TwoPhaseIterator predFuncValues = predicateValueSource.iterator(context, indexQueryScorer.iterator());
-        return new ConstantScoreScorer(this, score(), predFuncValues);
+        return new ConstantScoreScorer(this, score(), scoreMode, predFuncValues);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
index e6324da..1bcb7b5 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
@@ -100,7 +100,7 @@ public class IntersectsRPTVerifyQuery extends Query {
         if (result.exactDocIdSet != null) {
           // If both sets are the same, there's nothing to verify; we needn't return a TwoPhaseIterator
           if (result.approxDocIdSet == result.exactDocIdSet) {
-            return new ConstantScoreScorer(this, score(), approxDISI);
+            return new ConstantScoreScorer(this, score(), scoreMode, approxDISI);
           }
           exactIterator = result.exactDocIdSet.iterator();
           assert exactIterator != null;
@@ -132,7 +132,7 @@ public class IntersectsRPTVerifyQuery extends Query {
           }
         };
 
-        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
+        return new ConstantScoreScorer(this, score(), scoreMode, twoPhaseIterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
index 75b3c2b..a5171c3 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
@@ -89,7 +89,7 @@ public abstract class AbstractPrefixTreeQuery extends Query {
         if (disi == null) {
           return null;
         }
-        return new ConstantScoreScorer(this, score(), disi);
+        return new ConstantScoreScorer(this, score(), scoreMode, disi);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
index cd94bf4..ff019c0 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java
@@ -140,7 +140,7 @@ public class SerializedDVStrategy extends SpatialStrategy {
         public Scorer scorer(LeafReaderContext context) throws IOException {
           DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
           TwoPhaseIterator it = predicateValueSource.iterator(context, approximation);
-          return new ConstantScoreScorer(this, score(), it);
+          return new ConstantScoreScorer(this, score(), scoreMode, it);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
index f06770e..9de43d1 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
@@ -287,7 +287,7 @@ public class PointVectorStrategy extends SpatialStrategy {
               return 100;   // distance calculation can be heavy!
             }
           };
-          return new ConstantScoreScorer(this, score(), twoPhase);
+          return new ConstantScoreScorer(this, score(), scoreMode, twoPhase);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
index 83a471f..8fcf056 100644
--- a/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
+++ b/lucene/spatial3d/src/java/org/apache/lucene/spatial3d/PointInGeo3DShapeQuery.java
@@ -102,7 +102,7 @@ final class PointInGeo3DShapeQuery extends Query {
 
         values.intersect(new PointInShapeIntersectVisitor(result, shape, shapeBounds));
 
-        return new ConstantScoreScorer(this, score(), result.build().iterator());
+        return new ConstantScoreScorer(this, score(), scoreMode, result.build().iterator());
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java b/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
index 7735600..21ccf03 100644
--- a/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
+++ b/solr/core/src/java/org/apache/solr/query/SolrRangeQuery.java
@@ -142,7 +142,7 @@ public final class SolrRangeQuery extends ExtendedQueryBase implements DocSetPro
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    return new ConstWeight(searcher, scoreMode.needsScores(), boost);
+    return new ConstWeight(searcher, scoreMode, boost);
     /*
     DocSet docs = createDocSet(searcher.getIndexReader().leaves(), searcher.getIndexReader().maxDoc());
     SolrConstantScoreQuery csq = new SolrConstantScoreQuery( docs.getTopFilter() );
@@ -327,17 +327,17 @@ public final class SolrRangeQuery extends ExtendedQueryBase implements DocSetPro
     private static final int BOOLEAN_REWRITE_TERM_COUNT_THRESHOLD = 16;
 
     final IndexSearcher searcher;
-    final boolean needScores;
+    final ScoreMode scoreMode;
     boolean checkedFilterCache;
     Filter filter;
     final SegState[] segStates;
 
 
-    protected ConstWeight(IndexSearcher searcher, boolean needScores, float boost) {
+    protected ConstWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) {
       super( SolrRangeQuery.this, boost );
       this.searcher = searcher;
       this.segStates = new SegState[ searcher.getIndexReader().leaves().size() ];
-      this.needScores = needScores;
+      this.scoreMode = scoreMode;
     }
 
 
@@ -407,7 +407,7 @@ public final class SolrRangeQuery extends ExtendedQueryBase implements DocSetPro
           bq.add(new TermQuery(new Term( SolrRangeQuery.this.getField(), t.term), termStates), BooleanClause.Occur.SHOULD);
         }
         Query q = new ConstantScoreQuery(bq.build());
-        final Weight weight = searcher.rewrite(q).createWeight(searcher, needScores ? ScoreMode.COMPLETE : ScoreMode.COMPLETE_NO_SCORES, score());
+        final Weight weight = searcher.rewrite(q).createWeight(searcher, scoreMode, score());
         return segStates[context.ord] = new SegState(weight);
       }
 
@@ -467,7 +467,7 @@ public final class SolrRangeQuery extends ExtendedQueryBase implements DocSetPro
       if (disi == null) {
         return null;
       }
-      return new ConstantScoreScorer(this, score(), disi);
+      return new ConstantScoreScorer(this, score(), scoreMode, disi);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/java/org/apache/solr/search/Filter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/Filter.java b/solr/core/src/java/org/apache/solr/search/Filter.java
index 847ef46..4a48bb6 100644
--- a/solr/core/src/java/org/apache/solr/search/Filter.java
+++ b/solr/core/src/java/org/apache/solr/search/Filter.java
@@ -127,13 +127,13 @@ public abstract class Filter extends Query {
               return 10; // TODO use cost of bits.get()
             }
           };
-          return new ConstantScoreScorer(this, 0f, twoPhase);
+          return new ConstantScoreScorer(this, 0f, scoreMode, twoPhase);
         }
         final DocIdSetIterator iterator = set.iterator();
         if (iterator == null) {
           return null;
         }
-        return new ConstantScoreScorer(this, 0f, iterator);
+        return new ConstantScoreScorer(this, 0f, scoreMode, iterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
index b0a0699..0ce7f72 100644
--- a/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/GraphTermsQParserPlugin.java
@@ -305,7 +305,7 @@ public class GraphTermsQParserPlugin extends QParserPlugin {
           if (disi == null) {
             return null;
           }
-          return new ConstantScoreScorer(this, score(), disi);
+          return new ConstantScoreScorer(this, score(), scoreMode, disi);
         }
 
         @Override
@@ -624,7 +624,7 @@ abstract class PointSetQuery extends Query implements DocSetProducer {
         if (readerSetIterator == null) {
           return null;
         }
-        return new ConstantScoreScorer(this, score(), readerSetIterator);
+        return new ConstantScoreScorer(this, score(), scoreMode, readerSetIterator);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
index 1890ea8..4e832a0 100644
--- a/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
@@ -39,7 +39,6 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ScoreMode;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
@@ -167,18 +166,19 @@ class JoinQuery extends Query {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    return new JoinQueryWeight((SolrIndexSearcher)searcher, boost);
+    return new JoinQueryWeight((SolrIndexSearcher) searcher, scoreMode, boost);
   }
 
   private class JoinQueryWeight extends ConstantScoreWeight {
     SolrIndexSearcher fromSearcher;
     RefCounted<SolrIndexSearcher> fromRef;
     SolrIndexSearcher toSearcher;
-    private Similarity similarity;
     ResponseBuilder rb;
+    ScoreMode scoreMode;
 
-    public JoinQueryWeight(SolrIndexSearcher searcher, float boost) {
+    public JoinQueryWeight(SolrIndexSearcher searcher, ScoreMode scoreMode, float boost) {
       super(JoinQuery.this, boost);
+      this.scoreMode = scoreMode;
       this.fromSearcher = searcher;
       SolrRequestInfo info = SolrRequestInfo.getRequestInfo();
       if (info != null) {
@@ -280,7 +280,7 @@ class JoinQuery extends Query {
       if (readerSetIterator == null) {
         return null;
       }
-      return new ConstantScoreScorer(this, score(), readerSetIterator);
+      return new ConstantScoreScorer(this, score(), scoreMode, readerSetIterator);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java b/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
index 1f81e1e..b032dc0 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
@@ -84,9 +84,11 @@ public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
 
   protected class ConstantWeight extends ConstantScoreWeight {
     private Map context;
+    private ScoreMode scoreMode;
 
-    public ConstantWeight(IndexSearcher searcher, float boost) throws IOException {
+    public ConstantWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
       super(SolrConstantScoreQuery.this, boost);
+      this.scoreMode = scoreMode;
       this.context = ValueSource.newContext(searcher);
       if (filter instanceof SolrFilter)
         ((SolrFilter)filter).createWeight(context, searcher);
@@ -102,7 +104,7 @@ public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
       if (iterator == null) {
         return null;
       }
-      return new ConstantScoreScorer(this, score(), iterator);
+      return new ConstantScoreScorer(this, score(), scoreMode, iterator);
     }
 
     @Override
@@ -114,7 +116,7 @@ public class SolrConstantScoreQuery extends Query implements ExtendedQuery {
 
   @Override
   public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
-    return new SolrConstantScoreQuery.ConstantWeight(searcher, boost);
+    return new SolrConstantScoreQuery.ConstantWeight(searcher, scoreMode, boost);
   }
 
   /** Prints a user-readable version of this query. */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java b/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
index 334a29d..497a301 100644
--- a/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
+++ b/solr/core/src/java/org/apache/solr/update/SolrIndexSplitter.java
@@ -472,7 +472,7 @@ public class SolrIndexSplitter {
               log.error("### INVALID DELS " + dels.cardinality());
             }
           }
-          return new ConstantScoreScorer(this, score(), new BitSetIterator(set, set.length()));
+          return new ConstantScoreScorer(this, score(), scoreMode, new BitSetIterator(set, set.length()));
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/b4449c73/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSortRandom.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSortRandom.java b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSortRandom.java
index 85eea60..318ba59 100644
--- a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSortRandom.java
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSortRandom.java
@@ -287,7 +287,7 @@ public class TestFieldCacheSortRandom extends LuceneTestCase {
             }
           }
 
-          return new ConstantScoreScorer(this, score(), new BitSetIterator(bits, bits.approximateCardinality()));
+          return new ConstantScoreScorer(this, score(), scoreMode, new BitSetIterator(bits, bits.approximateCardinality()));
         }
 
         @Override