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/04/30 13:17:33 UTC

svn commit: r1477560 - in /lucene/dev/branches/branch_4x/lucene: ./ demo/src/java/org/apache/lucene/demo/facet/ demo/src/test/org/apache/lucene/demo/facet/ facet/src/java/org/apache/lucene/facet/range/ facet/src/java/org/apache/lucene/facet/search/ fac...

Author: mikemccand
Date: Tue Apr 30 11:17:32 2013
New Revision: 1477560

URL: http://svn.apache.org/r1477560
Log:
LUCENE-4965: add dynamic numeric range faceting

Added:
    lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java   (with props)
    lucene/dev/branches/branch_4x/lucene/demo/src/test/org/apache/lucene/demo/facet/TestRangeFacetsExample.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/FloatRange.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetResultNode.java   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/package.html   (with props)
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/range/
    lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java   (with props)
Modified:
    lucene/dev/branches/branch_4x/lucene/CHANGES.txt
    lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/SimpleFacetsExample.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
    lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java

Modified: lucene/dev/branches/branch_4x/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/CHANGES.txt?rev=1477560&r1=1477559&r2=1477560&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/lucene/CHANGES.txt Tue Apr 30 11:17:32 2013
@@ -69,6 +69,9 @@ New Features
 
 * LUCENE-4966: Add CachingWrapperFilter.sizeInBytes() (Mike McCandless)
 
+* LUCENE-4965: Add dynamic (no taxonomy index used) numeric range
+  faceting to Lucene's facet module (Mike McCandless, Shai Erera)
+
 ======================= Lucene 4.3.0 =======================
 
 Changes in backwards compatibility policy

Added: lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java (added)
+++ lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/RangeFacetsExample.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,141 @@
+package org.apache.lucene.demo.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.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.LongDocValuesField;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.RangeAccumulator;
+import org.apache.lucene.facet.range.RangeFacetRequest;
+import org.apache.lucene.facet.search.DrillDownQuery;
+import org.apache.lucene.facet.search.FacetResult;
+import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+
+
+/** Shows simple usage of dynamic range faceting. */
+public class RangeFacetsExample implements Closeable {
+
+  private final Directory indexDir = new RAMDirectory();
+  private IndexSearcher searcher;
+  private final long nowSec = System.currentTimeMillis();
+
+  /** Empty constructor */
+  public RangeFacetsExample() {}
+  
+  /** Build the example index. */
+  public void index() throws IOException {
+    IndexWriter indexWriter = new IndexWriter(indexDir, new IndexWriterConfig(FacetExamples.EXAMPLES_VER, 
+        new WhitespaceAnalyzer(FacetExamples.EXAMPLES_VER)));
+
+    // Add documents with a fake timestamp, 1000 sec before
+    // "now", 2000 sec before "now", ...:
+    for(int i=0;i<100;i++) {
+      Document doc = new Document();
+      long then = nowSec - i * 1000;
+      // Add as doc values field, so we can compute range facets:
+      doc.add(new LongDocValuesField("timestamp", then));
+      // Add as numeric field so we can drill-down:
+      doc.add(new LongField("timestamp", then, Field.Store.NO));
+      indexWriter.addDocument(doc);
+    }
+
+    // Open near-real-time searcher
+    searcher = new IndexSearcher(DirectoryReader.open(indexWriter, true));
+    indexWriter.close();
+  }
+
+  /** User runs a query and counts facets. */
+  public List<FacetResult> search() throws IOException {
+
+    FacetSearchParams fsp = new FacetSearchParams(
+                                new RangeFacetRequest<LongRange>("timestamp",
+                                                                 new LongRange("Past hour", nowSec-3600, true, nowSec, true),
+                                                                 new LongRange("Past six hours", nowSec-6*3600, true, nowSec, true),
+                                                                 new LongRange("Past day", nowSec-24*3600, true, nowSec, true)));
+    // Aggregatses the facet counts
+    FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, searcher.getIndexReader()));
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query, and use MultiCollector to
+    // wrap collecting the "normal" hits and also facets:
+    searcher.search(new MatchAllDocsQuery(), fc);
+
+    // Retrieve results
+    return fc.getFacetResults();
+  }
+  
+  /** User drills down on the specified range. */
+  public TopDocs drillDown(LongRange range) throws IOException {
+
+    // Passing no baseQuery means we drill down on all
+    // documents ("browse only"):
+    DrillDownQuery q = new DrillDownQuery(FacetIndexingParams.DEFAULT);
+
+    // Use FieldCacheRangeFilter; this will use
+    // NumericDocValues:
+    q.add("timestamp", NumericRangeQuery.newLongRange("timestamp", range.min, range.max, range.minInclusive, range.maxInclusive));
+
+    return searcher.search(q, 10);
+  }
+
+  public void close() throws IOException {
+    searcher.getIndexReader().close();
+    indexDir.close();
+  }
+
+  /** Runs the search and drill-down examples and prints the results. */
+  @SuppressWarnings("unchecked")
+  public static void main(String[] args) throws Exception {
+    RangeFacetsExample example = new RangeFacetsExample();
+    example.index();
+
+    System.out.println("Facet counting example:");
+    System.out.println("-----------------------");
+    List<FacetResult> results = example.search();
+    for (FacetResult res : results) {
+      System.out.println(res);
+    }
+
+    System.out.println("\n");
+    System.out.println("Facet drill-down example (timestamp/Past six hours):");
+    System.out.println("---------------------------------------------");
+    TopDocs hits = example.drillDown((LongRange) ((RangeFacetRequest<LongRange>) results.get(0).getFacetRequest()).ranges[1]);
+    System.out.println(hits.totalHits + " totalHits");
+
+    example.close();
+  }
+}

Modified: lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/SimpleFacetsExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/SimpleFacetsExample.java?rev=1477560&r1=1477559&r2=1477560&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/SimpleFacetsExample.java (original)
+++ lucene/dev/branches/branch_4x/lucene/demo/src/java/org/apache/lucene/demo/facet/SimpleFacetsExample.java Tue Apr 30 11:17:32 2013
@@ -119,7 +119,10 @@ public class SimpleFacetsExample {
 
     // Now user drills down on Publish Date/2010:
     FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("Author"), 10));
-    DrillDownQuery q = new DrillDownQuery(fsp.indexingParams, new MatchAllDocsQuery());
+
+    // Passing no baseQuery means we drill down on all
+    // documents ("browse only"):
+    DrillDownQuery q = new DrillDownQuery(fsp.indexingParams);
     q.add(new CategoryPath("Publish Date/2010", '/'));
     FacetsCollector fc = FacetsCollector.create(fsp, searcher.getIndexReader(), taxoReader);
     searcher.search(q, fc);

Added: lucene/dev/branches/branch_4x/lucene/demo/src/test/org/apache/lucene/demo/facet/TestRangeFacetsExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/demo/src/test/org/apache/lucene/demo/facet/TestRangeFacetsExample.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/demo/src/test/org/apache/lucene/demo/facet/TestRangeFacetsExample.java (added)
+++ lucene/dev/branches/branch_4x/lucene/demo/src/test/org/apache/lucene/demo/facet/TestRangeFacetsExample.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,70 @@
+package org.apache.lucene.demo.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.util.List;
+
+import org.apache.lucene.facet.collections.ObjectToIntMap;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.facet.range.RangeFacetRequest;
+import org.apache.lucene.facet.search.FacetResult;
+import org.apache.lucene.facet.search.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.Test;
+
+@SuppressCodecs("Lucene3x")
+public class TestRangeFacetsExample extends LuceneTestCase {
+
+  private static final ObjectToIntMap<CategoryPath> expectedCounts = new ObjectToIntMap<CategoryPath>();
+  static {
+    expectedCounts.put(new CategoryPath("timestamp", "Past hour"), 4);
+    expectedCounts.put(new CategoryPath("timestamp", "Past six hours"), 22);
+    expectedCounts.put(new CategoryPath("timestamp", "Past day"), 87);
+  }
+  
+  private void assertExpectedCounts(FacetResult res, ObjectToIntMap<CategoryPath> expCounts) {
+    FacetResultNode root = res.getFacetResultNode();
+    for (FacetResultNode node : root.subResults) {
+      assertEquals("incorrect count for " + node.label, expCounts.get(node.label), (int) node.value);
+    }
+  }
+  
+  @Test
+  public void testSimple() throws Exception {
+    RangeFacetsExample example = new RangeFacetsExample();
+    example.index();
+    List<FacetResult> facetResults = example.search();
+    assertEquals(1, facetResults.size());
+    assertExpectedCounts(facetResults.get(0), expectedCounts);
+    example.close();
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testDrillDown() throws Exception {
+    RangeFacetsExample example = new RangeFacetsExample();
+    example.index();
+    List<FacetResult> facetResults = example.search();
+    TopDocs hits = example.drillDown((LongRange) ((RangeFacetRequest<LongRange>) facetResults.get(0).getFacetRequest()).ranges[1]);
+    assertEquals(22, hits.totalHits);
+    example.close();
+  }
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/DoubleRange.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,71 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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 org.apache.lucene.document.DoubleDocValuesField; // javadocs
+
+/** Represents a range over double values indexed as {@link
+ *  DoubleDocValuesField}.  */
+public final class DoubleRange extends Range {
+  private final double minIncl;
+  private final double maxIncl;
+
+  public final double min;
+  public final double max;
+  public final boolean minInclusive;
+  public final boolean maxInclusive;
+
+  /** Create a DoubleRange. */
+  public DoubleRange(String label, double min, boolean minInclusive, double max, boolean maxInclusive) {
+    super(label);
+    this.min = min;
+    this.max = max;
+    this.minInclusive = minInclusive;
+    this.maxInclusive = maxInclusive;
+
+    // TODO: if DoubleDocValuesField used
+    // NumericUtils.doubleToSortableLong format (instead of
+    // Double.doubleToRawLongBits) we could do comparisons
+    // in long space 
+
+    if (Double.isNaN(min)) {
+      throw new IllegalArgumentException("min cannot be NaN");
+    }
+    if (!minInclusive) {
+      min = Math.nextUp(min);
+    }
+
+    if (Double.isNaN(max)) {
+      throw new IllegalArgumentException("max cannot be NaN");
+    }
+    if (!maxInclusive) {
+      // Why no Math.nextDown?
+      max = Math.nextAfter(max, Double.NEGATIVE_INFINITY);
+    }
+
+    this.minIncl = min;
+    this.maxIncl = max;
+  }
+
+  @Override
+  public boolean accept(long value) {
+    double doubleValue = Double.longBitsToDouble(value);
+    return doubleValue >= minIncl && doubleValue <= maxIncl;
+  }
+}
+

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/FloatRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/FloatRange.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/FloatRange.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/FloatRange.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,70 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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 org.apache.lucene.document.FloatDocValuesField; // javadocs
+
+/** Represents a range over float values indexed as {@link
+ *  FloatDocValuesField}.  */
+public final class FloatRange extends Range {
+  private final float minIncl;
+  private final float maxIncl;
+
+  public final float min;
+  public final float max;
+  public final boolean minInclusive;
+  public final boolean maxInclusive;
+
+  /** Create a FloatRange. */
+  public FloatRange(String label, float min, boolean minInclusive, float max, boolean maxInclusive) {
+    super(label);
+    this.min = min;
+    this.max = max;
+    this.minInclusive = minInclusive;
+    this.maxInclusive = maxInclusive;
+
+    // TODO: if FloatDocValuesField used
+    // NumericUtils.floatToSortableInt format (instead of
+    // Float.floatToRawIntBits) we could do comparisons
+    // in int space 
+
+    if (Float.isNaN(min)) {
+      throw new IllegalArgumentException("min cannot be NaN");
+    }
+    if (!minInclusive) {
+      min = Math.nextUp(min);
+    }
+
+    if (Float.isNaN(max)) {
+      throw new IllegalArgumentException("max cannot be NaN");
+    }
+    if (!maxInclusive) {
+      // Why no Math.nextDown?
+      max = Math.nextAfter(max, Float.NEGATIVE_INFINITY);
+    }
+
+    this.minIncl = min;
+    this.maxIncl = max;
+  }
+
+  @Override
+  public boolean accept(long value) {
+    float floatValue = Float.intBitsToFloat((int) value);
+    return floatValue >= minIncl && floatValue <= maxIncl;
+  }
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/LongRange.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,57 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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 org.apache.lucene.document.NumericDocValuesField; // javadocs
+
+/** Represents a range over long values indexed as {@link
+ *  NumericDocValuesField}.  */
+public final class LongRange extends Range {
+  private final long minIncl;
+  private final long maxIncl;
+
+  public final long min;
+  public final long max;
+  public final boolean minInclusive;
+  public final boolean maxInclusive;
+
+  /** Create a LongRange. */
+  public LongRange(String label, long min, boolean minInclusive, long max, boolean maxInclusive) {
+    super(label);
+    this.min = min;
+    this.max = max;
+    this.minInclusive = minInclusive;
+    this.maxInclusive = maxInclusive;
+
+    if (!minInclusive && min != Long.MAX_VALUE) {
+      min++;
+    }
+
+    if (!maxInclusive && max != Long.MIN_VALUE) {
+      max--;
+    }
+
+    this.minIncl = min;
+    this.maxIncl = max;
+  }
+
+  @Override
+  public boolean accept(long value) {
+    return value >= minIncl && value <= maxIncl;
+  }
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/Range.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,33 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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.
+ */
+
+/** Represents a single labelled range, one facet label in
+ *  the facets computed by {@link RangeAccumulator}.
+ *
+ *  @lucene.experimental */
+
+public abstract class Range {
+  public final String label;
+
+  protected Range(String label) {
+    this.label = label;
+  }
+
+  public abstract boolean accept(long value);
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,125 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.search.FacetResult;
+import org.apache.lucene.facet.search.FacetResultNode;
+import org.apache.lucene.facet.search.FacetsAccumulator;
+import org.apache.lucene.facet.search.FacetsAggregator;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.NumericDocValues;
+
+/** Uses a {@link NumericDocValues} and accumulates
+ *  counts for provided ranges.  This is dynamic (does not
+ *  use the taxonomy index or anything from the index
+ *  except the NumericDocValuesField). */
+
+public class RangeAccumulator extends FacetsAccumulator {
+
+  static class RangeSet {
+    final Range[] ranges;
+    final String field;
+
+    public RangeSet(Range[] ranges, String field) {
+      this.ranges = ranges;
+      this.field = field;
+    }
+  }
+
+  final List<RangeSet> requests = new ArrayList<RangeSet>();
+
+  public RangeAccumulator(FacetSearchParams fsp, IndexReader reader) {
+    super(fsp, reader, null, null);
+
+    for(FacetRequest fr : fsp.facetRequests) {
+
+      if (!(fr instanceof RangeFacetRequest)) {
+        throw new IllegalArgumentException("only RangeFacetRequest is supported; got " + fsp.facetRequests.get(0).getClass());
+      }
+
+      if (fr.categoryPath.length != 1) {
+        throw new IllegalArgumentException("only flat (dimension only) CategoryPath is allowed");
+      }
+
+      RangeFacetRequest<?> rfr = (RangeFacetRequest) fr;
+
+      requests.add(new RangeSet(rfr.ranges, rfr.categoryPath.components[0]));
+    }
+  }
+
+  @Override
+  public FacetsAggregator getAggregator() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<FacetResult> accumulate(List<MatchingDocs> matchingDocs) throws IOException {
+
+    // TODO: test if this is faster (in the past it was
+    // faster to do MachingDocs on the inside) ... see
+    // patches on LUCENE-4965):
+    List<FacetResult> results = new ArrayList<FacetResult>();
+    for(int i=0;i<requests.size();i++) {
+      RangeSet ranges = requests.get(i);
+
+      int[] counts = new int[ranges.ranges.length];
+      for(MatchingDocs hits : matchingDocs) {
+        NumericDocValues ndv = hits.context.reader().getNumericDocValues(ranges.field);
+        final int length = hits.bits.length();
+        int doc = 0;
+        while (doc < length && (doc = hits.bits.nextSetBit(doc)) != -1) {
+          long v = ndv.get(doc);
+          // TODO: use interval tree instead of linear search:
+          for(int j=0;j<ranges.ranges.length;j++) {
+            if (ranges.ranges[j].accept(v)) {
+              counts[j]++;
+            }
+          }
+
+          doc++;
+        }
+      }
+
+      List<FacetResultNode> nodes = new ArrayList<FacetResultNode>(ranges.ranges.length);
+      for(int j=0;j<ranges.ranges.length;j++) {
+        nodes.add(new RangeFacetResultNode(ranges.field, ranges.ranges[j], counts[j]));
+      }
+
+      FacetResultNode rootNode = new FacetResultNode(-1, 0);
+      rootNode.label = new CategoryPath(ranges.field);
+      rootNode.subResults = nodes;
+
+      results.add(new FacetResult(searchParams.facetRequests.get(i), rootNode, nodes.size()));
+    }
+
+    return results;
+  }
+
+  @Override
+  protected boolean requiresDocScores() {
+    return false;
+  }
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,64 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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.util.List;
+
+import org.apache.lucene.facet.search.Aggregator;
+import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+
+/**
+ * Facet request for dynamic ranges based on a
+ * NumericDocValues field.  This does not use the taxonomy
+ * index nor any indexed facet values.
+ * 
+ * @lucene.experimental
+ */
+public class RangeFacetRequest<T extends Range> extends FacetRequest {
+
+  public final Range[] ranges;
+
+  public RangeFacetRequest(String field, T...ranges) {
+    super(new CategoryPath(field), 1);
+    this.ranges = ranges;
+  }
+
+  @SuppressWarnings("unchecked")
+  public RangeFacetRequest(String field, List<T> ranges) {
+    this(field, (T[]) ranges.toArray(new Range[ranges.size()]));
+  }
+
+  @Override
+  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public double getValueOf(FacetArrays arrays, int ordinal) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public FacetArraysSource getFacetArraysSource() {
+    throw new UnsupportedOperationException();
+  }
+  
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetResultNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetResultNode.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetResultNode.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetResultNode.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,33 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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 org.apache.lucene.facet.search.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+
+/** Holds the facet results for a {@link
+ *  RangeFacetRequest}. */
+public class RangeFacetResultNode extends FacetResultNode {
+  public final Range range;
+
+  RangeFacetResultNode(String field, Range range, int count) {
+    super(-1, count);
+    this.range = range;
+    this.label = new CategoryPath(field, range.label);
+  }
+}

Added: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/package.html?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/package.html (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/range/package.html Tue Apr 30 11:17:32 2013
@@ -0,0 +1,24 @@
+<!--
+ 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.
+-->
+<html>
+<head>
+<title>Facets numeric range code</title>
+</head>
+<body>
+Code to compute facets for numeric ranges.
+</body>
+</html>

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java?rev=1477560&r1=1477559&r2=1477560&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java Tue Apr 30 11:17:32 2013
@@ -58,7 +58,7 @@ public class FacetsAccumulator {
    * constructor.
    */
   public FacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
-    this(searchParams, indexReader, taxonomyReader, null);
+    this(searchParams, indexReader, taxonomyReader, new FacetArrays(taxonomyReader.getSize()));
   }
 
   /**
@@ -97,9 +97,6 @@ public class FacetsAccumulator {
    */
   public FacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, TaxonomyReader taxonomyReader, 
       FacetArrays facetArrays) {
-    if (facetArrays == null) {
-      facetArrays = new FacetArrays(taxonomyReader.getSize());
-    }
     this.facetArrays = facetArrays;
     this.indexReader = indexReader;
     this.taxonomyReader = taxonomyReader;
@@ -199,4 +196,7 @@ public class FacetsAccumulator {
     return res;
   }
 
+  protected boolean requiresDocScores() {
+    return getAggregator().requiresDocScores();
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java?rev=1477560&r1=1477559&r2=1477560&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java (original)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java Tue Apr 30 11:17:32 2013
@@ -175,7 +175,7 @@ public abstract class FacetsCollector ex
    * given {@link FacetsAccumulator}.
    */
   public static FacetsCollector create(FacetsAccumulator accumulator) {
-    if (accumulator.getAggregator().requiresDocScores()) {
+    if (accumulator.requiresDocScores()) {
       return new DocsAndScoresCollector(accumulator);
     } else {
       return new DocsOnlyCollector(accumulator);

Added: lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java?rev=1477560&view=auto
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java (added)
+++ lucene/dev/branches/branch_4x/lucene/facet/src/test/org/apache/lucene/facet/range/TestRangeAccumulator.java Tue Apr 30 11:17:32 2013
@@ -0,0 +1,418 @@
+package org.apache.lucene.facet.range;
+
+/*
+ * 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.util.List;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatDocValuesField;
+import org.apache.lucene.document.FloatField;
+import org.apache.lucene.document.LongDocValuesField;
+import org.apache.lucene.document.LongField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.FacetTestUtils;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.search.DrillDownQuery;
+import org.apache.lucene.facet.search.FacetResult;
+import org.apache.lucene.facet.search.FacetResultNode;
+import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util._TestUtil;
+
+public class TestRangeAccumulator extends FacetTestCase {
+
+  public void testBasicLong() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+    Document doc = new Document();
+    NumericDocValuesField field = new NumericDocValuesField("field", 0L);
+    doc.add(field);
+    for(long l=0;l<100;l++) {
+      field.setLongValue(l);
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    FacetSearchParams fsp = new FacetSearchParams(
+                                new RangeFacetRequest<LongRange>("field",
+                                                      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),
+                                                      new LongRange("90 or above", 90L, true, 100L, false),
+                                                      new LongRange("over 1000", 1000L, false, Long.MAX_VALUE, false)));
+
+    RangeAccumulator a = new RangeAccumulator(fsp, r);
+    
+    FacetsCollector fc = FacetsCollector.create(a);
+
+    IndexSearcher s = newSearcher(r);
+    s.search(new MatchAllDocsQuery(), fc);
+    List<FacetResult> result = fc.getFacetResults();
+    assertEquals(1, result.size());
+    assertEquals("field (0)\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", FacetTestUtils.toSimpleString(result.get(0)));
+    
+    r.close();
+    d.close();
+  }
+
+  public void testBasicDouble() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+    Document doc = new Document();
+    DoubleDocValuesField field = new DoubleDocValuesField("field", 0.0);
+    doc.add(field);
+    for(long l=0;l<100;l++) {
+      field.setDoubleValue((double) l);
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    FacetSearchParams fsp = new FacetSearchParams(
+                                new RangeFacetRequest<DoubleRange>("field",
+                                                      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),
+                                                      new DoubleRange("90 or above", 90.0, true, 100.0, false),
+                                                      new DoubleRange("over 1000", 1000.0, false, Double.POSITIVE_INFINITY, false)));
+
+    RangeAccumulator a = new RangeAccumulator(fsp, r);
+    
+    FacetsCollector fc = FacetsCollector.create(a);
+
+    IndexSearcher s = newSearcher(r);
+    s.search(new MatchAllDocsQuery(), fc);
+    List<FacetResult> result = fc.getFacetResults();
+    assertEquals(1, result.size());
+    assertEquals("field (0)\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", FacetTestUtils.toSimpleString(result.get(0)));
+    
+    r.close();
+    d.close();
+  }
+
+  public void testBasicFloat() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+    Document doc = new Document();
+    FloatDocValuesField field = new FloatDocValuesField("field", 0.0f);
+    doc.add(field);
+    for(long l=0;l<100;l++) {
+      field.setFloatValue((float) l);
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    FacetSearchParams fsp = new FacetSearchParams(
+                                new RangeFacetRequest<FloatRange>("field",
+                                                      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)));
+
+    RangeAccumulator a = new RangeAccumulator(fsp, r);
+    
+    FacetsCollector fc = FacetsCollector.create(a);
+
+    IndexSearcher s = newSearcher(r);
+    s.search(new MatchAllDocsQuery(), fc);
+    List<FacetResult> result = fc.getFacetResults();
+    assertEquals(1, result.size());
+    assertEquals("field (0)\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", FacetTestUtils.toSimpleString(result.get(0)));
+    
+    r.close();
+    d.close();
+  }
+
+  public void testRandomLongs() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    int numDocs = atLeast(1000);
+    long[] values = new long[numDocs];
+    for(int i=0;i<numDocs;i++) {
+      Document doc = new Document();
+      long v = random().nextLong();
+      values[i] = v;
+      doc.add(new LongDocValuesField("field", v));
+      doc.add(new LongField("field", v, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    
+    int numIters = atLeast(10);
+    for(int iter=0;iter<numIters;iter++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter);
+      }
+      int numRange = _TestUtil.nextInt(random(), 1, 5);
+      LongRange[] ranges = new LongRange[numRange];
+      int[] expectedCounts = new int[numRange];
+      for(int rangeID=0;rangeID<numRange;rangeID++) {
+        long min = random().nextLong();
+        long max = random().nextLong();
+        if (min > max) {
+          long x = min;
+          min = max;
+          max = x;
+        }
+        boolean minIncl = random().nextBoolean();
+        boolean maxIncl = random().nextBoolean();
+        ranges[rangeID] = new LongRange("r" + rangeID, min, minIncl, max, maxIncl);
+
+        // Do "slow but hopefully correct" computation of
+        // expected count:
+        for(int i=0;i<numDocs;i++) {
+          boolean accept = true;
+          if (minIncl) {
+            accept &= values[i] >= min;
+          } else {
+            accept &= values[i] > min;
+          }
+          if (maxIncl) {
+            accept &= values[i] <= max;
+          } else {
+            accept &= values[i] < max;
+          }
+          if (accept) {
+            expectedCounts[rangeID]++;
+          }
+        }
+      }
+
+      FacetSearchParams fsp = new FacetSearchParams(new RangeFacetRequest<LongRange>("field", ranges));
+      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, r));
+      s.search(new MatchAllDocsQuery(), fc);
+      List<FacetResult> results = fc.getFacetResults();
+      assertEquals(1, results.size());
+      List<FacetResultNode> nodes = results.get(0).getFacetResultNode().subResults;
+      assertEquals(numRange, nodes.size());
+      for(int rangeID=0;rangeID<numRange;rangeID++) {
+        if (VERBOSE) {
+          System.out.println("  range " + rangeID + " expectedCount=" + expectedCounts[rangeID]);
+        }
+        FacetResultNode subNode = nodes.get(rangeID);
+        assertEquals("field/r" + rangeID, subNode.label.toString('/'));
+        assertEquals(expectedCounts[rangeID], (int) subNode.value);
+
+        LongRange range = (LongRange) ((RangeFacetRequest) results.get(0).getFacetRequest()).ranges[rangeID];
+
+        // Test drill-down:
+        DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);
+        ddq.add("field", NumericRangeQuery.newLongRange("field", range.min, range.max, range.minInclusive, range.maxInclusive));
+        assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
+      }
+    }
+
+    r.close();
+    dir.close();
+  }
+
+  public void testRandomFloats() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    int numDocs = atLeast(1000);
+    float[] values = new float[numDocs];
+    for(int i=0;i<numDocs;i++) {
+      Document doc = new Document();
+      float v = random().nextFloat();
+      values[i] = v;
+      doc.add(new FloatDocValuesField("field", v));
+      doc.add(new FloatField("field", v, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    
+    int numIters = atLeast(10);
+    for(int iter=0;iter<numIters;iter++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter);
+      }
+      int numRange = _TestUtil.nextInt(random(), 1, 5);
+      FloatRange[] ranges = new FloatRange[numRange];
+      int[] expectedCounts = new int[numRange];
+      for(int rangeID=0;rangeID<numRange;rangeID++) {
+        float min = random().nextFloat();
+        float max = random().nextFloat();
+        if (min > max) {
+          float x = min;
+          min = max;
+          max = x;
+        }
+        boolean minIncl = random().nextBoolean();
+        boolean maxIncl = random().nextBoolean();
+        ranges[rangeID] = new FloatRange("r" + rangeID, min, minIncl, max, maxIncl);
+
+        // Do "slow but hopefully correct" computation of
+        // expected count:
+        for(int i=0;i<numDocs;i++) {
+          boolean accept = true;
+          if (minIncl) {
+            accept &= values[i] >= min;
+          } else {
+            accept &= values[i] > min;
+          }
+          if (maxIncl) {
+            accept &= values[i] <= max;
+          } else {
+            accept &= values[i] < max;
+          }
+          if (accept) {
+            expectedCounts[rangeID]++;
+          }
+        }
+      }
+
+      FacetSearchParams fsp = new FacetSearchParams(new RangeFacetRequest<FloatRange>("field", ranges));
+      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, r));
+      s.search(new MatchAllDocsQuery(), fc);
+      List<FacetResult> results = fc.getFacetResults();
+      assertEquals(1, results.size());
+      List<FacetResultNode> nodes = results.get(0).getFacetResultNode().subResults;
+      assertEquals(numRange, nodes.size());
+      for(int rangeID=0;rangeID<numRange;rangeID++) {
+        if (VERBOSE) {
+          System.out.println("  range " + rangeID + " expectedCount=" + expectedCounts[rangeID]);
+        }
+        FacetResultNode subNode = nodes.get(rangeID);
+        assertEquals("field/r" + rangeID, subNode.label.toString('/'));
+        assertEquals(expectedCounts[rangeID], (int) subNode.value);
+
+        FloatRange range = (FloatRange) ((RangeFacetRequest) results.get(0).getFacetRequest()).ranges[rangeID];
+
+        // Test drill-down:
+        DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);
+        ddq.add("field", NumericRangeQuery.newFloatRange("field", range.min, range.max, range.minInclusive, range.maxInclusive));
+        assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
+      }
+    }
+
+    r.close();
+    dir.close();
+  }
+
+  public void testRandomDoubles() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    int numDocs = atLeast(1000);
+    double[] values = new double[numDocs];
+    for(int i=0;i<numDocs;i++) {
+      Document doc = new Document();
+      double v = random().nextDouble();
+      values[i] = v;
+      doc.add(new DoubleDocValuesField("field", v));
+      doc.add(new DoubleField("field", v, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    
+    int numIters = atLeast(10);
+    for(int iter=0;iter<numIters;iter++) {
+      if (VERBOSE) {
+        System.out.println("TEST: iter=" + iter);
+      }
+      int numRange = _TestUtil.nextInt(random(), 1, 5);
+      DoubleRange[] ranges = new DoubleRange[numRange];
+      int[] expectedCounts = new int[numRange];
+      for(int rangeID=0;rangeID<numRange;rangeID++) {
+        double min = random().nextDouble();
+        double max = random().nextDouble();
+        if (min > max) {
+          double x = min;
+          min = max;
+          max = x;
+        }
+        boolean minIncl = random().nextBoolean();
+        boolean maxIncl = random().nextBoolean();
+        ranges[rangeID] = new DoubleRange("r" + rangeID, min, minIncl, max, maxIncl);
+
+        // Do "slow but hopefully correct" computation of
+        // expected count:
+        for(int i=0;i<numDocs;i++) {
+          boolean accept = true;
+          if (minIncl) {
+            accept &= values[i] >= min;
+          } else {
+            accept &= values[i] > min;
+          }
+          if (maxIncl) {
+            accept &= values[i] <= max;
+          } else {
+            accept &= values[i] < max;
+          }
+          if (accept) {
+            expectedCounts[rangeID]++;
+          }
+        }
+      }
+
+      FacetSearchParams fsp = new FacetSearchParams(new RangeFacetRequest<DoubleRange>("field", ranges));
+      FacetsCollector fc = FacetsCollector.create(new RangeAccumulator(fsp, r));
+      s.search(new MatchAllDocsQuery(), fc);
+      List<FacetResult> results = fc.getFacetResults();
+      assertEquals(1, results.size());
+      List<FacetResultNode> nodes = results.get(0).getFacetResultNode().subResults;
+      assertEquals(numRange, nodes.size());
+      for(int rangeID=0;rangeID<numRange;rangeID++) {
+        if (VERBOSE) {
+          System.out.println("  range " + rangeID + " expectedCount=" + expectedCounts[rangeID]);
+        }
+        FacetResultNode subNode = nodes.get(rangeID);
+        assertEquals("field/r" + rangeID, subNode.label.toString('/'));
+        assertEquals(expectedCounts[rangeID], (int) subNode.value);
+
+        DoubleRange range = (DoubleRange) ((RangeFacetRequest) results.get(0).getFacetRequest()).ranges[rangeID];
+
+        // Test drill-down:
+        DrillDownQuery ddq = new DrillDownQuery(FacetIndexingParams.DEFAULT);
+        ddq.add("field", NumericRangeQuery.newDoubleRange("field", range.min, range.max, range.minInclusive, range.maxInclusive));
+        assertEquals(expectedCounts[rangeID], s.search(ddq, 10).totalHits);
+      }
+    }
+
+    r.close();
+    dir.close();
+  }
+}
+