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/07/01 16:48:21 UTC

svn commit: r1688661 - in /lucene/dev/trunk/lucene: ./ demo/src/java/org/apache/lucene/demo/facet/ facet/src/java/org/apache/lucene/facet/ facet/src/java/org/apache/lucene/facet/range/ facet/src/test/org/apache/lucene/facet/range/

Author: jpountz
Date: Wed Jul  1 14:48:21 2015
New Revision: 1688661

URL: http://svn.apache.org/r1688661
Log:
LUCENE-6648: Remove usage of oal.search.Filter in lucene/facet.

Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Jul  1 14:48:21 2015
@@ -161,6 +161,9 @@ API Changes
   than a string or numeric field would require to implement a custom selector.
   (Adrien Grand)
 
+* LUCENE-6648: All lucene/facet APIs now take Query objects where they used to
+  take Filter objects. (Adrien Grand)
+
 Bug fixes
 
 * LUCENE-6500: ParallelCompositeReader did not always call

Modified: lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java (original)
+++ lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/DistanceFacetsExample.java Wed Jul  1 14:48:21 2015
@@ -45,11 +45,10 @@ import org.apache.lucene.index.IndexWrit
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.NumericRangeQuery;
-import org.apache.lucene.search.QueryWrapperFilter;
+import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.store.Directory;
@@ -140,7 +139,7 @@ public class DistanceFacetsExample imple
    *  maximum great circle (surface of the earth) distance,
    *  returns a simple Filter bounding box to "fast match"
    *  candidates. */
-  public static Filter getBoundingBoxFilter(double originLat, double originLng, double maxDistanceKM) {
+  public static Query getBoundingBoxQuery(double originLat, double originLng, double maxDistanceKM) {
 
     // Basic bounding box geo math from
     // http://JanMatuschek.de/LatitudeLongitudeBoundingCoordinates,
@@ -201,7 +200,7 @@ public class DistanceFacetsExample imple
             BooleanClause.Occur.FILTER);
     }
 
-    return new QueryWrapperFilter(f.build());
+    return f.build();
   }
 
   /** User runs a query and counts facets. */
@@ -212,7 +211,7 @@ public class DistanceFacetsExample imple
     searcher.search(new MatchAllDocsQuery(), fc);
 
     Facets facets = new DoubleRangeFacetCounts("field", getDistanceValueSource(), fc,
-                                               getBoundingBoxFilter(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, 10.0),
+                                               getBoundingBoxQuery(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, 10.0),
                                                ONE_KM,
                                                TWO_KM,
                                                FIVE_KM,
@@ -228,7 +227,7 @@ public class DistanceFacetsExample imple
     // documents ("browse only"):
     DrillDownQuery q = new DrillDownQuery(null);
     final ValueSource vs = getDistanceValueSource();
-    q.add("field", range.getFilter(getBoundingBoxFilter(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, range.max), vs));
+    q.add("field", range.getQuery(getBoundingBoxQuery(ORIGIN_LATITUDE, ORIGIN_LONGITUDE, range.max), vs));
     DrillSideways ds = new DrillSideways(searcher, config, (TaxonomyReader) null) {
         @Override
         protected Facets buildFacetsResult(FacetsCollector drillDowns, FacetsCollector[] drillSideways, String[] drillSidewaysDims) throws IOException {        

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java Wed Jul  1 14:48:21 2015
@@ -29,7 +29,6 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause.Occur;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
@@ -67,7 +66,7 @@ public final class DrillDownQuery extend
   }
 
   /** Used by DrillSideways */
-  DrillDownQuery(FacetsConfig config, Filter filter, DrillDownQuery other) {
+  DrillDownQuery(FacetsConfig config, Query filter, DrillDownQuery other) {
     this.baseQuery = new BooleanQuery.Builder()
         .add(other.baseQuery == null ? new MatchAllDocsQuery() : other.baseQuery, Occur.MUST)
         .add(filter, Occur.FILTER)
@@ -117,20 +116,6 @@ public final class DrillDownQuery extend
     dimQueries.get(index).add(subQuery, Occur.SHOULD);
   }
 
-  static Filter getFilter(Query query) {
-    if (query instanceof ConstantScoreQuery) {
-      ConstantScoreQuery csq = (ConstantScoreQuery) query;
-      Query sub = csq.getQuery();
-      if (sub instanceof Filter) {
-        return (Filter) sub;
-      } else {
-        return getFilter(sub);
-      }
-    } else {
-      return null;
-    }
-  }
-
   @Override
   public DrillDownQuery clone() {
     return new DrillDownQuery(config, baseQuery, dimQueries, drillDownDims);

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSideways.java Wed Jul  1 14:48:21 2015
@@ -29,7 +29,6 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.search.FilterCollector;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.MultiCollector;
@@ -178,7 +177,7 @@ public class DrillSideways {
    * drill down and sideways counts.
    */
   public DrillSidewaysResult search(DrillDownQuery query,
-                                    Filter filter, FieldDoc after, int topN, Sort sort, boolean doDocScores,
+                                    Query filter, FieldDoc after, int topN, Sort sort, boolean doDocScores,
                                     boolean doMaxScore) throws IOException {
     if (filter != null) {
       query = new DrillDownQuery(config, filter, query);

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java Wed Jul  1 14:48:21 2015
@@ -18,23 +18,24 @@ package org.apache.lucene.facet;
  */
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.Objects;
 import java.util.Set;
 
+import org.apache.lucene.facet.DrillSidewaysScorer.DocsAndCost;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BulkScorer;
 import org.apache.lucene.search.Collector;
-import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.ConstantScoreScorer;
 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;
+
 /** Only purpose is to punch through and return a
  *  DrillSidewaysScorer*/ 
 
@@ -80,17 +81,9 @@ class DrillSidewaysQuery extends Query {
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
     final Weight baseWeight = baseQuery.createWeight(searcher, needsScores);
-    final Object[] drillDowns = new Object[drillDownQueries.length];
+    final Weight[] drillDowns = new Weight[drillDownQueries.length];
     for(int dim=0;dim<drillDownQueries.length;dim++) {
-      Query query = drillDownQueries[dim];
-      Filter filter = DrillDownQuery.getFilter(query);
-      if (filter != null) {
-        drillDowns[dim] = filter;
-      } else {
-        // TODO: would be nice if we could say "we will do no
-        // scoring" here....
-        drillDowns[dim] = searcher.rewrite(query).createWeight(searcher, needsScores);
-      }
+      drillDowns[dim] = searcher.createNormalizedWeight(drillDownQueries[dim], false);
     }
 
     return new Weight(DrillSidewaysQuery.this) {
@@ -120,61 +113,18 @@ class DrillSidewaysQuery extends Query {
 
       @Override
       public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
-
-        // TODO: it could be better if we take acceptDocs
-        // into account instead of baseScorer?
         Scorer baseScorer = baseWeight.scorer(context);
 
         DrillSidewaysScorer.DocsAndCost[] dims = new DrillSidewaysScorer.DocsAndCost[drillDowns.length];
         int nullCount = 0;
         for(int dim=0;dim<dims.length;dim++) {
-          dims[dim] = new DrillSidewaysScorer.DocsAndCost();
-          dims[dim].sidewaysCollector = drillSidewaysCollectors[dim];
-          if (drillDowns[dim] instanceof Filter) {
-            // Pass null for acceptDocs because we already
-            // passed it to baseScorer and baseScorer is
-            // MUST'd here
-            DocIdSet dis = ((Filter) drillDowns[dim]).getDocIdSet(context, null);
-
-            if (dis == null) {
-              continue;
-            }
-
-            Bits bits = dis.bits();
-
-            if (bits != null) {
-              // TODO: this logic is too naive: the
-              // existence of bits() in DIS today means
-              // either "I'm a cheap FixedBitSet so apply me down
-              // low as you decode the postings" or "I'm so
-              // horribly expensive so apply me after all
-              // other Query/Filter clauses pass"
-
-              // Filter supports random access; use that to
-              // prevent .advance() on costly filters:
-              dims[dim].bits = bits;
-
-              // TODO: Filter needs to express its expected
-              // cost somehow, before pulling the iterator;
-              // we should use that here to set the order to
-              // check the filters:
-
-            } else {
-              DocIdSetIterator disi = dis.iterator();
-              if (disi == null) {
-                nullCount++;
-                continue;
-              }
-              dims[dim].disi = disi;
-            }
-          } else {
-            DocIdSetIterator disi = ((Weight) drillDowns[dim]).scorer(context);
-            if (disi == null) {
-              nullCount++;
-              continue;
-            }
-            dims[dim].disi = disi;
+          Scorer scorer = drillDowns[dim].scorer(context);
+          if (scorer == null) {
+            nullCount++;
+            scorer = new ConstantScoreScorer(drillDowns[dim], 0f, DocIdSetIterator.empty());
           }
+
+          dims[dim] = new DrillSidewaysScorer.DocsAndCost(scorer, drillSidewaysCollectors[dim]);
         }
 
         // If more than one dim has no matches, then there
@@ -187,7 +137,12 @@ class DrillSidewaysQuery extends Query {
         }
 
         // Sort drill-downs by most restrictive first:
-        Arrays.sort(dims);
+        Arrays.sort(dims, new Comparator<DrillSidewaysScorer.DocsAndCost>() {
+          @Override
+          public int compare(DocsAndCost o1, DocsAndCost o2) {
+            return Long.compare(o1.approximation.cost(), o2.approximation.cost());
+          }
+        });
 
         if (baseScorer == null) {
           return null;

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java Wed Jul  1 14:48:21 2015
@@ -28,6 +28,7 @@ import org.apache.lucene.search.Collecto
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.LeafCollector;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.FixedBitSet;
@@ -105,46 +106,18 @@ class DrillSidewaysScorer extends BulkSc
 
     long drillDownCost = 0;
     for (int dim=0;dim<numDims;dim++) {
-      DocIdSetIterator disi = dims[dim].disi;
-      if (dims[dim].bits == null && disi != null) {
-        drillDownCost += disi.cost();
-      }
+      drillDownCost += dims[dim].approximation.cost();
     }
 
     long drillDownAdvancedCost = 0;
-    if (numDims > 1 && dims[1].disi != null) {
-      drillDownAdvancedCost = dims[1].disi.cost();
+    if (numDims > 1) {
+      drillDownAdvancedCost = dims[1].approximation.cost();
     }
 
     // Position all scorers to their first matching doc:
     baseScorer.nextDoc();
-    int numBits = 0;
     for (DocsAndCost dim : dims) {
-      if (dim.disi != null) {
-        dim.disi.nextDoc();
-      } else if (dim.bits != null) {
-        numBits++;
-      }
-    }
-
-    Bits[] bits = new Bits[numBits];
-    LeafCollector[] bitsSidewaysCollectors = new LeafCollector[numBits];
-
-    DocIdSetIterator[] disis = new DocIdSetIterator[numDims-numBits];
-    LeafCollector[] sidewaysCollectors = new LeafCollector[numDims-numBits];
-    int disiUpto = 0;
-    int bitsUpto = 0;
-    for (int dim=0;dim<numDims;dim++) {
-      DocIdSetIterator disi = dims[dim].disi;
-      if (dims[dim].bits == null) {
-        disis[disiUpto] = disi;
-        sidewaysCollectors[disiUpto] = dims[dim].sidewaysLeafCollector;
-        disiUpto++;
-      } else {
-        bits[bitsUpto] = dims[dim].bits;
-        bitsSidewaysCollectors[bitsUpto] = dims[dim].sidewaysLeafCollector;
-        bitsUpto++;
-      }
+      dim.approximation.nextDoc();
     }
 
     /*
@@ -157,15 +130,15 @@ class DrillSidewaysScorer extends BulkSc
     }
     */
 
-    if (bitsUpto > 0 || scoreSubDocsAtOnce || baseQueryCost < drillDownCost/10) {
+    if (scoreSubDocsAtOnce || baseQueryCost < drillDownCost/10) {
       //System.out.println("queryFirst: baseScorer=" + baseScorer + " disis.length=" + disis.length + " bits.length=" + bits.length);
-      doQueryFirstScoring(acceptDocs, collector, disis, sidewaysCollectors, bits, bitsSidewaysCollectors);
-    } else if (numDims > 1 && (dims[1].disi == null || drillDownAdvancedCost < baseQueryCost/10)) {
+      doQueryFirstScoring(acceptDocs, collector, dims);
+    } else if (numDims > 1 && drillDownAdvancedCost < baseQueryCost/10) {
       //System.out.println("drillDownAdvance");
-      doDrillDownAdvanceScoring(acceptDocs, collector, disis, sidewaysCollectors);
+      doDrillDownAdvanceScoring(acceptDocs, collector, dims);
     } else {
       //System.out.println("union");
-      doUnionScoring(acceptDocs, collector, disis, sidewaysCollectors);
+      doUnionScoring(acceptDocs, collector, dims);
     }
 
     return Integer.MAX_VALUE;
@@ -176,8 +149,7 @@ class DrillSidewaysScorer extends BulkSc
    *  (i.e., like BooleanScorer2, not BooleanScorer).  In
    *  this case we just .next() on base and .advance() on
    *  the dim filters. */ 
-  private void doQueryFirstScoring(Bits acceptDocs, LeafCollector collector, DocIdSetIterator[] disis, LeafCollector[] sidewaysCollectors,
-                                   Bits[] bits, LeafCollector[] bitsSidewaysCollectors) throws IOException {
+  private void doQueryFirstScoring(Bits acceptDocs, LeafCollector collector, DocsAndCost[] dims) throws IOException {
     //if (DEBUG) {
     //  System.out.println("  doQueryFirstScoring");
     //}
@@ -189,36 +161,23 @@ class DrillSidewaysScorer extends BulkSc
         continue;
       }
       LeafCollector failedCollector = null;
-      for (int i=0;i<disis.length;i++) {
+      for (DocsAndCost dim : dims) {
         // TODO: should we sort this 2nd dimension of
         // docsEnums from most frequent to least?
-        DocIdSetIterator disi = disis[i];
-        if (disi != null && disi.docID() < docID) {
-          disi.advance(docID);
+        if (dim.approximation.docID() < docID) {
+          dim.approximation.advance(docID);
         }
-        if (disi == null || disi.docID() > docID) {
-          if (failedCollector != null) {
-            // More than one dim fails on this document, so
-            // it's neither a hit nor a near-miss; move to
-            // next doc:
-            docID = baseScorer.nextDoc();
-            continue nextDoc;
+
+        boolean matches = false;
+        if (dim.approximation.docID() == docID) {
+          if (dim.twoPhase == null) {
+            matches = true;
           } else {
-            failedCollector = sidewaysCollectors[i];
+            matches = dim.twoPhase.matches();
           }
         }
-      }
 
-      // TODO: for the "non-costly Bits" we really should
-      // have passed them down as acceptDocs, but
-      // unfortunately we cannot distinguish today betwen
-      // "bits() is so costly that you should apply it last"
-      // from "bits() is so cheap that you should apply it
-      // everywhere down low"
-
-      // Fold in Filter Bits last, since they may be costly:
-      for(int i=0;i<bits.length;i++) {
-        if (bits[i].get(docID) == false) {
+        if (matches == false) {
           if (failedCollector != null) {
             // More than one dim fails on this document, so
             // it's neither a hit nor a near-miss; move to
@@ -226,7 +185,7 @@ class DrillSidewaysScorer extends BulkSc
             docID = baseScorer.nextDoc();
             continue nextDoc;
           } else {
-            failedCollector = bitsSidewaysCollectors[i];
+            failedCollector = dim.sidewaysLeafCollector;
           }
         }
       }
@@ -239,7 +198,7 @@ class DrillSidewaysScorer extends BulkSc
 
       if (failedCollector == null) {
         // Hit passed all filters, so it's "real":
-        collectHit(collector, sidewaysCollectors, bitsSidewaysCollectors);
+        collectHit(collector, dims);
       } else {
         // Hit missed exactly one filter:
         collectNearMiss(failedCollector);
@@ -251,7 +210,7 @@ class DrillSidewaysScorer extends BulkSc
 
   /** Used when drill downs are highly constraining vs
    *  baseQuery. */
-  private void doDrillDownAdvanceScoring(Bits acceptDocs, LeafCollector collector, DocIdSetIterator[] disis, LeafCollector[] sidewaysCollectors) throws IOException {
+  private void doDrillDownAdvanceScoring(Bits acceptDocs, LeafCollector collector, DocsAndCost[] dims) throws IOException {
     final int maxDoc = context.reader().maxDoc();
     final int numDims = dims.length;
 
@@ -280,69 +239,66 @@ class DrillSidewaysScorer extends BulkSc
       //if (DEBUG) {
       //  System.out.println("  dim0");
       //}
-      DocIdSetIterator disi = disis[0];
-      if (disi != null) {
-        int docID = disi.docID();
-        while (docID < nextChunkStart) {
-          if (acceptDocs == null || acceptDocs.get(docID)) {
-            int slot = docID & MASK;
+      DocsAndCost dc = dims[0];
+      int docID = dc.approximation.docID();
+      while (docID < nextChunkStart) {
+        if (acceptDocs == null || acceptDocs.get(docID)) {
+          int slot = docID & MASK;
 
-            if (docIDs[slot] != docID) {
-              seen.set(slot);
-              // Mark slot as valid:
-              //if (DEBUG) {
-              //  System.out.println("    set docID=" + docID + " id=" + context.reader().document(docID).get("id"));
-              //}
-              docIDs[slot] = docID;
-              missingDims[slot] = 1;
-              counts[slot] = 1;
-            }
+          if (docIDs[slot] != docID && (dc.twoPhase == null || dc.twoPhase.matches())) {
+            seen.set(slot);
+            // Mark slot as valid:
+            //if (DEBUG) {
+            //  System.out.println("    set docID=" + docID + " id=" + context.reader().document(docID).get("id"));
+            //}
+            docIDs[slot] = docID;
+            missingDims[slot] = 1;
+            counts[slot] = 1;
           }
-
-          docID = disi.nextDoc();
         }
+
+        docID = dc.approximation.nextDoc();
       }
       
       // Second dim:
       //if (DEBUG) {
       //  System.out.println("  dim1");
       //}
-      disi = disis[1];
-      if (disi != null) {
-        int docID = disi.docID();
-        while (docID < nextChunkStart) {
-          if (acceptDocs == null || acceptDocs.get(docID)) {
-            int slot = docID & MASK;
+      dc = dims[1];
+      docID = dc.approximation.docID();
+      while (docID < nextChunkStart) {
+        if (acceptDocs == null || acceptDocs.get(docID)
+            && (dc.twoPhase == null || dc.twoPhase.matches())) {
+          int slot = docID & MASK;
 
-            if (docIDs[slot] != docID) {
-              // Mark slot as valid:
-              seen.set(slot);
+          if (docIDs[slot] != docID) {
+            // Mark slot as valid:
+            seen.set(slot);
+            //if (DEBUG) {
+            //  System.out.println("    set docID=" + docID + " missingDim=0 id=" + context.reader().document(docID).get("id"));
+            //}
+            docIDs[slot] = docID;
+            missingDims[slot] = 0;
+            counts[slot] = 1;
+          } else {
+            // TODO: single-valued dims will always be true
+            // below; we could somehow specialize
+            if (missingDims[slot] >= 1) {
+              missingDims[slot] = 2;
+              counts[slot] = 2;
               //if (DEBUG) {
-              //  System.out.println("    set docID=" + docID + " missingDim=0 id=" + context.reader().document(docID).get("id"));
+              //  System.out.println("    set docID=" + docID + " missingDim=2 id=" + context.reader().document(docID).get("id"));
               //}
-              docIDs[slot] = docID;
-              missingDims[slot] = 0;
-              counts[slot] = 1;
             } else {
-              // TODO: single-valued dims will always be true
-              // below; we could somehow specialize
-              if (missingDims[slot] >= 1) {
-                missingDims[slot] = 2;
-                counts[slot] = 2;
-                //if (DEBUG) {
-                //  System.out.println("    set docID=" + docID + " missingDim=2 id=" + context.reader().document(docID).get("id"));
-                //}
-              } else {
-                counts[slot] = 1;
-                //if (DEBUG) {
-                //  System.out.println("    set docID=" + docID + " missingDim=" + missingDims[slot] + " id=" + context.reader().document(docID).get("id"));
-                //}
-              }
+              counts[slot] = 1;
+              //if (DEBUG) {
+              //  System.out.println("    set docID=" + docID + " missingDim=" + missingDims[slot] + " id=" + context.reader().document(docID).get("id"));
+              //}
             }
           }
-
-          docID = disi.nextDoc();
         }
+
+        docID = dc.approximation.nextDoc();
       }
 
       // After this we can "upgrade" to conjunction, because
@@ -399,31 +355,31 @@ class DrillSidewaysScorer extends BulkSc
         //if (DEBUG) {
         //  System.out.println("  dim=" + dim + " [" + dims[dim].dim + "]");
         //}
-        disi = disis[dim];
-        if (disi != null) {
-          int docID = disi.docID();
-          while (docID < nextChunkStart) {
-            int slot = docID & MASK;
-            if (docIDs[slot] == docID && counts[slot] >= dim) {
-              // TODO: single-valued dims will always be true
-              // below; we could somehow specialize
-              if (missingDims[slot] >= dim) {
-                //if (DEBUG) {
-                //  System.out.println("    set docID=" + docID + " count=" + (dim+2));
-                //}
-                missingDims[slot] = dim+1;
-                counts[slot] = dim+2;
-              } else {
-                //if (DEBUG) {
-                //  System.out.println("    set docID=" + docID + " missing count=" + (dim+1));
-                //}
-                counts[slot] = dim+1;
-              }
+        dc = dims[dim];
+        docID = dc.approximation.docID();
+        while (docID < nextChunkStart) {
+          int slot = docID & MASK;
+          if (docIDs[slot] == docID
+              && counts[slot] >= dim
+              && (dc.twoPhase == null || dc.twoPhase.matches())) {
+            // TODO: single-valued dims will always be true
+            // below; we could somehow specialize
+            if (missingDims[slot] >= dim) {
+              //if (DEBUG) {
+              //  System.out.println("    set docID=" + docID + " count=" + (dim+2));
+              //}
+              missingDims[slot] = dim+1;
+              counts[slot] = dim+2;
+            } else {
+              //if (DEBUG) {
+              //  System.out.println("    set docID=" + docID + " missing count=" + (dim+1));
+              //}
+              counts[slot] = dim+1;
             }
-
-            // TODO: sometimes use advance?
-            docID = disi.nextDoc();
           }
+
+          // TODO: sometimes use advance?
+          docID = dc.approximation.nextDoc();
         }
       }
 
@@ -439,9 +395,9 @@ class DrillSidewaysScorer extends BulkSc
         //  System.out.println("    docID=" + docIDs[slot] + " count=" + counts[slot]);
         //}
         if (counts[slot] == 1+numDims) {
-          collectHit(collector, sidewaysCollectors);
+          collectHit(collector, dims);
         } else if (counts[slot] == numDims) {
-          collectNearMiss(sidewaysCollectors[missingDims[slot]]);
+          collectNearMiss(dims[missingDims[slot]].sidewaysLeafCollector);
         }
       }
 
@@ -453,7 +409,7 @@ class DrillSidewaysScorer extends BulkSc
     }
   }
 
-  private void doUnionScoring(Bits acceptDocs, LeafCollector collector, DocIdSetIterator[] disis, LeafCollector[] sidewaysCollectors) throws IOException {
+  private void doUnionScoring(Bits acceptDocs, LeafCollector collector, DocsAndCost[] dims) throws IOException {
     //if (DEBUG) {
     //  System.out.println("  doUnionScoring");
     //}
@@ -516,22 +472,23 @@ class DrillSidewaysScorer extends BulkSc
       //if (DEBUG) {
       //  System.out.println("  dim=0 [" + dims[0].dim + "]");
       //}
-      DocIdSetIterator disi = disis[0];
-      if (disi != null) {
-        docID = disi.docID();
+      {
+        DocsAndCost dc = dims[0];
+        docID = dc.approximation.docID();
         //if (DEBUG) {
         //  System.out.println("    start docID=" + docID);
         //}
         while (docID < nextChunkStart) {
           int slot = docID & MASK;
-          if (docIDs[slot] == docID) { // this also checks that the doc is not deleted
+          if (docIDs[slot] == docID // this also checks that the doc is not deleted
+              && (dc.twoPhase == null || dc.twoPhase.matches())) {
             //if (DEBUG) {
             //  System.out.println("      set docID=" + docID + " count=2");
             //}
             missingDims[slot] = 1;
             counts[slot] = 2;
           }
-          docID = disi.nextDoc();
+          docID = dc.approximation.nextDoc();
         }
       }
 
@@ -540,34 +497,33 @@ class DrillSidewaysScorer extends BulkSc
         //  System.out.println("  dim=" + dim + " [" + dims[dim].dim + "]");
         //}
 
-        disi = disis[dim];
-        if (disi != null) {
-          docID = disi.docID();
-          //if (DEBUG) {
-          //  System.out.println("    start docID=" + docID);
-          //}
-          while (docID < nextChunkStart) {
-            int slot = docID & MASK;
-            if (docIDs[slot] == docID // also means that the doc is not deleted
-                && counts[slot] >= dim) {
-              // This doc is still in the running...
-              // TODO: single-valued dims will always be true
-              // below; we could somehow specialize
-              if (missingDims[slot] >= dim) {
-                //if (DEBUG) {
-                //  System.out.println("      set docID=" + docID + " count=" + (dim+2));
-                //}
-                missingDims[slot] = dim+1;
-                counts[slot] = dim+2;
-              } else {
-                //if (DEBUG) {
-                //  System.out.println("      set docID=" + docID + " missing count=" + (dim+1));
-                //}
-                counts[slot] = dim+1;
-              }
+        DocsAndCost dc = dims[dim];
+        docID = dc.approximation.docID();
+        //if (DEBUG) {
+        //  System.out.println("    start docID=" + docID);
+        //}
+        while (docID < nextChunkStart) {
+          int slot = docID & MASK;
+          if (docIDs[slot] == docID // also means that the doc is not deleted
+              && counts[slot] >= dim
+              && (dc.twoPhase == null || dc.twoPhase.matches())) {
+            // This doc is still in the running...
+            // TODO: single-valued dims will always be true
+            // below; we could somehow specialize
+            if (missingDims[slot] >= dim) {
+              //if (DEBUG) {
+              //  System.out.println("      set docID=" + docID + " count=" + (dim+2));
+              //}
+              missingDims[slot] = dim+1;
+              counts[slot] = dim+2;
+            } else {
+              //if (DEBUG) {
+              //  System.out.println("      set docID=" + docID + " missing count=" + (dim+1));
+              //}
+              counts[slot] = dim+1;
             }
-            docID = disi.nextDoc();
           }
+          docID = dc.approximation.nextDoc();
         }
       }
 
@@ -586,10 +542,10 @@ class DrillSidewaysScorer extends BulkSc
         //System.out.println("  collect doc=" + collectDocID + " main.freq=" + (counts[slot]-1) + " main.doc=" + collectDocID + " exactCount=" + numDims);
         if (counts[slot] == 1+numDims) {
           //System.out.println("    hit");
-          collectHit(collector, sidewaysCollectors);
+          collectHit(collector, dims);
         } else if (counts[slot] == numDims) {
           //System.out.println("    sw");
-          collectNearMiss(sidewaysCollectors[missingDims[slot]]);
+          collectNearMiss(dims[missingDims[slot]].sidewaysLeafCollector);
         }
       }
 
@@ -601,27 +557,7 @@ class DrillSidewaysScorer extends BulkSc
     }
   }
 
-  private void collectHit(LeafCollector collector, LeafCollector[] sidewaysCollectors) throws IOException {
-    //if (DEBUG) {
-    //  System.out.println("      hit");
-    //}
-
-    collector.collect(collectDocID);
-    if (drillDownCollector != null) {
-      drillDownLeafCollector.collect(collectDocID);
-    }
-
-    // TODO: we could "fix" faceting of the sideways counts
-    // to do this "union" (of the drill down hits) in the
-    // end instead:
-
-    // Tally sideways counts:
-    for (int dim=0;dim<sidewaysCollectors.length;dim++) {
-      sidewaysCollectors[dim].collect(collectDocID);
-    }
-  }
-
-  private void collectHit(LeafCollector collector, LeafCollector[] sidewaysCollectors, LeafCollector[] sidewaysCollectors2) throws IOException {
+  private void collectHit(LeafCollector collector, DocsAndCost[] dims) throws IOException {
     //if (DEBUG) {
     //  System.out.println("      hit");
     //}
@@ -636,11 +572,8 @@ class DrillSidewaysScorer extends BulkSc
     // end instead:
 
     // Tally sideways counts:
-    for (int i=0;i<sidewaysCollectors.length;i++) {
-      sidewaysCollectors[i].collect(collectDocID);
-    }
-    for (int i=0;i<sidewaysCollectors2.length;i++) {
-      sidewaysCollectors2[i].collect(collectDocID);
+    for (DocsAndCost dim : dims) {
+      dim.sidewaysLeafCollector.collect(collectDocID);
     }
   }
 
@@ -698,32 +631,24 @@ class DrillSidewaysScorer extends BulkSc
     }
   }
 
-  static class DocsAndCost implements Comparable<DocsAndCost> {
-    // Iterator for docs matching this dim's filter, or ...
-    DocIdSetIterator disi;
-    // Random access bits:
-    Bits bits;
-    Collector sidewaysCollector;
+  static class DocsAndCost {
+    // approximation of matching docs, or the scorer itself
+    final DocIdSetIterator approximation;
+    // two-phase confirmation, or null if the approximation is accurate
+    final TwoPhaseIterator twoPhase;
+    final Collector sidewaysCollector;
     LeafCollector sidewaysLeafCollector;
-    String dim;
 
-    @Override
-    public int compareTo(DocsAndCost other) {
-      if (disi == null) {
-        if (other.disi == null) {
-          return 0;
-        } else {
-          return 1;
-        }
-      } else if (other.disi == null) {
-        return -1;
-      } else if (disi.cost() < other.disi.cost()) {
-        return -1;
-      } else if (disi.cost() > other.disi.cost()) {
-        return 1;
+    DocsAndCost(Scorer scorer, Collector sidewaysCollector) {
+      final TwoPhaseIterator twoPhase = scorer.asTwoPhaseIterator();
+      if (twoPhase == null) {
+        this.approximation = scorer;
+        this.twoPhase = null;
       } else {
-        return 0;
+        this.approximation = twoPhase.approximation();
+        this.twoPhase = twoPhase;
       }
+      this.sidewaysCollector = sidewaysCollector;
     }
   }
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/FacetsCollector.java Wed Jul  1 14:48:21 2015
@@ -22,12 +22,9 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Collector;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.FieldDoc;
-import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MultiCollector;
 import org.apache.lucene.search.Query;
@@ -196,73 +193,54 @@ public class FacetsCollector extends Sim
   /** Utility method, to search and also collect all hits
    *  into the provided {@link Collector}. */
   public static TopDocs search(IndexSearcher searcher, Query q, int n, Collector fc) throws IOException {
-    return doSearch(searcher, null, q, null, n, null, false, false, fc);
+    return doSearch(searcher, null, q, n, null, false, false, fc);
   }
 
   /** Utility method, to search and also collect all hits
    *  into the provided {@link Collector}. */
-  public static TopDocs search(IndexSearcher searcher, Query q, Filter filter, int n, Collector fc) throws IOException {
-    return doSearch(searcher, null, q, filter, n, null, false, false, fc);
-  }
-
-  /** Utility method, to search and also collect all hits
-   *  into the provided {@link Collector}. */
-  public static TopFieldDocs search(IndexSearcher searcher, Query q, Filter filter, int n, Sort sort, Collector fc) throws IOException {
+  public static TopFieldDocs search(IndexSearcher searcher, Query q, int n, Sort sort, Collector fc) throws IOException {
     if (sort == null) {
       throw new IllegalArgumentException("sort must not be null");
     }
-    return (TopFieldDocs) doSearch(searcher, null, q, filter, n, sort, false, false, fc);
+    return (TopFieldDocs) doSearch(searcher, null, q, n, sort, false, false, fc);
   }
 
   /** Utility method, to search and also collect all hits
    *  into the provided {@link Collector}. */
-  public static TopFieldDocs search(IndexSearcher searcher, Query q, Filter filter, int n, Sort sort, boolean doDocScores, boolean doMaxScore, Collector fc) throws IOException {
+  public static TopFieldDocs search(IndexSearcher searcher, Query q, int n, Sort sort, boolean doDocScores, boolean doMaxScore, Collector fc) throws IOException {
     if (sort == null) {
       throw new IllegalArgumentException("sort must not be null");
     }
-    return (TopFieldDocs) doSearch(searcher, null, q, filter, n, sort, doDocScores, doMaxScore, fc);
-  }
-
-  /** Utility method, to search and also collect all hits
-   *  into the provided {@link Collector}. */
-  public TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, int n, Collector fc) throws IOException {
-    return doSearch(searcher, after, q, null, n, null, false, false, fc);
+    return (TopFieldDocs) doSearch(searcher, null, q, n, sort, doDocScores, doMaxScore, fc);
   }
 
   /** Utility method, to search and also collect all hits
    *  into the provided {@link Collector}. */
-  public static TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, Filter filter, int n, Collector fc) throws IOException {
-    return doSearch(searcher, after, q, filter, n, null, false, false, fc);
+  public static TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, int n, Collector fc) throws IOException {
+    return doSearch(searcher, after, q, n, null, false, false, fc);
   }
 
   /** Utility method, to search and also collect all hits
    *  into the provided {@link Collector}. */
-  public static TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, Filter filter, int n, Sort sort, Collector fc) throws IOException {
+  public static TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, int n, Sort sort, Collector fc) throws IOException {
     if (sort == null) {
       throw new IllegalArgumentException("sort must not be null");
     }
-    return doSearch(searcher, after, q, filter, n, sort, false, false, fc);
+    return doSearch(searcher, after, q, n, sort, false, false, fc);
   }
 
   /** Utility method, to search and also collect all hits
    *  into the provided {@link Collector}. */
-  public static TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, Filter filter, int n, Sort sort, boolean doDocScores, boolean doMaxScore, Collector fc) throws IOException {
+  public static TopDocs searchAfter(IndexSearcher searcher, ScoreDoc after, Query q, int n, Sort sort, boolean doDocScores, boolean doMaxScore, Collector fc) throws IOException {
     if (sort == null) {
       throw new IllegalArgumentException("sort must not be null");
     }
-    return doSearch(searcher, after, q, filter, n, sort, doDocScores, doMaxScore, fc);
+    return doSearch(searcher, after, q, n, sort, doDocScores, doMaxScore, fc);
   }
 
-  private static TopDocs doSearch(IndexSearcher searcher, ScoreDoc after, Query q, Filter filter, int n, Sort sort,
+  private static TopDocs doSearch(IndexSearcher searcher, ScoreDoc after, Query q, int n, Sort sort,
                                   boolean doDocScores, boolean doMaxScore, Collector fc) throws IOException {
 
-    if (filter != null) {
-      q = new BooleanQuery.Builder()
-          .add(q, Occur.MUST)
-          .add(filter, Occur.FILTER)
-          .build();
-    }
-
     int limit = searcher.getIndexReader().maxDoc();
     if (limit == 0) {
       limit = 1;

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java Wed Jul  1 14:48:21 2015
@@ -21,14 +21,18 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Objects;
 
+import org.apache.lucene.index.IndexReader;
 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.DocIdSet;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.FilteredDocIdSet;
-import org.apache.lucene.util.Bits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.NumericUtils;
 
 /** Represents a range over double values.
@@ -102,15 +106,14 @@ public final class DoubleRange extends R
     return "DoubleRange(" + minIncl + " to " + maxIncl + ")";
   }
 
-  private static class ValueSourceFilter extends Filter {
+  private static class ValueSourceQuery extends Query {
     private final DoubleRange range;
-    private final Filter fastMatchFilter;
+    private final Query fastMatchQuery;
     private final ValueSource valueSource;
 
-    ValueSourceFilter(DoubleRange range, Filter fastMatchFilter, ValueSource valueSource) {
-      super(true);
+    ValueSourceQuery(DoubleRange range, Query fastMatchQuery, ValueSource valueSource) {
       this.range = range;
-      this.fastMatchFilter = fastMatchFilter;
+      this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
     }
 
@@ -119,15 +122,15 @@ public final class DoubleRange extends R
       if (super.equals(obj) == false) {
         return false;
       }
-      ValueSourceFilter other = (ValueSourceFilter) obj;
+      ValueSourceQuery other = (ValueSourceQuery) obj;
       return range.equals(other.range)
-          && Objects.equals(fastMatchFilter, other.fastMatchFilter)
+          && Objects.equals(fastMatchQuery, other.fastMatchQuery)
           && valueSource.equals(other.valueSource);
     }
 
     @Override
     public int hashCode() {
-      return 31 * Objects.hash(range, fastMatchFilter, valueSource) + super.hashCode();
+      return 31 * Objects.hash(range, fastMatchQuery, valueSource) + super.hashCode();
     }
 
     @Override
@@ -136,52 +139,56 @@ public final class DoubleRange extends R
     }
 
     @Override
-    public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
-
-      // TODO: this is just like ValueSourceScorer,
-      // ValueSourceFilter (spatial),
-      // ValueSourceRangeFilter (solr); also,
-      // https://issues.apache.org/jira/browse/LUCENE-4251
-
-      final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
-
-      final int maxDoc = context.reader().maxDoc();
-
-      final DocIdSet fastMatchDocs;
-      if (fastMatchFilter != null) {
-        fastMatchDocs = fastMatchFilter.getDocIdSet(context, null);
-        if (fastMatchDocs == null) {
-          // No documents match
-          return null;
+    public Query rewrite(IndexReader reader) throws IOException {
+      if (fastMatchQuery != null) {
+        final Query fastMatchRewritten = fastMatchQuery.rewrite(reader);
+        if (fastMatchRewritten != fastMatchQuery) {
+          Query rewritten = new ValueSourceQuery(range, fastMatchRewritten, valueSource);
+          rewritten.setBoost(getBoost());
+          return rewritten;
         }
-      } else {
-        fastMatchDocs = new DocIdSet() {
-          @Override
-          public long ramBytesUsed() {
-            return 0;
-          }
-          @Override
-          public DocIdSetIterator iterator() throws IOException {
-            return DocIdSetIterator.all(maxDoc);
-          }
-        };
       }
+      return super.rewrite(reader);
+    }
 
-      return new FilteredDocIdSet(fastMatchDocs) {
+    @Override
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      final Weight fastMatchWeight = fastMatchQuery == null
+          ? null
+          : searcher.createWeight(fastMatchQuery, false);
+
+      return new ConstantScoreWeight(this) {
         @Override
-        protected boolean match(int docID) {
-          if (acceptDocs != null && acceptDocs.get(docID) == false) {
-            return false;
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          final int maxDoc = context.reader().maxDoc();
+
+          final DocIdSetIterator approximation;
+          if (fastMatchWeight == null) {
+            approximation = DocIdSetIterator.all(maxDoc);
+          } else {
+            approximation = fastMatchWeight.scorer(context);
+            if (approximation == null) {
+              return null;
+            }
           }
-          return range.accept(values.doubleVal(docID));
+
+          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
+            @Override
+            public boolean matches() throws IOException {
+              return range.accept(values.doubleVal(approximation.docID()));
+            }
+          };
+          return new ConstantScoreScorer(this, score(), twoPhase);
         }
       };
     }
+
   }
 
   @Override
-  public Filter getFilter(final Filter fastMatchFilter, final ValueSource valueSource) {
-    return new ValueSourceFilter(this, fastMatchFilter, valueSource);
+  public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }
 

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRangeFacetCounts.java Wed Jul  1 14:48:21 2015
@@ -26,14 +26,17 @@ import org.apache.lucene.document.FloatD
 import org.apache.lucene.facet.Facets;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.DoubleFieldSource;
 import org.apache.lucene.queries.function.valuesource.FloatFieldSource; // javadocs
 import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.NumericUtils;
 
 /** {@link Facets} implementation that computes counts for
@@ -68,12 +71,12 @@ public class DoubleRangeFacetCounts exte
   }
 
   /** Create {@code RangeFacetCounts}, using the provided
-   *  {@link ValueSource}, and using the provided Filter as
+   *  {@link ValueSource}, and using the provided Query as
    *  a fastmatch: only documents passing the filter are
    *  checked for the matching ranges.  The filter must be
    *  random access (implement {@link DocIdSet#bits}). */
-  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Filter fastMatchFilter, DoubleRange... ranges) throws IOException {
-    super(field, ranges, fastMatchFilter);
+  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, DoubleRange... ranges) throws IOException {
+    super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
@@ -97,13 +100,15 @@ public class DoubleRangeFacetCounts exte
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
-      if (fastMatchFilter != null) {
-        DocIdSet dis = fastMatchFilter.getDocIdSet(hits.context, null);
-        if (dis == null) {
-          // No documents match
+      if (fastMatchQuery != null) {
+        final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
+        final IndexSearcher searcher = new IndexSearcher(topLevelContext);
+        searcher.setQueryCache(null);
+        final Weight fastMatchWeight = searcher.createNormalizedWeight(fastMatchQuery, false);
+        fastMatchDocs = fastMatchWeight.scorer(hits.context);
+        if (fastMatchDocs == null) {
           continue;
         }
-        fastMatchDocs = dis.iterator();
       } else {
         fastMatchDocs = null;
       }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java Wed Jul  1 14:48:21 2015
@@ -21,14 +21,18 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Objects;
 
+import org.apache.lucene.index.IndexReader;
 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.DocIdSet;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.FilteredDocIdSet;
-import org.apache.lucene.util.Bits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
 
 /** Represents a range over long values.
  *
@@ -94,15 +98,14 @@ public final class LongRange extends Ran
     return "LongRange(" + minIncl + " to " + maxIncl + ")";
   }
 
-  private static class ValueSourceFilter extends Filter {
+  private static class ValueSourceQuery extends Query {
     private final LongRange range;
-    private final Filter fastMatchFilter;
+    private final Query fastMatchQuery;
     private final ValueSource valueSource;
 
-    ValueSourceFilter(LongRange range, Filter fastMatchFilter, ValueSource valueSource) {
-      super(true);
+    ValueSourceQuery(LongRange range, Query fastMatchQuery, ValueSource valueSource) {
       this.range = range;
-      this.fastMatchFilter = fastMatchFilter;
+      this.fastMatchQuery = fastMatchQuery;
       this.valueSource = valueSource;
     }
 
@@ -111,15 +114,15 @@ public final class LongRange extends Ran
       if (super.equals(obj) == false) {
         return false;
       }
-      ValueSourceFilter other = (ValueSourceFilter) obj;
+      ValueSourceQuery other = (ValueSourceQuery) obj;
       return range.equals(other.range)
-          && Objects.equals(fastMatchFilter, other.fastMatchFilter)
+          && Objects.equals(fastMatchQuery, other.fastMatchQuery)
           && valueSource.equals(other.valueSource);
     }
 
     @Override
     public int hashCode() {
-      return 31 * Objects.hash(range, fastMatchFilter, valueSource) + super.hashCode();
+      return 31 * Objects.hash(range, fastMatchQuery, valueSource) + super.hashCode();
     }
 
     @Override
@@ -128,51 +131,55 @@ public final class LongRange extends Ran
     }
 
     @Override
-    public DocIdSet getDocIdSet(LeafReaderContext context, final Bits acceptDocs) throws IOException {
-
-      // TODO: this is just like ValueSourceScorer,
-      // ValueSourceFilter (spatial),
-      // ValueSourceRangeFilter (solr); also,
-      // https://issues.apache.org/jira/browse/LUCENE-4251
-
-      final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
-
-      final int maxDoc = context.reader().maxDoc();
-
-      final DocIdSet fastMatchDocs;
-      if (fastMatchFilter != null) {
-        fastMatchDocs = fastMatchFilter.getDocIdSet(context, null);
-        if (fastMatchDocs == null) {
-          // No documents match
-          return null;
+    public Query rewrite(IndexReader reader) throws IOException {
+      if (fastMatchQuery != null) {
+        final Query fastMatchRewritten = fastMatchQuery.rewrite(reader);
+        if (fastMatchRewritten != fastMatchQuery) {
+          Query rewritten = new ValueSourceQuery(range, fastMatchRewritten, valueSource);
+          rewritten.setBoost(getBoost());
+          return rewritten;
         }
-      } else {
-        fastMatchDocs = new DocIdSet() {
-          @Override
-          public long ramBytesUsed() {
-            return 0;
-          }
-          @Override
-          public DocIdSetIterator iterator() throws IOException {
-            return DocIdSetIterator.all(maxDoc);
-          }
-        };
       }
+      return super.rewrite(reader);
+    }
 
-      return new FilteredDocIdSet(fastMatchDocs) {
+    @Override
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      final Weight fastMatchWeight = fastMatchQuery == null
+          ? null
+          : searcher.createWeight(fastMatchQuery, false);
+
+      return new ConstantScoreWeight(this) {
         @Override
-        protected boolean match(int docID) {
-          if (acceptDocs != null && acceptDocs.get(docID) == false) {
-            return false;
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          final int maxDoc = context.reader().maxDoc();
+
+          final DocIdSetIterator approximation;
+          if (fastMatchWeight == null) {
+            approximation = DocIdSetIterator.all(maxDoc);
+          } else {
+            approximation = fastMatchWeight.scorer(context);
+            if (approximation == null) {
+              return null;
+            }
           }
-          return range.accept(values.longVal(docID));
+
+          final FunctionValues values = valueSource.getValues(Collections.emptyMap(), context);
+          final TwoPhaseIterator twoPhase = new TwoPhaseIterator(approximation) {
+            @Override
+            public boolean matches() throws IOException {
+              return range.accept(values.longVal(approximation.docID()));
+            }
+          };
+          return new ConstantScoreScorer(this, score(), twoPhase);
         }
       };
     }
+
   }
 
   @Override
-  public Filter getFilter(final Filter fastMatchFilter, final ValueSource valueSource) {
-    return new ValueSourceFilter(this, fastMatchFilter, valueSource);
+  public Query getQuery(final Query fastMatchQuery, final ValueSource valueSource) {
+    return new ValueSourceQuery(this, fastMatchQuery, valueSource);
   }
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/LongRangeFacetCounts.java Wed Jul  1 14:48:21 2015
@@ -24,13 +24,16 @@ import java.util.List;
 import org.apache.lucene.facet.Facets;
 import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.Filter;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Weight;
 
 /** {@link Facets} implementation that computes counts for
  *  dynamic long ranges from a provided {@link ValueSource},
@@ -61,8 +64,8 @@ public class LongRangeFacetCounts extend
    *  a fastmatch: only documents passing the filter are
    *  checked for the matching ranges.  The filter must be
    *  random access (implement {@link DocIdSet#bits}). */
-  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Filter fastMatchFilter, LongRange... ranges) throws IOException {
-    super(field, ranges, fastMatchFilter);
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Query fastMatchQuery, LongRange... ranges) throws IOException {
+    super(field, ranges, fastMatchQuery);
     count(valueSource, hits.getMatchingDocs());
   }
 
@@ -78,13 +81,15 @@ public class LongRangeFacetCounts extend
       
       totCount += hits.totalHits;
       final DocIdSetIterator fastMatchDocs;
-      if (fastMatchFilter != null) {
-        DocIdSet dis = fastMatchFilter.getDocIdSet(hits.context, null);
-        if (dis == null) {
-          // No documents match
+      if (fastMatchQuery != null) {
+        final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(hits.context);
+        final IndexSearcher searcher = new IndexSearcher(topLevelContext);
+        searcher.setQueryCache(null);
+        final Weight fastMatchWeight = searcher.createNormalizedWeight(fastMatchQuery, false);
+        fastMatchDocs = fastMatchWeight.scorer(hits.context);
+        if (fastMatchDocs == null) {
           continue;
         }
-        fastMatchDocs = dis.iterator();
       } else {
         fastMatchDocs = null;
       }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java Wed Jul  1 14:48:21 2015
@@ -18,10 +18,9 @@ package org.apache.lucene.facet.range;
  */
 
 import org.apache.lucene.facet.DrillDownQuery; // javadocs
-import org.apache.lucene.facet.DrillSideways; // javadocs
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
 
 /** Base class for a single labeled range.
  *
@@ -39,26 +38,31 @@ public abstract class Range {
     this.label = label;
   }
 
-  /** Returns a new {@link Filter} accepting only documents
-   *  in this range.  This filter is not general-purpose;
-   *  you should either use it with {@link DrillSideways} by
-   *  adding it to {@link DrillDownQuery#add}.  If the
-   *  {@link ValueSource} is static, e.g. an indexed numeric
-   *  field, then it may be more efficient to use {@link
-   *  NumericRangeQuery}.  The provided fastMatchFilter,
+  /** Returns a new {@link Query} accepting only documents
+   *  in this range.  This query might not be very efficient
+   *  when run on its own since it is optimized towards
+   *  random-access, so it is best used either with
+   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
+   *  or when intersected with another query that can lead the
+   *  iteration.  If the {@link ValueSource} is static, e.g. an
+   *  indexed numeric field, then it may be more efficient to use
+   *  {@link NumericRangeQuery}. The provided fastMatchQuery,
    *  if non-null, will first be consulted, and only if
    *  that is set for each document will the range then be
    *  checked. */
-  public abstract Filter getFilter(Filter fastMatchFilter, ValueSource valueSource);
+  public abstract Query getQuery(Query fastMatchQuery, ValueSource valueSource);
 
-  /** Returns a new {@link Filter} accepting only documents
-   *  in this range.  This filter is not general-purpose;
-   *  you should either use it with {@link DrillSideways} by
-   *  adding it to {@link DrillDownQuery#add}.  If the
-   *  {@link ValueSource} is static, e.g. an indexed numeric
-   *  field, then it may be more efficient to use {@link NumericRangeQuery}. */
-  public Filter getFilter(ValueSource valueSource) {
-    return getFilter(null, valueSource);
+  /** Returns a new {@link Query} accepting only documents
+   *  in this range.  This query might not be very efficient
+   *  when run on its own since it is optimized towards
+   *  random-access, so it is best used either with
+   *  {@link DrillDownQuery#add(String, Query) DrillDownQuery}
+   *  or when intersected with another query that can lead the
+   *  iteration.  If the {@link ValueSource} is static, e.g. an
+   *  indexed numeric field, then it may be more efficient to
+   *  use {@link NumericRangeQuery}. */
+  public Query getQuery(ValueSource valueSource) {
+    return getQuery(null, valueSource);
   }
 
   /** Invoke this for a useless range. */

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java Wed Jul  1 14:48:21 2015
@@ -24,7 +24,7 @@ import java.util.List;
 import org.apache.lucene.facet.FacetResult;
 import org.apache.lucene.facet.Facets;
 import org.apache.lucene.facet.LabelAndValue;
-import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
 
 /** Base class for range faceting.
  *
@@ -36,11 +36,11 @@ abstract class RangeFacetCounts extends
   /** Counts, initialized in by subclass. */
   protected final int[] counts;
 
-  /** Optional: if specified, we first test this Filter to
+  /** Optional: if specified, we first test this Query to
    *  see whether the document should be checked for
    *  matching ranges.  If this is null, all documents are
    *  checked. */
-  protected final Filter fastMatchFilter;
+  protected final Query fastMatchQuery;
 
   /** Our field name. */
   protected final String field;
@@ -49,10 +49,10 @@ abstract class RangeFacetCounts extends
   protected int totCount;
 
   /** Create {@code RangeFacetCounts} */
-  protected RangeFacetCounts(String field, Range[] ranges, Filter fastMatchFilter) throws IOException {
+  protected RangeFacetCounts(String field, Range[] ranges, Query fastMatchQuery) throws IOException {
     this.field = field;
     this.ranges = ranges;
-    this.fastMatchFilter = fastMatchFilter;
+    this.fastMatchQuery = fastMatchQuery;
     counts = new int[ranges.length];
   }
 

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java?rev=1688661&r1=1688660&r2=1688661&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeFacetCounts.java Wed Jul  1 14:48:21 2015
@@ -20,6 +20,7 @@ package org.apache.lucene.facet.range;
 import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.lucene.document.Document;
@@ -48,6 +49,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
@@ -55,11 +57,13 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.search.DocIdSet;
-import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.NumericRangeQuery;
-import org.apache.lucene.search.QueryWrapperFilter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.IOUtils;
@@ -472,18 +476,18 @@ public class TestRangeFacetCounts extend
 
       FacetsCollector sfc = new FacetsCollector();
       s.search(new MatchAllDocsQuery(), sfc);
-      Filter fastMatchFilter;
+      Query fastMatchQuery;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchFilter = new QueryWrapperFilter(NumericRangeQuery.newLongRange("field", minValue, maxValue, true, true));
+          fastMatchQuery = NumericRangeQuery.newLongRange("field", minValue, maxValue, true, true);
         } else {
-          fastMatchFilter = new QueryWrapperFilter(NumericRangeQuery.newLongRange("field", minAcceptedValue, maxAcceptedValue, true, true));
+          fastMatchQuery = NumericRangeQuery.newLongRange("field", minAcceptedValue, maxAcceptedValue, true, true);
         }
       } else {
-        fastMatchFilter = null;
+        fastMatchQuery = null;
       }
       ValueSource vs = new LongFieldSource("field");
-      Facets facets = new LongRangeFacetCounts("field", vs, sfc, fastMatchFilter, ranges);
+      Facets facets = new LongRangeFacetCounts("field", vs, sfc, fastMatchQuery, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
       for(int rangeID=0;rangeID<numRange;rangeID++) {
@@ -501,7 +505,7 @@ public class TestRangeFacetCounts extend
         if (random().nextBoolean()) {
           ddq.add("field", NumericRangeQuery.newLongRange("field", range.min, range.max, range.minInclusive, range.maxInclusive));
         } else {
-          ddq.add("field", range.getFilter(fastMatchFilter, vs));
+          ddq.add("field", range.getQuery(fastMatchQuery, vs));
         }
         assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
       }
@@ -627,18 +631,18 @@ public class TestRangeFacetCounts extend
 
       FacetsCollector sfc = new FacetsCollector();
       s.search(new MatchAllDocsQuery(), sfc);
-      Filter fastMatchFilter;
+      Query fastMatchQuery;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchFilter = new QueryWrapperFilter(NumericRangeQuery.newFloatRange("field", minValue, maxValue, true, true));
+          fastMatchQuery = NumericRangeQuery.newFloatRange("field", minValue, maxValue, true, true);
         } else {
-          fastMatchFilter = new QueryWrapperFilter(NumericRangeQuery.newFloatRange("field", minAcceptedValue, maxAcceptedValue, true, true));
+          fastMatchQuery = NumericRangeQuery.newFloatRange("field", minAcceptedValue, maxAcceptedValue, true, true);
         }
       } else {
-        fastMatchFilter = null;
+        fastMatchQuery = null;
       }
       ValueSource vs = new FloatFieldSource("field");
-      Facets facets = new DoubleRangeFacetCounts("field", vs, sfc, fastMatchFilter, ranges);
+      Facets facets = new DoubleRangeFacetCounts("field", vs, sfc, fastMatchQuery, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
       for(int rangeID=0;rangeID<numRange;rangeID++) {
@@ -656,7 +660,7 @@ public class TestRangeFacetCounts extend
         if (random().nextBoolean()) {
           ddq.add("field", NumericRangeQuery.newFloatRange("field", (float) range.min, (float) range.max, range.minInclusive, range.maxInclusive));
         } else {
-          ddq.add("field", range.getFilter(fastMatchFilter, vs));
+          ddq.add("field", range.getQuery(fastMatchQuery, vs));
         }
         assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
       }
@@ -766,12 +770,12 @@ public class TestRangeFacetCounts extend
 
       FacetsCollector sfc = new FacetsCollector();
       s.search(new MatchAllDocsQuery(), sfc);
-      Filter fastMatchFilter;
+      Query fastMatchFilter;
       if (random().nextBoolean()) {
         if (random().nextBoolean()) {
-          fastMatchFilter = new QueryWrapperFilter(NumericRangeQuery.newDoubleRange("field", minValue, maxValue, true, true));
+          fastMatchFilter = NumericRangeQuery.newDoubleRange("field", minValue, maxValue, true, true);
         } else {
-          fastMatchFilter = new QueryWrapperFilter(NumericRangeQuery.newDoubleRange("field", minAcceptedValue, maxAcceptedValue, true, true));
+          fastMatchFilter = NumericRangeQuery.newDoubleRange("field", minAcceptedValue, maxAcceptedValue, true, true);
         }
       } else {
         fastMatchFilter = null;
@@ -795,7 +799,7 @@ public class TestRangeFacetCounts extend
         if (random().nextBoolean()) {
           ddq.add("field", NumericRangeQuery.newDoubleRange("field", range.min, range.max, range.minInclusive, range.maxInclusive));
         } else {
-          ddq.add("field", range.getFilter(fastMatchFilter, vs));
+          ddq.add("field", range.getQuery(fastMatchFilter, vs));
         }
 
         assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
@@ -843,40 +847,80 @@ public class TestRangeFacetCounts extend
     IOUtils.close(r, d);
   }
 
-  private static class UsedFilter extends Filter {
-    
+  private static class UsedQuery extends Query {
+
     private final AtomicBoolean used;
-    private final Filter in;
-    
-    UsedFilter(Filter in, AtomicBoolean used) {
+    private final Query in;
+
+    UsedQuery(Query in, AtomicBoolean used) {
       this.in = in;
       this.used = used;
     }
 
     @Override
-    public DocIdSet getDocIdSet(LeafReaderContext context, Bits acceptDocs)
-        throws IOException {
-      used.set(true);
-      return in.getDocIdSet(context, acceptDocs);
-    }
-
-    @Override
-    public String toString(String field) {
-      return in.toString(field);
-    }
-    
-    @Override
     public boolean equals(Object obj) {
       if (super.equals(obj) == false) {
         return false;
       }
-      return in.equals(((UsedFilter) obj).in);
+      UsedQuery that = (UsedQuery) obj;
+      return in.equals(that.in);
     }
 
     @Override
     public int hashCode() {
       return 31 * super.hashCode() + in.hashCode();
     }
+
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+      final Query inRewritten = in.rewrite(reader);
+      if (in != inRewritten) {
+        Query rewritten = new UsedQuery(inRewritten, used);
+        rewritten.setBoost(getBoost());
+        return rewritten;
+      }
+      return super.rewrite(reader);
+    }
+
+    @Override
+    public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+      final Weight in = this.in.createWeight(searcher, needsScores);
+      return new Weight(in.getQuery()) {
+
+        @Override
+        public void extractTerms(Set<Term> terms) {
+          in.extractTerms(terms);
+        }
+
+        @Override
+        public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+          return in.explain(context, doc);
+        }
+
+        @Override
+        public float getValueForNormalization() throws IOException {
+          return in.getValueForNormalization();
+        }
+
+        @Override
+        public void normalize(float norm, float topLevelBoost) {
+          in.normalize(norm, topLevelBoost);
+        }
+
+        @Override
+        public Scorer scorer(LeafReaderContext context) throws IOException {
+          used.set(true);
+          return in.scorer(context);
+        }
+        
+      };
+    }
+
+    @Override
+    public String toString(String field) {
+      return "UsedQuery(" + in + ")";
+    }
+
   }
 
   public void testCustomDoublesValueSource() throws Exception {
@@ -935,12 +979,12 @@ public class TestRangeFacetCounts extend
         new DoubleRange("< 20", 0.0, true, 20.0, false),
         new DoubleRange("< 50", 0.0, true, 50.0, false)};
 
-    final Filter fastMatchFilter;
+    final Query fastMatchFilter;
     final AtomicBoolean filterWasUsed = new AtomicBoolean();
     if (random().nextBoolean()) {
       // Sort of silly:
-      final Filter in = new QueryWrapperFilter(new MatchAllDocsQuery());
-      fastMatchFilter = new UsedFilter(in, filterWasUsed);
+      final Query in = new MatchAllDocsQuery();
+      fastMatchFilter = new UsedQuery(in, filterWasUsed);
     } else {
       fastMatchFilter = null;
     }
@@ -955,7 +999,7 @@ public class TestRangeFacetCounts extend
     assertTrue(fastMatchFilter == null || filterWasUsed.get());
 
     DrillDownQuery ddq = new DrillDownQuery(config);
-    ddq.add("field", ranges[1].getFilter(fastMatchFilter, vs));
+    ddq.add("field", ranges[1].getQuery(fastMatchFilter, vs));
 
     // Test simple drill-down:
     assertEquals(1, s.search(ddq, 10).totalHits);