You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2015/05/07 14:39:21 UTC

svn commit: r1678181 - in /lucene/dev/trunk: lucene/core/src/java/org/apache/lucene/search/ lucene/core/src/test/org/apache/lucene/search/ lucene/queries/src/java/org/apache/lucene/queries/ lucene/spatial/src/java/org/apache/lucene/spatial/composite/ s...

Author: jpountz
Date: Thu May  7 12:39:20 2015
New Revision: 1678181

URL: http://svn.apache.org/r1678181
Log:
LUCENE-6463: Share more logic between approximated queries.

Added:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/RandomAccessWeight.java   (with props)
Removed:
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/ConstantScoreScorer.java
Modified:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.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/ConstantScoreWeight.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Filter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.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/MultiTermQueryConstantScoreWrapper.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/TestConstantScoreQuery.java
    lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
    lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
    lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingWrapperQuery.java Thu May  7 12:39:20 2015
@@ -124,7 +124,7 @@ public class CachingWrapperQuery extends
       }
 
       @Override
-      protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final LeafReader reader = context.reader();
         final Object key = reader.getCoreCacheKey();
 
@@ -148,19 +148,17 @@ public class CachingWrapperQuery extends
         if (docIdSet == DocIdSet.EMPTY) {
           return null;
         }
-        final DocIdSetIterator approximation = docIdSet.iterator();
-        if (approximation == null) {
+        final DocIdSetIterator disi = docIdSet.iterator();
+        if (disi == null) {
           return null;
         }
 
-        final DocIdSetIterator disi;
-        final TwoPhaseIterator twoPhaseView;
+        // We apply acceptDocs as an approximation
         if (acceptDocs == null) {
-          twoPhaseView = null;
-          disi = approximation;
+          return new ConstantScoreScorer(this, 0f, disi);
         } else {
-          twoPhaseView = new TwoPhaseIterator(approximation) {
-            
+          final TwoPhaseIterator twoPhaseView = new TwoPhaseIterator(disi) {
+
             @Override
             public boolean matches() throws IOException {
               final int doc = approximation.docID();
@@ -168,46 +166,8 @@ public class CachingWrapperQuery extends
             }
 
           };
-          disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseView);
+          return new ConstantScoreScorer(this, 0f, twoPhaseView);
         }
-        return new Scorer(weight) {
-
-          @Override
-          public TwoPhaseIterator asTwoPhaseIterator() {
-            return twoPhaseView;
-          }
-
-          @Override
-          public float score() throws IOException {
-            return 0f;
-          }
-
-          @Override
-          public int freq() throws IOException {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return disi.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return disi.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return disi.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return disi.cost();
-          }
-          
-        };
       }
     };
   }

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=1678181&r1=1678180&r2=1678181&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 May  7 12:39:20 2015
@@ -21,11 +21,9 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Objects;
-import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -69,70 +67,6 @@ public class ConstantScoreQuery extends
     return this;
   }
 
-  protected class ConstantWeight extends Weight {
-    private final Weight innerWeight;
-    private float queryNorm;
-    private float queryWeight;
-    
-    public ConstantWeight(Weight innerWeight) throws IOException {
-      super(ConstantScoreQuery.this);
-      this.innerWeight = innerWeight;
-    }
-
-    @Override
-    public void extractTerms(Set<Term> terms) {
-      // no-op
-    }
-
-    @Override
-    public float getValueForNormalization() throws IOException {
-      // we calculate sumOfSquaredWeights of the inner weight, but ignore it (just to initialize everything)
-      innerWeight.getValueForNormalization();
-      queryWeight = getBoost();
-      return queryWeight * queryWeight;
-    }
-
-    @Override
-    public void normalize(float norm, float topLevelBoost) {
-      this.queryNorm = norm * topLevelBoost;
-      queryWeight *= this.queryNorm;
-      // we normalize the inner weight, but ignore it (just to initialize everything)
-      innerWeight.normalize(norm, topLevelBoost);
-    }
-
-    @Override
-    public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      BulkScorer bulkScorer = innerWeight.bulkScorer(context, acceptDocs);
-      if (bulkScorer == null) {
-        return null;
-      }
-      return new ConstantBulkScorer(bulkScorer, this, queryWeight);
-    }
-
-    @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      Scorer scorer = innerWeight.scorer(context, acceptDocs);
-      if (scorer == null) {
-        return null;
-      }
-      return new ConstantScoreScorer(scorer, queryWeight);
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      final Scorer cs = scorer(context, context.reader().getLiveDocs());
-      final boolean exists = (cs != null && cs.advance(doc) == doc);
-
-      if (exists) {
-        return Explanation.match(
-            queryWeight, ConstantScoreQuery.this.toString() + ", product of:",
-            Explanation.match(getBoost(), "boost"), Explanation.match(queryNorm, "queryNorm"));
-      } else {
-        return Explanation.noMatch(ConstantScoreQuery.this.toString() + " doesn't match id " + doc);
-      }
-    }
-  }
-
   /** We return this as our {@link BulkScorer} so that if the CSQ
    *  wraps a query with its own optimized top-level
    *  scorer (e.g. BooleanScorer) we can use that
@@ -158,7 +92,12 @@ public class ConstantScoreQuery extends
         @Override
         public void setScorer(Scorer scorer) throws IOException {
           // we must wrap again here, but using the scorer passed in as parameter:
-          in.setScorer(new ConstantScoreScorer(scorer, theScore));
+          in.setScorer(new FilterScorer(scorer) {
+            @Override
+            public float score() throws IOException {
+              return theScore;
+            }
+          });
         }
       };
     }
@@ -169,87 +108,41 @@ public class ConstantScoreQuery extends
     }
   }
 
-  protected class ConstantScoreScorer extends FilterScorer {
-
-    private final float score;
-
-    public ConstantScoreScorer(Scorer wrapped, float score) {
-      super(wrapped);
-      this.score = score;
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return 1;
-    }
-
-    @Override
-    public float score() throws IOException {
-      return score;
-    }
-
-    @Override
-    public Collection<ChildScorer> getChildren() {
-      return Collections.singletonList(new ChildScorer(in, "constant"));
-    }
-  }
-
-  protected class ConstantDocIdSetIteratorScorer extends Scorer {
-    final DocIdSetIterator docIdSetIterator;
-    final float theScore;
-
-    public ConstantDocIdSetIteratorScorer(DocIdSetIterator docIdSetIterator, Weight w, float theScore) {
-      super(w);
-      this.theScore = theScore;
-      this.docIdSetIterator = docIdSetIterator;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      return docIdSetIterator.nextDoc();
-    }
-    
-    @Override
-    public int docID() {
-      return docIdSetIterator.docID();
-    }
-
-    @Override
-    public float score() throws IOException {
-      assert docIdSetIterator.docID() != NO_MORE_DOCS;
-      return theScore;
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return 1;
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return docIdSetIterator.advance(target);
-    }
-
-    @Override
-    public long cost() {
-      return docIdSetIterator.cost();
-    }
-
-    @Override
-    public Collection<ChildScorer> getChildren() {
-      if (query != null) {
-        return Collections.singletonList(new ChildScorer((Scorer) docIdSetIterator, "constant"));
-      } else {
-        return Collections.emptyList();
-      }
-    }
-  }
-
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
     final Weight innerWeight = searcher.createWeight(query, false);
     if (needsScores) {
-      return new ConstantScoreQuery.ConstantWeight(innerWeight);
+      return new ConstantScoreWeight(this) {
+
+        @Override
+        public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+          final BulkScorer innerScorer = innerWeight.bulkScorer(context, acceptDocs);
+          if (innerScorer == null) {
+            return null;
+          }
+          return new ConstantBulkScorer(innerScorer, this, score());
+        }
+
+        @Override
+        public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
+          final Scorer innerScorer = innerWeight.scorer(context, acceptDocs);
+          if (innerScorer == null) {
+            return null;
+          }
+          final float score = score();
+          return new FilterScorer(innerScorer) {
+            @Override
+            public float score() throws IOException {
+              return score;
+            }
+            @Override
+            public Collection<ChildScorer> getChildren() {
+              return Collections.singleton(new ChildScorer(innerScorer, "constant"));
+            }
+          };
+        }
+
+      };
     } else {
       return innerWeight;
     }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java?rev=1678181&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreScorer.java Thu May  7 12:39:20 2015
@@ -0,0 +1,91 @@
+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;
+
+/**
+ * A constant-scoring {@link Scorer}.
+ * @lucene.internal
+ */
+public final class ConstantScoreScorer extends Scorer {
+
+  private final float score;
+  private final TwoPhaseIterator twoPhaseIterator;
+  private final DocIdSetIterator disi;
+
+  /** Constructor based on a {@link DocIdSetIterator} which will be used to
+   *  drive iteration. Two phase iteration will not be supported.
+   *  @param weight the parent weight
+   *  @param score the score to return on each document
+   *  @param disi the iterator that defines matching documents */
+  public ConstantScoreScorer(Weight weight, float score, DocIdSetIterator disi) {
+    super(weight);
+    this.score = score;
+    this.twoPhaseIterator = null;
+    this.disi = disi;
+  }
+
+  /** 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 twoPhaseIterator the iterator that defines matching documents */
+  public ConstantScoreScorer(Weight weight, float score, TwoPhaseIterator twoPhaseIterator) {
+    super(weight);
+    this.score = score;
+    this.twoPhaseIterator = twoPhaseIterator;
+    this.disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator);
+  }
+
+  @Override
+  public TwoPhaseIterator asTwoPhaseIterator() {
+    return twoPhaseIterator;
+  }
+
+  @Override
+  public float score() throws IOException {
+    return score;
+  }
+
+  @Override
+  public int freq() throws IOException {
+    return 1;
+  }
+
+  @Override
+  public int docID() {
+    return disi.docID();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    return disi.nextDoc();
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    return disi.advance(target);
+  }
+
+  @Override
+  public long cost() {
+    return disi.cost();
+  }
+}
+

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreWeight.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreWeight.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreWeight.java Thu May  7 12:39:20 2015
@@ -22,10 +22,11 @@ import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.util.Bits;
 
 /**
  * A Weight that has a constant score equal to the boost of the wrapped query.
+ * This is typically useful when building queries which do not produce
+ * meaningful scores and are mostly useful for filtering.
  *
  * @lucene.internal
  */
@@ -36,6 +37,8 @@ public abstract class ConstantScoreWeigh
 
   protected ConstantScoreWeight(Query query) {
     super(query);
+    queryWeight = getQuery().getBoost();
+    queryNorm = 1f;
   }
 
   @Override
@@ -47,7 +50,6 @@ public abstract class ConstantScoreWeigh
 
   @Override
   public final float getValueForNormalization() throws IOException {
-    queryWeight = getQuery().getBoost();
     return queryWeight * queryWeight;
   }
 
@@ -57,10 +59,25 @@ public abstract class ConstantScoreWeigh
     queryWeight *= queryNorm;
   }
 
+  /** Return the score produced by this {@link Weight}. */
+  protected final float score() {
+    return queryWeight;
+  }
+
   @Override
   public final Explanation explain(LeafReaderContext context, int doc) throws IOException {
     final Scorer s = scorer(context, context.reader().getLiveDocs());
-    final boolean exists = (s != null && s.advance(doc) == doc);
+    final boolean exists;
+    if (s == null) {
+      exists = false;
+    } else {
+      final TwoPhaseIterator twoPhase = s.asTwoPhaseIterator();
+      if (twoPhase == null) {
+        exists = s.advance(doc) == doc;
+      } else {
+        exists = twoPhase.approximation().advance(doc) == doc && twoPhase.matches();
+      }
+    }
 
     if (exists) {
       return Explanation.match(
@@ -71,11 +88,4 @@ public abstract class ConstantScoreWeigh
     }
   }
 
-  @Override
-  public final Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-    return scorer(context, acceptDocs, queryWeight);
-  }
-
-  protected abstract Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException;
-
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRangeQuery.java Thu May  7 12:39:20 2015
@@ -27,7 +27,6 @@ import org.apache.lucene.index.LeafReade
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.Bits.MatchNoBits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -123,18 +122,10 @@ public final class DocValuesRangeQuery e
     if (lowerVal == null && upperVal == null) {
       throw new IllegalStateException("Both min and max values cannot be null, call rewrite first");
     }
-    return new ConstantScoreWeight(DocValuesRangeQuery.this) {
-
+    return new RandomAccessWeight(DocValuesRangeQuery.this) {
+      
       @Override
-      public Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
-
-        final Bits docsWithField = context.reader().getDocsWithField(field);
-        if (docsWithField == null || docsWithField instanceof MatchNoBits) {
-          return null;
-        }
-
-        final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
-        final TwoPhaseIterator twoPhaseRange;
+      protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
         if (lowerVal instanceof Long || upperVal instanceof Long) {
 
           final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), field);
@@ -161,7 +152,27 @@ public final class DocValuesRangeQuery e
             return null;
           }
 
-          twoPhaseRange = new TwoPhaseNumericRange(values, min, max, approximation, acceptDocs);
+          return new Bits() {
+
+            @Override
+            public boolean get(int doc) {
+              values.setDocument(doc);
+              final int count = values.count();
+              for (int i = 0; i < count; ++i) {
+                final long value = values.valueAt(i);
+                if (value >= min && value <= max) {
+                  return true;
+                }
+              }
+              return false;
+            }
+
+            @Override
+            public int length() {
+              return context.reader().maxDoc();
+            }
+
+          };
 
         } else if (lowerVal instanceof BytesRef || upperVal instanceof BytesRef) {
 
@@ -199,128 +210,31 @@ public final class DocValuesRangeQuery e
             return null;
           }
 
-          twoPhaseRange = new TwoPhaseOrdRange(values, minOrd, maxOrd, approximation, acceptDocs);
-
-        } else {
-          throw new AssertionError();
-        }
-
-        return new RangeScorer(this, twoPhaseRange, score);
-      }
+          return new Bits() {
 
-    };
-  }
-
-  private static class TwoPhaseNumericRange extends TwoPhaseIterator {
-
-    private final SortedNumericDocValues values;
-    private final long min, max;
-    private final Bits acceptDocs;
-
-    TwoPhaseNumericRange(SortedNumericDocValues values, long min, long max, DocIdSetIterator approximation, Bits acceptDocs) {
-      super(approximation);
-      this.values = values;
-      this.min = min;
-      this.max = max;
-      this.acceptDocs = acceptDocs;
-    }
-
-    @Override
-    public boolean matches() throws IOException {
-      final int doc = approximation.docID();
-      if (acceptDocs == null || acceptDocs.get(doc)) {
-        values.setDocument(doc);
-        final int count = values.count();
-        for (int i = 0; i < count; ++i) {
-          final long value = values.valueAt(i);
-          if (value >= min && value <= max) {
-            return true;
-          }
-        }
-      }
-      return false;
-    }
-
-  }
+            @Override
+            public boolean get(int doc) {
+              values.setDocument(doc);
+              for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
+                if (ord >= minOrd && ord <= maxOrd) {
+                  return true;
+                }
+              }
+              return false;
+            }
 
-  private static class TwoPhaseOrdRange extends TwoPhaseIterator {
+            @Override
+            public int length() {
+              return context.reader().maxDoc();
+            }
 
-    private final SortedSetDocValues values;
-    private final long minOrd, maxOrd;
-    private final Bits acceptDocs;
-
-    TwoPhaseOrdRange(SortedSetDocValues values, long minOrd, long maxOrd, DocIdSetIterator approximation, Bits acceptDocs) {
-      super(approximation);
-      this.values = values;
-      this.minOrd = minOrd;
-      this.maxOrd = maxOrd;
-      this.acceptDocs = acceptDocs;
-    }
+          };
 
-    @Override
-    public boolean matches() throws IOException {
-      final int doc = approximation.docID();
-      if (acceptDocs == null || acceptDocs.get(doc)) {
-        values.setDocument(doc);
-        for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
-          if (ord >= minOrd && ord <= maxOrd) {
-            return true;
-          }
+        } else {
+          throw new AssertionError();
         }
       }
-      return false;
-    }
-
-  }
-
-  private static class RangeScorer extends Scorer {
-
-    private final TwoPhaseIterator twoPhaseRange;
-    private final DocIdSetIterator disi;
-    private final float score;
-
-    RangeScorer(Weight weight, TwoPhaseIterator twoPhaseRange, float score) {
-      super(weight);
-      this.twoPhaseRange = twoPhaseRange;
-      this.disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseRange);
-      this.score = score;
-    }
-
-    @Override
-    public TwoPhaseIterator asTwoPhaseIterator() {
-      return twoPhaseRange;
-    }
-
-    @Override
-    public float score() throws IOException {
-      return score;
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return 1;
-    }
-
-    @Override
-    public int docID() {
-      return disi.docID();
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      return disi.nextDoc();
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return disi.advance(target);
-    }
-
-    @Override
-    public long cost() {
-      return disi.cost();
-    }
-
+    };
   }
 
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesRewriteMethod.java Thu May  7 12:39:20 2015
@@ -82,9 +82,9 @@ public final class DocValuesRewriteMetho
     
     @Override
     public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-      return new ConstantScoreWeight(this) {
+      return new RandomAccessWeight(this) {
         @Override
-        protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+        protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
           final SortedSetDocValues fcsi = DocValues.getSortedSet(context.reader(), query.field);
           TermsEnum termsEnum = query.getTermsEnum(new Terms() {
             
@@ -148,15 +148,11 @@ public final class DocValuesRewriteMetho
               termSet.set(ord);
             }
           } while (termsEnum.next() != null);
-          
-          final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
-          final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(approximation) {
+
+          return new Bits() {
+
             @Override
-            public boolean matches() throws IOException {
-              final int doc = approximation.docID();
-              if (acceptDocs != null && acceptDocs.get(doc) == false) {
-                return false;
-              }
+            public boolean get(int doc) {
               fcsi.setDocument(doc);
               for (long ord = fcsi.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = fcsi.nextOrd()) {
                 if (termSet.get(ord)) {
@@ -165,43 +161,10 @@ public final class DocValuesRewriteMetho
               }
               return false;
             }
-          };
-          final DocIdSetIterator disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator);
-          return new Scorer(this) {
-
-            @Override
-            public TwoPhaseIterator asTwoPhaseIterator() {
-              return twoPhaseIterator;
-            }
-
-            @Override
-            public float score() throws IOException {
-              return score;
-            }
-
-            @Override
-            public int freq() throws IOException {
-              return 1;
-            }
-
-            @Override
-            public int docID() {
-              return disi.docID();
-            }
-
-            @Override
-            public int nextDoc() throws IOException {
-              return disi.nextDoc();
-            }
-
-            @Override
-            public int advance(int target) throws IOException {
-              return disi.advance(target);
-            }
 
             @Override
-            public long cost() {
-              return disi.cost();
+            public int length() {
+              return context.reader().maxDoc();
             }
 
           };

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsQuery.java Thu May  7 12:39:20 2015
@@ -147,10 +147,10 @@ public class DocValuesTermsQuery extends
 
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    return new ConstantScoreWeight(this) {
+    return new RandomAccessWeight(this) {
 
       @Override
-      protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
         final SortedSetDocValues values = DocValues.getSortedSet(context.reader(), field);
         final LongBitSet bits = new LongBitSet(values.getValueCount());
         for (BytesRef term : terms) {
@@ -159,15 +159,10 @@ public class DocValuesTermsQuery extends
             bits.set(ord);
           }
         }
+        return new Bits() {
 
-        final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
-        final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(approximation) {
           @Override
-          public boolean matches() throws IOException {
-            final int doc = approximation.docID();
-            if (acceptDocs != null && acceptDocs.get(doc) == false) {
-              return false;
-            }
+          public boolean get(int doc) {
             values.setDocument(doc);
             for (long ord = values.nextOrd(); ord != SortedSetDocValues.NO_MORE_ORDS; ord = values.nextOrd()) {
               if (bits.get(ord)) {
@@ -176,48 +171,14 @@ public class DocValuesTermsQuery extends
             }
             return false;
           }
-        };
-        final DocIdSetIterator disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator);
-        return new Scorer(this) {
-
-          @Override
-          public TwoPhaseIterator asTwoPhaseIterator() {
-            return twoPhaseIterator;
-          }
-
-          @Override
-          public float score() throws IOException {
-            return score;
-          }
-
-          @Override
-          public int freq() throws IOException {
-            return 1;
-          }
 
           @Override
-          public int docID() {
-            return disi.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return disi.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return disi.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return disi.cost();
+          public int length() {
+            return context.reader().maxDoc();
           }
 
         };
       }
-
     };
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FieldValueQuery.java Thu May  7 12:39:20 2015
@@ -23,7 +23,6 @@ import java.util.Objects;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.Bits.MatchNoBits;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -61,71 +60,13 @@ public final class FieldValueQuery exten
 
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
-    return new ConstantScoreWeight(this) {
+    return new RandomAccessWeight(this) {
 
       @Override
-      public Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
-        final Bits docsWithField = context.reader().getDocsWithField(field);
-        if (docsWithField == null || docsWithField instanceof MatchNoBits) {
-          return null;
-        }
-
-        final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
-        final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(approximation) {
-
-          @Override
-          public boolean matches() throws IOException {
-            final int doc = approximation.docID();
-            if (acceptDocs != null && acceptDocs.get(doc) == false) {
-              return false;
-            }
-            if (docsWithField.get(doc) == false) {
-              return false;
-            }
-            return true;
-          }
-
-        };
-        final DocIdSetIterator disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseIterator);
-
-        return new Scorer(this) {
-
-          @Override
-          public TwoPhaseIterator asTwoPhaseIterator() {
-            return twoPhaseIterator;
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return disi.nextDoc();
-          }
-
-          @Override
-          public int docID() {
-            return disi.docID();
-          }
-
-          @Override
-          public long cost() {
-            return disi.cost();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return disi.advance(target);
-          }
-
-          @Override
-          public int freq() throws IOException {
-            return 1;
-          }
-
-          @Override
-          public float score() throws IOException {
-            return score;
-          }
-        };
+      protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
+        return context.reader().getDocsWithField(field);
       }
+
     };
   }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Filter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Filter.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Filter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Filter.java Thu May  7 12:39:20 2015
@@ -100,38 +100,7 @@ public abstract class Filter extends Que
         if (iterator == null) {
           return null;
         }
-        return new Scorer(this) {
-
-          @Override
-          public float score() throws IOException {
-            return 0f;
-          }
-
-          @Override
-          public int freq() throws IOException {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return iterator.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return iterator.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return iterator.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return iterator.cost();
-          }
-        };
+        return new ConstantScoreScorer(this, 0f, iterator);
       }
 
     };

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java Thu May  7 12:39:20 2015
@@ -30,10 +30,10 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.ReaderUtil;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.LeafReader.CoreClosedListener;
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.ReaderUtil;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.Bits;
@@ -566,7 +566,7 @@ public class LRUQueryCache implements Qu
     }
 
     @Override
-    protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
       if (context.ord == 0) {
         policy.onUse(getQuery());
       }
@@ -590,64 +590,24 @@ public class LRUQueryCache implements Qu
       if (docIdSet == DocIdSet.EMPTY) {
         return null;
       }
-      final DocIdSetIterator approximation = docIdSet.iterator();
-      if (approximation == null) {
+      final DocIdSetIterator disi = docIdSet.iterator();
+      if (disi == null) {
         return null;
       }
 
-      final DocIdSetIterator disi;
-      final TwoPhaseIterator twoPhaseView;
+      // we apply acceptDocs as an approximation
       if (acceptDocs == null) {
-        twoPhaseView = null;
-        disi = approximation;
+        return new ConstantScoreScorer(this, 0f, disi);
       } else {
-        twoPhaseView = new TwoPhaseIterator(approximation) {
+        final TwoPhaseIterator twoPhaseView = new TwoPhaseIterator(disi) {
           @Override
           public boolean matches() throws IOException {
             final int doc = approximation.docID();
             return acceptDocs.get(doc);
           }
         };
-        disi = TwoPhaseIterator.asDocIdSetIterator(twoPhaseView);
+        return new ConstantScoreScorer(this, 0f, twoPhaseView);
       }
-      return new Scorer(this) {
-
-        @Override
-        public TwoPhaseIterator asTwoPhaseIterator() {
-          return twoPhaseView;
-        }
-
-        @Override
-        public float score() throws IOException {
-          return 0f;
-        }
-
-        @Override
-        public int freq() throws IOException {
-          return 1;
-        }
-
-        @Override
-        public int docID() {
-          return disi.docID();
-        }
-
-        @Override
-        public int nextDoc() throws IOException {
-          return disi.nextDoc();
-        }
-
-        @Override
-        public int advance(int target) throws IOException {
-          return disi.advance(target);
-        }
-
-        @Override
-        public long cost() {
-          return disi.cost();
-        }
-
-      };
     }
 
   }

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=1678181&r1=1678180&r2=1678181&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 May  7 12:39:20 2015
@@ -18,13 +18,8 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
@@ -34,105 +29,18 @@ import org.apache.lucene.util.ToStringUt
  */
 public final class MatchAllDocsQuery extends Query {
 
-  private class MatchAllScorer extends Scorer {
-    final float score;
-    private int doc = -1;
-    private final int maxDoc;
-    private final Bits liveDocs;
-
-    MatchAllScorer(IndexReader reader, Bits liveDocs, Weight w, float score) {
-      super(w);
-      this.liveDocs = liveDocs;
-      this.score = score;
-      maxDoc = reader.maxDoc();
-    }
-
-    @Override
-    public int docID() {
-      return doc;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      doc++;
-      while(liveDocs != null && doc < maxDoc && !liveDocs.get(doc)) {
-        doc++;
-      }
-      if (doc >= maxDoc) { // can be > maxDoc when called from advance()
-        doc = NO_MORE_DOCS;
-      }
-      return doc;
-    }
-    
-    @Override
-    public float score() {
-      return score;
-    }
-
-    @Override
-    public int freq() {
-      return 1;
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      doc = target-1;
-      return nextDoc();
-    }
-
-    @Override
-    public long cost() {
-      return maxDoc;
-    }
-  }
-
-  private class MatchAllDocsWeight extends Weight {
-    private float queryWeight;
-    private float queryNorm;
-
-    public MatchAllDocsWeight() {
-      super(MatchAllDocsQuery.this);
-    }
-
-    @Override
-    public void extractTerms(Set<Term> terms) {}
-
-    @Override
-    public String toString() {
-      return "weight(" + MatchAllDocsQuery.this + ")";
-    }
-
-    @Override
-    public float getValueForNormalization() {
-      queryWeight = getBoost();
-      return queryWeight * queryWeight;
-    }
-
-    @Override
-    public void normalize(float queryNorm, float topLevelBoost) {
-      this.queryNorm = queryNorm * topLevelBoost;
-      queryWeight *= this.queryNorm;
-    }
-
-    @Override
-    public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      return new MatchAllScorer(context.reader(), acceptDocs, this, queryWeight);
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) {
-      List<Explanation> subs = new ArrayList<>();
-      if (getBoost() != 1.0f) {
-        subs.add(Explanation.match(getBoost(),"boost"));
-      }
-      subs.add(Explanation.match(queryNorm, "queryNorm"));
-      return Explanation.match(queryWeight, "MatchAllDocsQuery, product of:", subs);
-    }
-  }
-
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) {
-    return new MatchAllDocsWeight();
+    return new RandomAccessWeight(this) {
+      @Override
+      protected Bits getMatchingDocs(LeafReaderContext context) throws IOException {
+        return new Bits.MatchAllBits(context.reader().maxDoc());
+      }
+      @Override
+      public String toString() {
+        return "weight(" + MatchAllDocsQuery.this + ")";
+      }
+    };
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryConstantScoreWrapper.java Thu May  7 12:39:20 2015
@@ -19,11 +19,9 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 import java.util.Objects;
-import java.util.Set;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.PostingsEnum;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BitDocIdSet;
@@ -82,7 +80,7 @@ final class MultiTermQueryConstantScoreW
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
     return new ConstantScoreWeight(this) {
       @Override
-      protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final Terms terms = context.reader().terms(query.field);
         if (terms == null) {
           // field does not exist
@@ -106,39 +104,7 @@ final class MultiTermQueryConstantScoreW
         if (disi == null) {
           return null;
         }
-        return new Scorer(this) {
-
-          @Override
-          public float score() throws IOException {
-            return score;
-          }
-
-          @Override
-          public int freq() throws IOException {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return disi.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return disi.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return disi.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return disi.cost();
-          }
-
-        };
+        return new ConstantScoreScorer(this, score(), disi);
       }
     };
   }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/RandomAccessWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/RandomAccessWeight.java?rev=1678181&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/RandomAccessWeight.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/RandomAccessWeight.java Thu May  7 12:39:20 2015
@@ -0,0 +1,75 @@
+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 org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.Bits.MatchNoBits;
+
+/**
+ * Base class to build {@link Weight}s that are based on random-access
+ * structures such as live docs or doc values. Such weights return a
+ * {@link Scorer} which consists of an approximation that matches
+ * everything, and a confirmation phase that first checks live docs and
+ * then the {@link Bits} returned by {@link #getMatchingDocs(LeafReaderContext)}.
+ * @lucene.internal
+ */
+public abstract class RandomAccessWeight extends ConstantScoreWeight {
+
+  /** Sole constructor. */
+  protected RandomAccessWeight(Query query) {
+    super(query);
+  }
+
+  /**
+   * Return a {@link Bits} instance representing documents that match this
+   * weight on the given context. A return value of {@code null} indicates
+   * that no documents matched.
+   * Note: it is not needed to care about live docs as they will be checked
+   * before the returned bits.
+   */
+  protected abstract Bits getMatchingDocs(LeafReaderContext context) throws IOException;
+
+  @Override
+  public final Scorer scorer(LeafReaderContext context, final Bits acceptDocs) throws IOException {
+    final Bits matchingDocs = getMatchingDocs(context);
+    if (matchingDocs == null || matchingDocs instanceof MatchNoBits) {
+      return null;
+    }
+    final DocIdSetIterator approximation = DocIdSetIterator.all(context.reader().maxDoc());
+    final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
+
+      @Override
+      public boolean matches() throws IOException {
+        final int doc = approximation.docID();
+
+        if (acceptDocs != null && acceptDocs.get(doc) == false) {
+          return false;
+        }
+
+        return matchingDocs.get(doc);
+      }
+    };
+
+    return new ConstantScoreScorer(this, score(), twoPhase);
+  }
+
+}
+

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=1678181&r1=1678180&r2=1678181&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 May  7 12:39:20 2015
@@ -198,7 +198,7 @@ public class TestBooleanQueryVisitSubsco
     for (String summary : collector.getSummaries()) {
       assertEquals(
           "CoordinatingConjunctionScorer\n" +
-          "    MUST MatchAllScorer\n" +
+          "    MUST ConstantScoreScorer\n" +
           "    MUST MinShouldMatchSumScorer\n" +
           "            SHOULD TermScorer body:nutch\n" +
           "            SHOULD TermScorer body:web\n" +

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=1678181&r1=1678180&r2=1678181&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 May  7 12:39:20 2015
@@ -33,6 +33,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 
 /** This class only tests some basic functionality in CSQ, the main parts are mostly
@@ -54,7 +55,7 @@ public class TestConstantScoreQuery exte
     QueryUtils.checkUnequal(q1, new TermQuery(new Term("a", "b")));
   }
   
-  private void checkHits(IndexSearcher searcher, Query q, final float expectedScore, final String scorerClassName, final String innerScorerClassName) throws IOException {
+  private void checkHits(IndexSearcher searcher, Query q, final float expectedScore, final Class<? extends Scorer> innerScorerClass) throws IOException {
     final int[] count = new int[1];
     searcher.search(q, new SimpleCollector() {
       private Scorer scorer;
@@ -62,10 +63,9 @@ public class TestConstantScoreQuery exte
       @Override
       public void setScorer(Scorer scorer) {
         this.scorer = scorer;
-        assertEquals("Scorer is implemented by wrong class", scorerClassName, scorer.getClass().getName());
-        if (innerScorerClassName != null && scorer instanceof ConstantScoreQuery.ConstantScoreScorer) {
-          final ConstantScoreQuery.ConstantScoreScorer innerScorer = (ConstantScoreQuery.ConstantScoreScorer) scorer;
-          assertEquals("inner Scorer is implemented by wrong class", innerScorerClassName, innerScorer.in.getClass().getName());
+        if (innerScorerClass != null) {
+          final FilterScorer innerScorer = (FilterScorer) scorer;
+          assertEquals("inner Scorer is implemented by wrong class", innerScorerClass, innerScorer.in.getClass());
         }
       }
       
@@ -121,16 +121,14 @@ public class TestConstantScoreQuery exte
       final Query csqbq = new ConstantScoreQuery(bq);
       csqbq.setBoost(17.0f);
       
-      checkHits(searcher, csq1, csq1.getBoost(), ConstantScoreQuery.ConstantScoreScorer.class.getName(), TermScorer.class.getName());
-      checkHits(searcher, csq2, csq2.getBoost(), ConstantScoreQuery.ConstantScoreScorer.class.getName(), TermScorer.class.getName());
+      checkHits(searcher, csq1, csq1.getBoost(), TermScorer.class);
+      checkHits(searcher, csq2, csq2.getBoost(), TermScorer.class);
       
       // for the combined BQ, the scorer should always be BooleanScorer's BucketScorer, because our scorer supports out-of order collection!
-      final String bucketScorerClass = FakeScorer.class.getName();
-      checkHits(searcher, bq, csq1.getBoost() + csq2.getBoost(), bucketScorerClass, null);
-      checkHits(searcher, csqbq, csqbq.getBoost(), ConstantScoreQuery.ConstantScoreScorer.class.getName(), bucketScorerClass);
+      final Class<FakeScorer> bucketScorerClass = FakeScorer.class;
+      checkHits(searcher, csqbq, csqbq.getBoost(), bucketScorerClass);
     } finally {
-      if (reader != null) reader.close();
-      if (directory != null) directory.close();
+      IOUtils.close(reader, directory);
     }
   }
 

Modified: lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java (original)
+++ lucene/dev/trunk/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java Thu May  7 12:39:20 2015
@@ -348,7 +348,7 @@ public class TestLRUQueryCache extends L
     public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
       return new ConstantScoreWeight(this) {
         @Override
-        protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+        public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
           return null;
         }
       };
@@ -933,9 +933,8 @@ public class TestLRUQueryCache extends L
     @Override
     public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
       return new ConstantScoreWeight(this) {
-        
         @Override
-        protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+        public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
           return null;
         }
       };

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsQuery.java Thu May  7 12:39:20 2015
@@ -35,6 +35,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
@@ -189,7 +190,7 @@ public class TermsQuery extends Query im
       }
 
       @Override
-      public Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
         final LeafReader reader = context.reader();
         BitDocIdSet.Builder builder = new BitDocIdSet.Builder(reader.maxDoc());
         final Fields fields = reader.fields();
@@ -224,39 +225,7 @@ public class TermsQuery extends Query im
           return null;
         }
 
-        return new Scorer(this) {
-
-          @Override
-          public float score() throws IOException {
-            return score;
-          }
-
-          @Override
-          public int freq() throws IOException {
-            return 1;
-          }
-
-          @Override
-          public int docID() {
-            return disi.docID();
-          }
-
-          @Override
-          public int nextDoc() throws IOException {
-            return disi.nextDoc();
-          }
-
-          @Override
-          public int advance(int target) throws IOException {
-            return disi.advance(target);
-          }
-
-          @Override
-          public long cost() {
-            return disi.cost();
-          }
-
-        };
+        return new ConstantScoreScorer(this, score(), disi);
       }
     };
   }

Modified: lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java Thu May  7 12:39:20 2015
@@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -93,7 +94,7 @@ public class CompositeVerifyQuery extend
     return new ConstantScoreWeight(this) {
 
       @Override
-      protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
 
         final Scorer indexQueryScorer = indexQueryWeight.scorer(context, acceptDocs);//pass acceptDocs through
         if (indexQueryScorer == null) {
@@ -109,7 +110,7 @@ public class CompositeVerifyQuery extend
           }
         };
 
-        return new ConstantScoreScorer(this, score, twoPhaseIterator);
+        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
       }
     };
   }

Modified: lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java Thu May  7 12:39:20 2015
@@ -25,6 +25,7 @@ import com.spatial4j.core.shape.SpatialR
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.ConstantScoreScorer;
 import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -89,7 +90,7 @@ public class IntersectsRPTVerifyQuery ex
 
     return new ConstantScoreWeight(this) {
       @Override
-      protected Scorer scorer(LeafReaderContext context, Bits acceptDocs, float score) throws IOException {
+      public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
         // Compute approx & exact
         final IntersectsDifferentiatingFilter.IntersectsDifferentiatingVisitor result =
             intersectsDiffFilter.compute(context, acceptDocs);
@@ -104,7 +105,7 @@ public class IntersectsRPTVerifyQuery ex
         if (result.exactDocIdSet != null) {
           // If both sets are the same, there's nothing to verify; we needn't return a TwoPhaseIterator
           if (result.approxDocIdSet.equals(result.exactDocIdSet)) {
-            return new ConstantScoreScorer(this, score, approxDISI);
+            return new ConstantScoreScorer(this, score(), approxDISI);
           }
           exactDocBits = result.exactDocIdSet.bits();
           assert exactDocBits != null;
@@ -125,7 +126,7 @@ public class IntersectsRPTVerifyQuery ex
           }
         };
 
-        return new ConstantScoreScorer(this, score, twoPhaseIterator);
+        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
       }
     };
   }

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/JoinQParserPlugin.java Thu May  7 12:39:20 2015
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexReader;
@@ -31,9 +30,10 @@ import org.apache.lucene.index.MultiPost
 import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -207,13 +207,11 @@ class JoinQuery extends Query {
     return new JoinQueryWeight((SolrIndexSearcher)searcher);
   }
 
-  private class JoinQueryWeight extends Weight {
+  private class JoinQueryWeight extends ConstantScoreWeight {
     SolrIndexSearcher fromSearcher;
     RefCounted<SolrIndexSearcher> fromRef;
     SolrIndexSearcher toSearcher;
     private Similarity similarity;
-    private float queryNorm;
-    private float queryWeight;
     ResponseBuilder rb;
 
     public JoinQueryWeight(SolrIndexSearcher searcher) {
@@ -274,26 +272,10 @@ class JoinQuery extends Query {
       this.toSearcher = searcher;
     }
 
-    @Override
-    public void extractTerms(Set<org.apache.lucene.index.Term> terms) {}
-
-    @Override
-    public float getValueForNormalization() throws IOException {
-      queryWeight = getBoost();
-      return queryWeight * queryWeight;
-    }
-
-    @Override
-    public void normalize(float norm, float topLevelBoost) {
-      this.queryNorm = norm * topLevelBoost;
-      queryWeight *= this.queryNorm;
-    }
-
     DocSet resultSet;
     Filter filter;
 
 
-
     @Override
     public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
       if (filter == null) {
@@ -328,7 +310,14 @@ class JoinQuery extends Query {
 
       // Although this set only includes live docs, other filters can be pushed down to queries.
       DocIdSet readerSet = filter.getDocIdSet(context, acceptDocs);
-      return new JoinScorer(this, readerSet == null ? DocIdSetIterator.empty() : readerSet.iterator(), getBoost());
+      if (readerSet == null) {
+        return null;
+      }
+      DocIdSetIterator readerSetIterator = readerSet.iterator();
+      if (readerSetIterator == null) {
+        return null;
+      }
+      return new ConstantScoreScorer(this, score(), readerSetIterator);
     }
 
 
@@ -562,65 +551,8 @@ class JoinQuery extends Query {
       return new SortedIntDocSet(dedup, dedup.length);
     }
 
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-      Scorer scorer = scorer(context, context.reader().getLiveDocs());
-      boolean exists = scorer.advance(doc) == doc;
-
-      if (exists) {
-        return Explanation.match(queryWeight, this.toString() + " , product of:",
-            Explanation.match(getBoost(), "boost"),
-            Explanation.match(queryNorm,"queryNorm"));
-      } else {
-        return Explanation.noMatch(this.toString() + " doesn't match id " + doc);
-      }
-    }
   }
 
-
-  protected static class JoinScorer extends Scorer {
-    final DocIdSetIterator iter;
-    final float score;
-    int doc = -1;
-
-    public JoinScorer(Weight w, DocIdSetIterator iter, float score) throws IOException {
-      super(w);
-      this.score = score;
-      this.iter = iter==null ? DocIdSetIterator.empty() : iter;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      return iter.nextDoc();
-    }
-
-    @Override
-    public int docID() {
-      return iter.docID();
-    }
-
-    @Override
-    public float score() throws IOException {
-      return score;
-    }
-    
-    @Override
-    public int freq() throws IOException {
-      return 1;
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return iter.advance(target);
-    }
-
-    @Override
-    public long cost() {
-      return iter.cost();
-    }
-  }
-
-
   @Override
   public String toString(String field) {
     return "{!join from="+fromField+" to="+toField

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=1678181&r1=1678180&r2=1678181&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Thu May  7 12:39:20 2015
@@ -2,22 +2,21 @@ package org.apache.solr.search;
 
 import java.io.IOException;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
-import org.apache.solr.common.SolrException;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -93,9 +92,7 @@ public class SolrConstantScoreQuery exte
     return this;
   }
 
-  protected class ConstantWeight extends Weight {
-    private float queryNorm;
-    private float queryWeight;
+  protected class ConstantWeight extends ConstantScoreWeight {
     private Map context;
 
     public ConstantWeight(IndexSearcher searcher) throws IOException {
@@ -106,106 +103,23 @@ public class SolrConstantScoreQuery exte
     }
 
     @Override
-    public void extractTerms(Set<org.apache.lucene.index.Term> terms) {
-      // OK to not add any terms when used for MultiSearcher,
-      // but may not be OK for highlighting
-    }
-
-    @Override
-    public float getValueForNormalization() throws IOException {
-      queryWeight = getBoost();
-      return queryWeight * queryWeight;
-    }
-
-    @Override
-    public void normalize(float norm, float topLevelBoost) {
-      this.queryNorm = norm * topLevelBoost;
-      queryWeight *= this.queryNorm;
-    }
-
-    @Override
     public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      return new ConstantScorer(context, this, queryWeight, acceptDocs);
-    }
-
-    @Override
-    public Explanation explain(LeafReaderContext context, int doc) throws IOException {
-
-      ConstantScorer cs = new ConstantScorer(context, this, queryWeight, context.reader().getLiveDocs());
-      boolean exists = cs.docIdSetIterator.advance(doc) == doc;
-
-      if (exists) {
-        return Explanation.match(queryWeight, "ConstantScoreQuery(" + filter + "), product of:",
-            Explanation.match(getBoost(), "boost"),
-            Explanation.match(queryNorm,"queryNorm"));
-      } else {
-        return Explanation.noMatch("ConstantScoreQuery(" + filter + ") doesn't match id " + doc);
-      }
-    }
-  }
-
-  protected class ConstantScorer extends Scorer {
-    final DocIdSetIterator docIdSetIterator;
-    final float theScore;
-    final Bits acceptDocs;
-    int doc = -1;
-
-    public ConstantScorer(LeafReaderContext context, ConstantWeight w, float theScore, Bits acceptDocs) throws IOException {
-      super(w);
-      this.theScore = theScore;
-      this.acceptDocs = acceptDocs;
-      DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, context, acceptDocs) : filter.getDocIdSet(context, acceptDocs);
+      DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(this.context, context, acceptDocs) : filter.getDocIdSet(context, acceptDocs);
       if (docIdSet == null) {
-        docIdSetIterator = DocIdSetIterator.empty();
-      } else {
-        DocIdSetIterator iter = docIdSet.iterator();
-        if (iter == null) {
-          docIdSetIterator = DocIdSetIterator.empty();
-        } else {
-          docIdSetIterator = iter;
-        }
+        return null;
       }
+      DocIdSetIterator iterator = docIdSet.iterator();
+      if (iterator == null) {
+        return null;
+      }
+      return new ConstantScoreScorer(this, score(), iterator);
     }
 
-    @Override
-    public int nextDoc() throws IOException {
-      return docIdSetIterator.nextDoc();
-    }
-
-    @Override
-    public int docID() {
-      return docIdSetIterator.docID();
-    }
-
-    @Override
-    public float score() throws IOException {
-      return theScore;
-    }
-    
-    @Override
-    public int freq() throws IOException {
-      return 1;
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return docIdSetIterator.advance(target);
-    }
-
-    @Override
-    public long cost() {
-      return docIdSetIterator.cost();
-    }
   }
 
   @Override
-  public Weight createWeight(IndexSearcher searcher, boolean needsScores) {
-    try {
-      return new SolrConstantScoreQuery.ConstantWeight(searcher);
-    } catch (IOException e) {
-      // TODO: remove this if ConstantScoreQuery.createWeight adds IOException
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-    }
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new SolrConstantScoreQuery.ConstantWeight(searcher);
   }
 
   /** Prints a user-readable version of this query. */
@@ -219,16 +133,15 @@ public class SolrConstantScoreQuery exte
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
-    if (!(o instanceof SolrConstantScoreQuery)) return false;
+    if (super.equals(o) == false) return false;
     SolrConstantScoreQuery other = (SolrConstantScoreQuery)o;
-    return this.getBoost()==other.getBoost() && filter.equals(other.filter);
+    return filter.equals(other.filter);
   }
 
   /** Returns a hash code value for this object. */
   @Override
   public int hashCode() {
-    // Simple add is OK since no existing filter hashcode has a float component.
-    return filter.hashCode() + Float.floatToIntBits(getBoost());
+    return 31 * super.hashCode() + filter.hashCode();
   }
 
 }