You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/12/01 13:21:54 UTC

svn commit: r1546809 - in /lucene/dev/branches/lucene5339/lucene/facet: ./ src/java/org/apache/lucene/facet/ src/java/org/apache/lucene/facet/taxonomy/ src/test/org/apache/lucene/facet/

Author: mikemccand
Date: Sun Dec  1 12:21:53 2013
New Revision: 1546809

URL: http://svn.apache.org/r1546809
Log:
LUCENE-5345: split RangeFacetCounts into Long/DoubleRangeFacetCounts

Added:
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRangeFacetCounts.java   (with props)
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRangeFacetCounts.java
      - copied, changed from r1546008, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/RangeFacetCounts.java
Removed:
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/FloatRange.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/RangeFacetCounts.java
Modified:
    lucene/dev/branches/lucene5339/lucene/facet/build.xml
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRange.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRange.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/Range.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/TaxonomyFacetSumValueSource.java
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/package.html
    lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java
    lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/TestRangeFacetCounts.java

Modified: lucene/dev/branches/lucene5339/lucene/facet/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/build.xml?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/build.xml (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/build.xml Sun Dec  1 12:21:53 2013
@@ -32,9 +32,11 @@
   <path id="classpath">
     <path refid="base.classpath"/>
     <pathelement path="${queries.jar}"/>
+    <pathelement path="${expressions.jar}"/>
+    <fileset dir="../expressions/lib"/>
   </path>
 
-  <target name="compile-core" depends="jar-queries,common.compile-core" />
+  <target name="compile-core" depends="jar-queries,jar-expressions,common.compile-core" />
 
   <target name="javadocs" depends="javadocs-queries,compile-core">
     <invoke-module-javadoc>

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRange.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRange.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRange.java Sun Dec  1 12:21:53 2013
@@ -19,8 +19,7 @@ package org.apache.lucene.facet;
 
 import org.apache.lucene.document.DoubleDocValuesField; // javadocs
 
-/** Represents a range over double values indexed as {@link
- *  DoubleDocValuesField}.  */
+/** Represents a range over double values. */
 public final class DoubleRange extends Range {
   private final double minIncl;
   private final double maxIncl;
@@ -62,10 +61,8 @@ public final class DoubleRange extends R
     this.maxIncl = max;
   }
 
-  @Override
-  public boolean accept(long value) {
-    double doubleValue = Double.longBitsToDouble(value);
-    return doubleValue >= minIncl && doubleValue <= maxIncl;
+  public boolean accept(double value) {
+    return value >= minIncl && value <= maxIncl;
   }
 }
 

Added: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRangeFacetCounts.java?rev=1546809&view=auto
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRangeFacetCounts.java (added)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DoubleRangeFacetCounts.java Sun Dec  1 12:21:53 2013
@@ -0,0 +1,130 @@
+package org.apache.lucene.facet;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.lucene.document.DoubleDocValuesField; // javadocs
+import org.apache.lucene.document.FloatDocValuesField; // javadocs
+import org.apache.lucene.facet.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+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
+
+/** {@link Facets} implementation that computes counts for
+ *  dynamic double ranges from a provided {@link
+ *  ValueSource}, using {@link FunctionValues#doubleVal}.  Use
+ *  this for dimensions that change in real-time (e.g. a
+ *  relative time based dimension like "Past day", "Past 2
+ *  days", etc.) or that change for each user (e.g. a
+ *  distance dimension like "< 1 km", "< 2 km", etc.).
+ *
+ *  <p> If you had indexed your field using {@link
+ *  FloatDocValuesField} then pass {@link FloatFieldSource}
+ *  as the {@link ValueSource}; if you used {@link
+ *  DoubleDocValuesField} then pass {@link
+ *  DoubleFieldSource} (this is the default used when you
+ *  pass just a the field name).
+ *
+ *  @lucene.experimental */
+public class DoubleRangeFacetCounts extends Facets {
+  private final DoubleRange[] ranges;
+  private final int[] counts;
+  private final String field;
+  private int totCount;
+
+  /** Create {@code RangeFacetCounts}, using {@link
+   *  DoubleFieldSource} from the specified field. */
+  public DoubleRangeFacetCounts(String field, FacetsCollector hits, DoubleRange... ranges) throws IOException {
+    this(field, new DoubleFieldSource(field), hits, ranges);
+  }
+
+  /** Create {@code RangeFacetCounts}, using the provided
+   *  {@link ValueSource}. */
+  public DoubleRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, DoubleRange... ranges) throws IOException {
+    this.ranges = ranges;
+    this.field = field;
+    counts = new int[ranges.length];
+    count(valueSource, hits.getMatchingDocs());
+  }
+
+  private void count(ValueSource valueSource, List<MatchingDocs> matchingDocs) throws IOException {
+
+    // TODO: test if this is faster (in the past it was
+    // faster to do MatchingDocs on the inside) ... see
+    // patches on LUCENE-4965):
+    for (MatchingDocs hits : matchingDocs) {
+      FunctionValues fv = valueSource.getValues(Collections.emptyMap(), hits.context);
+      final int length = hits.bits.length();
+      int doc = 0;
+      totCount += hits.totalHits;
+      while (doc < length && (doc = hits.bits.nextSetBit(doc)) != -1) {
+        // Skip missing docs:
+        if (fv.exists(doc)) {
+          
+          double v = fv.doubleVal(doc);
+
+          // TODO: if all ranges are non-overlapping, we
+          // should instead do a bin-search up front
+          // (really, a specialized case of the interval
+          // tree)
+          // TODO: use interval tree instead of linear search:
+          for (int j = 0; j < ranges.length; j++) {
+            if (ranges[j].accept(v)) {
+              counts[j]++;
+            }
+          }
+        }
+
+        doc++;
+      }
+    }
+  }
+
+  // nocommit all args are ... unused ... this doesn't "fit"
+  // very well:
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) {
+    if (dim.equals(field) == false) {
+      throw new IllegalArgumentException("invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    LabelAndValue[] labelValues = new LabelAndValue[counts.length];
+    for(int i=0;i<counts.length;i++) {
+      // nocommit can we add the range into this?
+      labelValues[i] = new LabelAndValue(ranges[i].label, counts[i]);
+    }
+
+    return new FacetResult(totCount, labelValues, labelValues.length);
+  }
+
+  @Override
+  public Number getSpecificValue(String dim, String... path) throws IOException {
+    // nocommit we could impl this?
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<FacetResult> getAllDims(int topN) throws IOException {
+    return Collections.singletonList(getTopChildren(topN, null));
+  }
+}

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/DrillDownQuery.java Sun Dec  1 12:21:53 2013
@@ -157,7 +157,7 @@ public final class DrillDownQuery extend
    *  when you have a separate way to drill-down on the
    *  dimension than the indexed facet ordinals (for
    *  example, use a {@link NumericRangeQuery} to drill down
-   *  after{@link RangeFacetCounts}. */
+   *  after {@link LongRangeFacetCounts} or {@link DoubleRangeFacetCounts}. */
   public void add(String dim, Query subQuery) {
 
     // TODO: we should use FilteredQuery?

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRange.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRange.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRange.java Sun Dec  1 12:21:53 2013
@@ -19,8 +19,7 @@ package org.apache.lucene.facet;
 
 import org.apache.lucene.document.NumericDocValuesField; // javadocs
 
-/** Represents a range over long values indexed as {@link
- *  NumericDocValuesField}.  */
+/** Represents a range over long values. */
 public final class LongRange extends Range {
   private final long minIncl;
   private final long maxIncl;
@@ -53,7 +52,6 @@ public final class LongRange extends Ran
     this.maxIncl = max;
   }
 
-  @Override
   public boolean accept(long value) {
     return value >= minIncl && value <= maxIncl;
   }

Copied: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRangeFacetCounts.java (from r1546008, lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/RangeFacetCounts.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRangeFacetCounts.java?p2=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRangeFacetCounts.java&p1=lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/RangeFacetCounts.java&r1=1546008&r2=1546809&rev=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/RangeFacetCounts.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/LongRangeFacetCounts.java Sun Dec  1 12:21:53 2013
@@ -28,28 +28,29 @@ import org.apache.lucene.queries.functio
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 
 /** {@link Facets} implementation that computes counts for
- *  dynamic ranges from a provided {@link ValueSource}.  Use
+ *  dynamic long ranges from a provided {@link ValueSource},
+ *  using {@link FunctionValues#longVal}.  Use
  *  this for dimensions that change in real-time (e.g. a
  *  relative time based dimension like "Past day", "Past 2
  *  days", etc.) or that change for each user (e.g. a
  *  distance dimension like "< 1 km", "< 2 km", etc.).
  *
  *  @lucene.experimental */
-public class RangeFacetCounts extends Facets {
-  private final Range[] ranges;
+public class LongRangeFacetCounts extends Facets {
+  private final LongRange[] ranges;
   private final int[] counts;
   private final String field;
   private int totCount;
 
   /** Create {@code RangeFacetCounts}, using {@link
    *  LongFieldSource} from the specified field. */
-  public RangeFacetCounts(String field, FacetsCollector hits, Range... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, FacetsCollector hits, LongRange... ranges) throws IOException {
     this(field, new LongFieldSource(field), hits, ranges);
   }
 
   /** Create {@code RangeFacetCounts}, using the provided
    *  {@link ValueSource}. */
-  public RangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, Range... ranges) throws IOException {
+  public LongRangeFacetCounts(String field, ValueSource valueSource, FacetsCollector hits, LongRange... ranges) throws IOException {
     this.ranges = ranges;
     this.field = field;
     counts = new int[ranges.length];

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/Range.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/Range.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/Range.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/Range.java Sun Dec  1 12:21:53 2013
@@ -17,8 +17,7 @@ package org.apache.lucene.facet;
  * limitations under the License.
  */
 
-/** Represents a single labeled range, one facet label in
- *  the facets computed by {@link RangeFacetCounts}.
+/** Base class for a single labeled range.
  *
  *  @lucene.experimental */
 public abstract class Range {
@@ -27,6 +26,4 @@ public abstract class Range {
   protected Range(String label) {
     this.label = label;
   }
-
-  public abstract boolean accept(long value);
 }

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/TaxonomyFacetSumValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/TaxonomyFacetSumValueSource.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/TaxonomyFacetSumValueSource.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/TaxonomyFacetSumValueSource.java Sun Dec  1 12:21:53 2013
@@ -33,7 +33,7 @@ import org.apache.lucene.util.FixedBitSe
 import org.apache.lucene.util.IntsRef;
 
 /** Aggregates sum of values from {@link
- *  ValueSource#doubleValue}, for each facet label.
+ *  FunctionValues#doubleVal}, for each facet label.
  *
  *  @lucene.experimental */
 public class TaxonomyFacetSumValueSource extends FloatTaxonomyFacets {

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/package.html?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/package.html (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/package.html Sun Dec  1 12:21:53 2013
@@ -20,17 +20,46 @@
   </head>
   <body>
     <h1>faceted search</h1>
-  </body>
-  <p>
-    This module provides multiple methods for computing facet counts and
-    value aggregations.
-  </p>
-  <p>
-    The taxonomy-based methods ({@link FastTaxonomyFacetCounts},
-  {@link TaxonomyFacetSumIntAssociations
-  The Facets.search utility methods are useful for doing an "ordinary"
-  search (sorting by score, or by a specified Sort) but also
-  collecting into a FacetsCollector for subsequent faceting.
+    <p>
+      This module provides multiple methods for computing facet counts and
+      value aggregations:
+      <ul>
+	<li> Taxonomy-based methods rely on a separate taxonomy index to
+          map hierarchical facet paths to global int ordinals for fast
+          counting at search time; these methods can compute counts
+          (({@link org.apache.lucene.facet.FastTaxonomyFacetCounts}, {@link
+          org.apache.lucene.facet.TaxonomyFacetCounts}) aggregate long or double values {@link
+          org.apache.lucene.facet.TaxonomyFacetSumIntAssociations}, {@link
+          org.apache.lucene.facet.TaxonomyFacetSumFloatAssociations}, {@link
+          org.apache.lucene.facet.TaxonomyFacetSumValueSource}.  Add {@link org.apache.lucene.facet.FacetField} or
+          {@link org.apache.lucene.facet.AssociationFacetField} to your documents at index time
+          to use taxonomy-based methods.
+
+	<li> Sorted-set doc values method does not require a separate
+          taxonomy index, and computes counts based on sorted set doc
+          values fields ({@link org.apache.lucene.facet.SortedSetDocValuesFacetCounts}).  Add
+          {@link org.apache.lucene.facet.SortedSetDocValuesFacetField} to your documents at
+          index time to use sorted set facet counts.
 
-  <p>
+	<li> Range faceting {@link org.apache.lucene.facet.LongRangeFacetCounts}, {@link
+          org.apache.lucene.facet.DoubleRangeFacetCounts} compute counts for a dynamic numeric
+          range from a provided {@link org.apache.lucene.facet.ValueSource} (previously indexed
+          numeric field, or a dynamic expression such as distance).
+      </ul>
+    </p>
+    <p>
+      At search time you first run your search, but pass a {@link
+      org.apache.lucene.facet.FacetsCollector} to gather all hits (and optionally, scores for each
+      hit).  Then, instantiate whichever facet methods you'd like to use
+      to compute aggregates.  Finally, all methods implement a common
+      {@link org.apache.lucene.facet.Facets} base API that you use to obtain specific facet
+      counts.
+    </p>
+    <p>
+      The various {@link org.apache.lucene.facet.FacetsCollector#search} utility methods are
+      useful for doing an "ordinary" search (sorting by score, or by a
+      specified Sort) but also collecting into a {@link org.apache.lucene.facet.FacetsCollector} for
+      subsequent faceting.
+    </p>
+  </body>
 </html>

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java Sun Dec  1 12:21:53 2013
@@ -26,7 +26,7 @@ import static org.apache.lucene.util.Byt
  * Holds a sequence of string components, specifying the hierarchical name of a
  * category.
  * 
- * @lucene.experimental
+ * @lucene.internal
  */
 public class FacetLabel implements Comparable<FacetLabel> {
 
@@ -37,8 +37,7 @@ public class FacetLabel implements Compa
    * be on the safe side.
    */
   /**
-   * The maximum number of characters a {@link FacetLabel} can have. That is
-   * {@link FacetLabel#toString(char)} length must not exceed that limit.
+   * The maximum number of characters a {@link FacetLabel} can have.
    */
   public final static int MAX_CATEGORY_PATH_LENGTH = (BYTE_BLOCK_SIZE - 2) / 4;
 
@@ -184,10 +183,7 @@ public class FacetLabel implements Compa
   }
 
   /**
-   * Returns a string representation of the path, separating components with
-   * '/'.
-   * 
-   * @see #toString(char)
+   * Returns a string representation of the path.
    */
   @Override
   public String toString() {

Modified: lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/TestRangeFacetCounts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/TestRangeFacetCounts.java?rev=1546809&r1=1546808&r2=1546809&view=diff
==============================================================================
--- lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/TestRangeFacetCounts.java (original)
+++ lucene/dev/branches/lucene5339/lucene/facet/src/test/org/apache/lucene/facet/TestRangeFacetCounts.java Sun Dec  1 12:21:53 2013
@@ -29,6 +29,9 @@ import org.apache.lucene.document.FloatD
 import org.apache.lucene.document.FloatField;
 import org.apache.lucene.document.LongField;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.expressions.Expression;
+import org.apache.lucene.expressions.SimpleBindings;
+import org.apache.lucene.expressions.js.JavascriptCompiler;
 import org.apache.lucene.facet.DrillSideways.DrillSidewaysResult;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@@ -36,9 +39,11 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.queries.function.valuesource.FloatFieldSource;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.SortField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util._TestUtil;
@@ -66,7 +71,7 @@ public class TestRangeFacetCounts extend
     IndexSearcher s = newSearcher(r);
     s.search(new MatchAllDocsQuery(), fc);
 
-    RangeFacetCounts facets = new RangeFacetCounts("field", fc,
+    Facets facets = new LongRangeFacetCounts("field", fc,
         new LongRange("less than 10", 0L, true, 10L, false),
         new LongRange("less than or equal to 10", 0L, true, 10L, true),
         new LongRange("over 90", 90L, false, 100L, false),
@@ -134,7 +139,7 @@ public class TestRangeFacetCounts extend
 
           Map<String,Facets> byDim = new HashMap<String,Facets>();
           byDim.put("field",
-                    new RangeFacetCounts("field", fieldFC,
+                    new LongRangeFacetCounts("field", fieldFC,
                           new LongRange("less than 10", 0L, true, 10L, false),
                           new LongRange("less than or equal to 10", 0L, true, 10L, true),
                           new LongRange("over 90", 90L, false, 100L, false),
@@ -198,7 +203,7 @@ public class TestRangeFacetCounts extend
 
     IndexSearcher s = newSearcher(r);
     s.search(new MatchAllDocsQuery(), fc);
-    Facets facets = new RangeFacetCounts("field", fc,
+    Facets facets = new DoubleRangeFacetCounts("field", fc,
         new DoubleRange("less than 10", 0.0, true, 10.0, false),
         new DoubleRange("less than or equal to 10", 0.0, true, 10.0, true),
         new DoubleRange("over 90", 90.0, false, 100.0, false),
@@ -229,12 +234,12 @@ public class TestRangeFacetCounts extend
     IndexSearcher s = newSearcher(r);
     s.search(new MatchAllDocsQuery(), fc);
 
-    Facets facets = new RangeFacetCounts("field", fc,
-        new FloatRange("less than 10", 0.0f, true, 10.0f, false),
-        new FloatRange("less than or equal to 10", 0.0f, true, 10.0f, true),
-        new FloatRange("over 90", 90.0f, false, 100.0f, false),
-        new FloatRange("90 or above", 90.0f, true, 100.0f, false),
-        new FloatRange("over 1000", 1000.0f, false, Float.POSITIVE_INFINITY, false));
+    Facets facets = new DoubleRangeFacetCounts("field", new FloatFieldSource("field"), fc,
+        new DoubleRange("less than 10", 0.0f, true, 10.0f, false),
+        new DoubleRange("less than or equal to 10", 0.0f, true, 10.0f, true),
+        new DoubleRange("over 90", 90.0f, false, 100.0f, false),
+        new DoubleRange("90 or above", 90.0f, true, 100.0f, false),
+        new DoubleRange("over 1000", 1000.0f, false, Double.POSITIVE_INFINITY, false));
     
     assertEquals("value=100 childCount=5\n  less than 10 (10)\n  less than or equal to 10 (11)\n  over 90 (9)\n  90 or above (10)\n  over 1000 (0)\n",
                  facets.getTopChildren(10, "field").toString());
@@ -303,7 +308,7 @@ public class TestRangeFacetCounts extend
 
       FacetsCollector sfc = new FacetsCollector();
       s.search(new MatchAllDocsQuery(), sfc);
-      Facets facets = new RangeFacetCounts("field", sfc, ranges);
+      Facets facets = new LongRangeFacetCounts("field", sfc, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
       for(int rangeID=0;rangeID<numRange;rangeID++) {
@@ -351,19 +356,19 @@ public class TestRangeFacetCounts extend
         System.out.println("TEST: iter=" + iter);
       }
       int numRange = _TestUtil.nextInt(random(), 1, 5);
-      FloatRange[] ranges = new FloatRange[numRange];
+      DoubleRange[] ranges = new DoubleRange[numRange];
       int[] expectedCounts = new int[numRange];
       for(int rangeID=0;rangeID<numRange;rangeID++) {
-        float min = random().nextFloat();
-        float max = random().nextFloat();
+        double min = random().nextDouble();
+        double max = random().nextDouble();
         if (min > max) {
-          float x = min;
+          double x = min;
           min = max;
           max = x;
         }
         boolean minIncl = random().nextBoolean();
         boolean maxIncl = random().nextBoolean();
-        ranges[rangeID] = new FloatRange("r" + rangeID, min, minIncl, max, maxIncl);
+        ranges[rangeID] = new DoubleRange("r" + rangeID, min, minIncl, max, maxIncl);
 
         // Do "slow but hopefully correct" computation of
         // expected count:
@@ -387,7 +392,7 @@ public class TestRangeFacetCounts extend
 
       FacetsCollector sfc = new FacetsCollector();
       s.search(new MatchAllDocsQuery(), sfc);
-      Facets facets = new RangeFacetCounts("field", sfc, ranges);
+      Facets facets = new DoubleRangeFacetCounts("field", new FloatFieldSource("field"), sfc, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
       for(int rangeID=0;rangeID<numRange;rangeID++) {
@@ -398,11 +403,11 @@ public class TestRangeFacetCounts extend
         assertEquals("r" + rangeID, subNode.label);
         assertEquals(expectedCounts[rangeID], subNode.value.intValue());
 
-        FloatRange range = ranges[rangeID];
+        DoubleRange range = ranges[rangeID];
 
         // Test drill-down:
         DrillDownQuery ddq = new DrillDownQuery(config);
-        ddq.add("field", NumericRangeQuery.newFloatRange("field", range.min, range.max, range.minInclusive, range.maxInclusive));
+        ddq.add("field", NumericRangeQuery.newFloatRange("field", (float) range.min, (float) range.max, range.minInclusive, range.maxInclusive));
         assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
       }
     }
@@ -471,7 +476,7 @@ public class TestRangeFacetCounts extend
 
       FacetsCollector sfc = new FacetsCollector();
       s.search(new MatchAllDocsQuery(), sfc);
-      Facets facets = new RangeFacetCounts("field", sfc, ranges);
+      Facets facets = new DoubleRangeFacetCounts("field", sfc, ranges);
       FacetResult result = facets.getTopChildren(10, "field");
       assertEquals(numRange, result.labelValues.length);
       for(int rangeID=0;rangeID<numRange;rangeID++) {
@@ -514,11 +519,11 @@ public class TestRangeFacetCounts extend
 
     IndexReader r = w.getReader();
 
-    FacetsCollector sfc = new FacetsCollector();
+    FacetsCollector fc = new FacetsCollector();
 
     IndexSearcher s = newSearcher(r);
-    s.search(new MatchAllDocsQuery(), sfc);
-    Facets facets = new RangeFacetCounts("field", sfc,
+    s.search(new MatchAllDocsQuery(), fc);
+    Facets facets = new LongRangeFacetCounts("field", fc,
         new LongRange("less than 10", 0L, true, 10L, false),
         new LongRange("less than or equal to 10", 0L, true, 10L, true),
         new LongRange("over 90", 90L, false, 100L, false),
@@ -530,4 +535,46 @@ public class TestRangeFacetCounts extend
 
     IOUtils.close(w, r, d);
   }
+
+  public void testDistanceRangeFaceting() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
+    
+    Document doc = new Document();
+    doc.add(new DoubleField("latitude", 40.759011, Field.Store.NO));
+    doc.add(new DoubleField("longitude", -73.9844722, Field.Store.NO));
+    writer.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new DoubleField("latitude", 40.718266, Field.Store.NO));
+    doc.add(new DoubleField("longitude", -74.007819, Field.Store.NO));
+    writer.addDocument(doc);
+    
+    doc = new Document();
+    doc.add(new DoubleField("latitude", 40.7051157, Field.Store.NO));
+    doc.add(new DoubleField("longitude", -74.0088305, Field.Store.NO));
+    writer.addDocument(doc);
+    
+    Expression distance = JavascriptCompiler.compile("haversin(40.7143528,-74.0059731,latitude,longitude)");
+    SimpleBindings bindings = new SimpleBindings();
+    bindings.add(new SortField("latitude", SortField.Type.DOUBLE));
+    bindings.add(new SortField("longitude", SortField.Type.DOUBLE));
+
+    FacetsCollector fc = new FacetsCollector();
+
+    IndexReader r = writer.getReader();
+    IndexSearcher s = newSearcher(r);
+    s.search(new MatchAllDocsQuery(), fc);
+
+    Facets facets = new DoubleRangeFacetCounts("field", distance.getValueSource(bindings), fc,
+        new DoubleRange("< 1 km", 0.0, true, 1.0, false),
+        new DoubleRange("< 2 km", 0.0, true, 2.0, false),
+        new DoubleRange("< 5 km", 0.0, true, 5.0, false),
+        new DoubleRange("< 10 km", 0.0, true, 10.0, false),
+        new DoubleRange("< 20 km", 0.0, true, 20.0, false),
+        new DoubleRange("< 50 km", 0.0, true, 50.0, false));
+
+    assertEquals("value=3 childCount=6\n  < 1 km (1)\n  < 2 km (2)\n  < 5 km (2)\n  < 10 km (3)\n  < 20 km (3)\n  < 50 km (3)\n", facets.getTopChildren(10, "field").toString());
+    IOUtils.close(r, writer, dir);
+  }
 }