You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/02/07 22:32:28 UTC

svn commit: r1443736 [1/3] - in /lucene/dev/trunk/lucene: ./ demo/src/java/org/apache/lucene/demo/facet/adaptive/ demo/src/java/org/apache/lucene/demo/facet/multiCL/ facet/src/java/org/apache/lucene/facet/associations/search/ facet/src/java/org/apache/...

Author: shaie
Date: Thu Feb  7 21:32:27 2013
New Revision: 1443736

URL: http://svn.apache.org/r1443736
Log:
LUCENE-4757: cleanup FacetsAccumulator API

Added:
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/search/
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/search/AssociationIntSumFacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/IntermediateFacetResult.java
      - copied, changed from r1443446, lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/IntermediateFacetResult.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/PartitionsFacetResultsHandler.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FloatFacetResultsHandler.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/IntFacetResultsHandler.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/MatchingDocsAsScoredDocIDs.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java   (with props)
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/SumScoreFacetRequest.java
      - copied, changed from r1442878, lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsAggregatorTest.java
      - copied, changed from r1442490, lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsCollectorTest.java
Removed:
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/ScoreFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/IntermediateFacetResult.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/CountingFacetsCollectorTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestScoredDocIdCollector.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/adaptive/AdaptiveSearcher.java
    lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/multiCL/MultiCLSearcher.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/SamplingWrapper.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIDs.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/TotalFacetCounts.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/FacetSearchParams.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationFloatSumFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/associations/AssociationIntSumFacetRequest.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/FacetResult.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/FacetResultNode.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/Sampler.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/sampling/SamplingAccumulator.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/ResultSortUtils.java
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/FacetTestBase.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/OrdinalMappingReaderTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/index/TestFacetsPayloadMigrationReader.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/AdaptiveAccumulatorTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/SamplingWrapperTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsAccumulatorWithComplement.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestFacetsCollector.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestMultipleCategoryLists.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestStandardFacetsAccumulator.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKInEachNodeResultHandler.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandler.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/TestTopKResultsHandlerRandom.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/associations/AssociationsFacetRequestTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/params/FacetRequestTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/BaseSampleTestTopK.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/OversampleWithDepthTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/sampling/SamplingAccumulatorTest.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/util/TestScoredDocIDsUtils.java
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/util/collections/ObjectToIntMapTest.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Thu Feb  7 21:32:27 2013
@@ -56,6 +56,13 @@ Changes in backwards compatibility polic
     based on any properties of the content.
    (Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
 
+* LUCENE-4757: Cleanup and refactoring of FacetsAccumulator, FacetRequest,
+  FacetsAggregator and FacetResultsHandler API. If your application did 
+  FacetsCollector.create(), you should not be affected, but if you wrote
+  an Aggregator, then you should migrate it to the per-segment
+  FacetsAggregator. You can still use StandardFacetsAccumulator, which works
+  with the old API (for now). (Shai Erera) 
+
 Optimizations
 
 * LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate

Modified: lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/adaptive/AdaptiveSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/adaptive/AdaptiveSearcher.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/adaptive/AdaptiveSearcher.java (original)
+++ lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/adaptive/AdaptiveSearcher.java Thu Feb  7 21:32:27 2013
@@ -5,7 +5,7 @@ import java.util.List;
 import org.apache.lucene.demo.facet.ExampleUtils;
 import org.apache.lucene.demo.facet.simple.SimpleUtils;
 import org.apache.lucene.facet.search.AdaptiveFacetsAccumulator;
-import org.apache.lucene.facet.search.ScoredDocIdCollector;
+import org.apache.lucene.facet.search.FacetsCollector;
 import org.apache.lucene.facet.search.params.CountFacetRequest;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
@@ -56,7 +56,7 @@ public class AdaptiveSearcher {
    * @throws Exception on error (no detailed exception handling here for sample simplicity
    * @return facet results
    */
-  public static List<FacetResult> searchWithFacets (Directory indexDir, Directory taxoDir) throws Exception {
+  public static List<FacetResult> searchWithFacets(Directory indexDir, Directory taxoDir) throws Exception {
     // prepare index reader and taxonomy.
     TaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
     IndexReader indexReader = DirectoryReader.open(indexDir);
@@ -76,20 +76,17 @@ public class AdaptiveSearcher {
     // regular collector for scoring matched documents
     TopScoreDocCollector topDocsCollector = TopScoreDocCollector.create(10, true); 
     
-    // docids collector for guiding facets accumulation (scoring disabled)
-    ScoredDocIdCollector docIdsCollecor = ScoredDocIdCollector.create(indexReader.maxDoc(), false);
-    
     // Faceted search parameters indicate which facets are we interested in 
-    FacetSearchParams facetSearchParams = new FacetSearchParams(
-        new CountFacetRequest(new CategoryPath("root", "a"), 10));
+    FacetSearchParams fsp = new FacetSearchParams(new CountFacetRequest(new CategoryPath("root", "a"), 10));
+    AdaptiveFacetsAccumulator accumulator = new AdaptiveFacetsAccumulator(fsp, indexReader, taxo);
+    FacetsCollector fc = FacetsCollector.create(accumulator);
     
     // search, into both collectors. note: in case only facets accumulation 
     // is required, the topDocCollector part can be totally discarded
-    searcher.search(q, MultiCollector.wrap(topDocsCollector, docIdsCollecor));
+    searcher.search(q, MultiCollector.wrap(topDocsCollector, fc));
         
     // Obtain facets results and print them
-    AdaptiveFacetsAccumulator accumulator = new AdaptiveFacetsAccumulator(facetSearchParams, indexReader, taxo);
-    List<FacetResult> res = accumulator.accumulate(docIdsCollecor.getScoredDocIDs());
+    List<FacetResult> res = fc.getFacetResults();
     
     int i = 0;
     for (FacetResult facetResult : res) {

Modified: lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/multiCL/MultiCLSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/multiCL/MultiCLSearcher.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/multiCL/MultiCLSearcher.java (original)
+++ lucene/dev/trunk/lucene/demo/src/java/org/apache/lucene/demo/facet/multiCL/MultiCLSearcher.java Thu Feb  7 21:32:27 2013
@@ -115,7 +115,7 @@ public class MultiCLSearcher {
     facetRequests.add(new CountFacetRequest(new CategoryPath("5"), 10));
     facetRequests.add(new CountFacetRequest(new CategoryPath("5", "5"), 10));
     facetRequests.add(new CountFacetRequest(new CategoryPath("6", "2"), 10));
-    FacetSearchParams facetSearchParams = new FacetSearchParams(facetRequests, iParams);
+    FacetSearchParams facetSearchParams = new FacetSearchParams(iParams, facetRequests);
 
     // Facets collector is the simplest interface for faceted search.
     // It provides faceted search functions that are sufficient to many

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/search/AssociationIntSumFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/search/AssociationIntSumFacetsAggregator.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/search/AssociationIntSumFacetsAggregator.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/associations/search/AssociationIntSumFacetsAggregator.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,46 @@
+package org.apache.lucene.facet.associations.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.search.FacetsAggregator;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link FacetsAggregator} which computes the weight of a category as the sum
+ * of the integer values associated with it in the result documents.
+ */
+public class AssociationIntSumFacetsAggregator implements FacetsAggregator {
+  
+  @Override
+  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp,
+      FacetArrays facetArrays) throws IOException {}
+  
+  @Override
+  public void rollupValues(int ordinal, int[] children, int[] siblings,
+      FacetArrays facetArrays) {}
+  
+  @Override
+  public boolean requiresDocScores() {
+    return false;
+  }
+  
+}

Copied: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/IntermediateFacetResult.java (from r1443446, lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/IntermediateFacetResult.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/IntermediateFacetResult.java?p2=lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/IntermediateFacetResult.java&p1=lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/IntermediateFacetResult.java&r1=1443446&r2=1443736&rev=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/results/IntermediateFacetResult.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/IntermediateFacetResult.java Thu Feb  7 21:32:27 2013
@@ -1,7 +1,8 @@
-package org.apache.lucene.facet.search.results;
+package org.apache.lucene.facet.partitions.search;
 
 import org.apache.lucene.facet.search.FacetResultsHandler;
 import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.results.FacetResult;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/PartitionsFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/PartitionsFacetResultsHandler.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/PartitionsFacetResultsHandler.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/partitions/search/PartitionsFacetResultsHandler.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,137 @@
+package org.apache.lucene.facet.partitions.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.search.FacetResultsHandler;
+import org.apache.lucene.facet.search.ScoredDocIDs;
+import org.apache.lucene.facet.search.StandardFacetsAccumulator;
+import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.results.FacetResult;
+import org.apache.lucene.facet.search.results.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link FacetResultsHandler} designed to work with facet partitions.
+ * 
+ * @lucene.experimental
+ */
+public abstract class PartitionsFacetResultsHandler extends FacetResultsHandler {
+  
+  public PartitionsFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, 
+      FacetArrays facetArrays) {
+    super(taxonomyReader, facetRequest, facetArrays);
+  }
+
+
+  /**
+   * Fetch results of a single partition, given facet arrays for that partition,
+   * and based on the matching documents and faceted search parameters.
+   * @param offset
+   *          offset in input arrays where partition starts
+   * 
+   * @return temporary facet result, potentially, to be passed back to
+   *         <b>this</b> result handler for merging, or <b>null</b> in case that
+   *         constructor parameter, <code>facetRequest</code>, requests an
+   *         illegal FacetResult, like, e.g., a root node category path that
+   *         does not exist in constructor parameter <code>taxonomyReader</code>
+   *         .
+   * @throws IOException
+   *           on error
+   */
+  public abstract IntermediateFacetResult fetchPartitionResult(int offset) throws IOException;
+
+  /**
+   * Merge results of several facet partitions. Logic of the merge is undefined
+   * and open for interpretations. For example, a merge implementation could
+   * keep top K results. Passed {@link IntermediateFacetResult} must be ones
+   * that were created by this handler otherwise a {@link ClassCastException} is
+   * thrown. In addition, all passed {@link IntermediateFacetResult} must have
+   * the same {@link FacetRequest} otherwise an {@link IllegalArgumentException}
+   * is thrown.
+   * 
+   * @param tmpResults one or more temporary results created by <b>this</b>
+   *        handler.
+   * @return temporary facet result that represents to union, as specified by
+   *         <b>this</b> handler, of the input temporary facet results.
+   * @throws IOException on error.
+   * @throws ClassCastException if the temporary result passed was not created
+   *         by this handler
+   * @throws IllegalArgumentException if passed <code>facetResults</code> do not
+   *         have the same {@link FacetRequest}
+   * @see IntermediateFacetResult#getFacetRequest()
+   */
+  public abstract IntermediateFacetResult mergeResults(IntermediateFacetResult... tmpResults) throws IOException;
+
+  /**
+   * Create a facet result from the temporary result.
+   * @param tmpResult temporary result to be rendered as a {@link FacetResult}
+   * @throws IOException on error.
+   */
+  public abstract FacetResult renderFacetResult(IntermediateFacetResult tmpResult) throws IOException ;
+
+  /**
+   * Perform any rearrangement as required on a facet result that has changed after
+   * it was rendered.
+   * <P>
+   * Possible use case: a sampling facets accumulator invoked another 
+   * other facets accumulator on a sample set of documents, obtained
+   * rendered facet results, fixed their counts, and now it is needed 
+   * to sort the results differently according to the fixed counts. 
+   * @param facetResult result to be rearranged.
+   * @see FacetResultNode#value
+   */
+  public abstract FacetResult rearrangeFacetResult(FacetResult facetResult);
+
+  /**
+   * Label results according to settings in {@link FacetRequest}, such as
+   * {@link FacetRequest#getNumLabel()}. Usually invoked by
+   * {@link StandardFacetsAccumulator#accumulate(ScoredDocIDs)}
+   * 
+   * @param facetResult
+   *          facet result to be labeled.
+   * @throws IOException
+   *           on error
+   */
+  public abstract void labelResult(FacetResult facetResult) throws IOException;
+
+  /**
+   * Check if an array contains the partition which contains ordinal
+   * 
+   * @param ordinal
+   *          checked facet
+   * @param facetArrays
+   *          facet arrays for the certain partition
+   * @param offset
+   *          offset in input arrays where partition starts
+   */
+  protected boolean isSelfPartition (int ordinal, FacetArrays facetArrays, int offset) {
+    int partitionSize = facetArrays.arrayLength;
+    return ordinal / partitionSize == offset / partitionSize;
+  }
+  
+  @Override
+  public final FacetResult compute() throws IOException {
+    FacetResult res = renderFacetResult(fetchPartitionResult(0));
+    labelResult(res);
+    return res;
+  }
+
+}

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java Thu Feb  7 21:32:27 2013
@@ -74,7 +74,7 @@ public final class AdaptiveFacetsAccumul
   
   @Override
   public List<FacetResult> accumulate(ScoredDocIDs docids) throws IOException {
-    FacetsAccumulator delegee = appropriateFacetCountingAccumulator(docids);
+    StandardFacetsAccumulator delegee = appropriateFacetCountingAccumulator(docids);
 
     if (delegee == this) {
       return super.accumulate(docids);
@@ -87,7 +87,7 @@ public final class AdaptiveFacetsAccumul
    * Compute the appropriate facet accumulator to use.
    * If no special/clever adaptation is possible/needed return this (self).
    */
-  private FacetsAccumulator appropriateFacetCountingAccumulator(ScoredDocIDs docids) {
+  private StandardFacetsAccumulator appropriateFacetCountingAccumulator(ScoredDocIDs docids) {
     // Verify that searchPareams permit sampling/complement/etc... otherwise do default
     if (!mayComplement()) {
       return this;

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,83 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.IntsRef;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link FacetsAggregator} which counts the number of times each category
+ * appears in the given set of documents. This aggregator uses the
+ * {@link CategoryListIterator} to read the encoded categories. If you used the
+ * default settings while idnexing, you can use
+ * {@link FastCountingFacetsAggregator} for better performance.
+ * 
+ * @lucene.experimental
+ */
+public class CountingFacetsAggregator implements FacetsAggregator {
+  
+  private final IntsRef ordinals = new IntsRef(32);
+  
+  @Override
+  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException {
+    final CategoryListIterator cli = clp.createCategoryListIterator(0);
+    if (!cli.setNextReader(matchingDocs.context)) {
+      return;
+    }
+    
+    final int length = matchingDocs.bits.length();
+    final int[] counts = facetArrays.getIntArray();
+    int doc = 0;
+    while (doc < length && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+      cli.getOrdinals(doc, ordinals);
+      final int upto = ordinals.offset + ordinals.length;
+      for (int i = ordinals.offset; i < upto; i++) {
+        ++counts[ordinals.ints[i]];
+      }
+      ++doc;
+    }
+  }
+  
+  private int rollupCounts(int ordinal, int[] children, int[] siblings, int[] counts) {
+    int count = 0;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      int childCount = counts[ordinal];
+      childCount += rollupCounts(children[ordinal], children, siblings, counts);
+      counts[ordinal] = childCount;
+      count += childCount;
+      ordinal = siblings[ordinal];
+    }
+    return count;
+  }
+
+  @Override
+  public void rollupValues(int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
+    final int[] counts = facetArrays.getIntArray();
+    counts[ordinal] += rollupCounts(children[ordinal], children, siblings, counts);
+  }
+  
+  @Override
+  public final boolean requiresDocScores() {
+    return false;
+  }
+  
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,144 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+
+import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.params.FacetRequest.SortBy;
+import org.apache.lucene.facet.search.params.FacetRequest.SortOrder;
+import org.apache.lucene.facet.search.results.FacetResult;
+import org.apache.lucene.facet.search.results.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.ParallelTaxonomyArrays;
+import org.apache.lucene.util.PriorityQueue;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link FacetResultsHandler} which counts the top-K facets at depth 1 only
+ * and always labels all result categories. The results are always sorted by
+ * value, in descending order. Sub-classes are responsible to pull the values
+ * from the corresponding {@link FacetArrays}.
+ * 
+ * @lucene.experimental
+ */
+public abstract class DepthOneFacetResultsHandler extends FacetResultsHandler {
+  
+  private static class FacetResultNodeQueue extends PriorityQueue<FacetResultNode> {
+    
+    public FacetResultNodeQueue(int maxSize, boolean prepopulate) {
+      super(maxSize, prepopulate);
+    }
+    
+    @Override
+    protected FacetResultNode getSentinelObject() {
+      return new FacetResultNode();
+    }
+    
+    @Override
+    protected boolean lessThan(FacetResultNode a, FacetResultNode b) {
+      if (a.value < b.value) return true;
+      if (a.value > b.value) return false;
+      // both have the same value, break tie by ordinal
+      return a.ordinal < b.ordinal;
+    }
+    
+  }
+
+  public DepthOneFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
+    super(taxonomyReader, facetRequest, facetArrays);
+    assert facetRequest.getDepth() == 1 : "this handler only computes the top-K facets at depth 1";
+    assert facetRequest.numResults == facetRequest.getNumLabel() : "this handler always labels all top-K results";
+    assert facetRequest.getSortOrder() == SortOrder.DESCENDING : "this handler always sorts results in descending order";
+    assert facetRequest.getSortBy() == SortBy.VALUE : "this handler always sorts results by value";
+  }
+
+  /** Returnt the value of the requested ordinal. Called once for the result root. */
+  protected abstract double valueOf(int ordinal);
+  
+  /**
+   * Add the siblings of {@code ordinal} to the given list. This is called
+   * whenever the number of results is too high (&gt; taxonomy size), instead of
+   * adding them to a {@link PriorityQueue}.
+   */
+  protected abstract void addSiblings(int ordinal, int[] siblings, ArrayList<FacetResultNode> nodes) throws IOException;
+  
+  /**
+   * Add the siblings of {@code ordinal} to the given {@link PriorityQueue}. The
+   * given {@link PriorityQueue} is already filled with sentinel objects, so
+   * implementations are encouraged to use {@link PriorityQueue#top()} and
+   * {@link PriorityQueue#updateTop()} for best performance.
+   */
+  protected abstract int addSiblings(int ordinal, int[] siblings, PriorityQueue<FacetResultNode> pq);
+  
+  @Override
+  public final FacetResult compute() throws IOException {
+    ParallelTaxonomyArrays arrays = taxonomyReader.getParallelTaxonomyArrays();
+    final int[] children = arrays.children();
+    final int[] siblings = arrays.siblings();
+    
+    int rootOrd = taxonomyReader.getOrdinal(facetRequest.categoryPath);
+        
+    FacetResultNode root = new FacetResultNode();
+    root.ordinal = rootOrd;
+    root.label = facetRequest.categoryPath;
+    root.value = valueOf(rootOrd);
+    if (facetRequest.numResults > taxonomyReader.getSize()) {
+      // specialize this case, user is interested in all available results
+      ArrayList<FacetResultNode> nodes = new ArrayList<FacetResultNode>();
+      int child = children[rootOrd];
+      addSiblings(child, siblings, nodes);
+      Collections.sort(nodes, new Comparator<FacetResultNode>() {
+        @Override
+        public int compare(FacetResultNode o1, FacetResultNode o2) {
+          int value = (int) (o2.value - o1.value);
+          if (value == 0) {
+            value = o2.ordinal - o1.ordinal;
+          }
+          return value;
+        }
+      });
+      
+      root.subResults = nodes;
+      return new FacetResult(facetRequest, root, nodes.size());
+    }
+    
+    // since we use sentinel objects, we cannot reuse PQ. but that's ok because it's not big
+    PriorityQueue<FacetResultNode> pq = new FacetResultNodeQueue(facetRequest.numResults, true);
+    int numResults = addSiblings(children[rootOrd], siblings, pq);
+
+    // pop() the least (sentinel) elements
+    int pqsize = pq.size();
+    int size = numResults < pqsize ? numResults : pqsize;
+    for (int i = pqsize - size; i > 0; i--) { pq.pop(); }
+
+    // create the FacetResultNodes.
+    FacetResultNode[] subResults = new FacetResultNode[size];
+    for (int i = size - 1; i >= 0; i--) {
+      FacetResultNode node = pq.pop();
+      node.label = taxonomyReader.getPath(node.ordinal);
+      subResults[i] = node;
+    }
+    root.subResults = Arrays.asList(subResults);
+    return new FacetResult(facetRequest, root, size);
+  }
+  
+}

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/DocValuesCategoryListIterator.java Thu Feb  7 21:32:27 2013
@@ -71,6 +71,7 @@ public class DocValuesCategoryListIterat
   
   @Override
   public void getOrdinals(int docID, IntsRef ints) throws IOException {
+    assert current != null : "don't call this if setNextReader returned false";
     current.get(docID, bytes);
     ints.length = 0;
     if (bytes.length > 0) {

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java Thu Feb  7 21:32:27 2013
@@ -4,8 +4,6 @@ import java.io.IOException;
 
 import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.search.results.FacetResult;
-import org.apache.lucene.facet.search.results.FacetResultNode;
-import org.apache.lucene.facet.search.results.IntermediateFacetResult;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
@@ -26,136 +24,25 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Handler for facet results.
- * <p>
- * The facet results handler provided by the {@link FacetRequest} to 
- * a {@link FacetsAccumulator}.
- * <p>
- * First it is used by {@link FacetsAccumulator} to obtain a temporary 
- * facet result for each partition and to merge results of several partitions.
- * <p>
- * Later the accumulator invokes the handler to render the results, creating 
- * {@link FacetResult} objects.
- * <p>
- * Last the accumulator invokes the handler to label final results. 
+ * Computes the top categories for a given {@link FacetRequest}. 
  * 
  * @lucene.experimental
  */
 public abstract class FacetResultsHandler {
 
-  /** Taxonomy for which facets are handled */
-  protected final TaxonomyReader taxonomyReader;
+  public final TaxonomyReader taxonomyReader;
 
-  /**
-   * Facet request served by this handler.
-   */
-  protected final FacetRequest facetRequest;
-
-  /**
-   * Create a faceted search handler.
-   * @param taxonomyReader See {@link #getTaxonomyReader()}.
-   * @param facetRequest See {@link #getFacetRequest()}.
-   */
-  public FacetResultsHandler(TaxonomyReader taxonomyReader,
-                              FacetRequest facetRequest) {
+  public final FacetRequest facetRequest;
+  
+  protected final FacetArrays facetArrays;
+
+  public FacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
     this.taxonomyReader = taxonomyReader;
     this.facetRequest = facetRequest;
+    this.facetArrays = facetArrays;
   }
 
-  /**
-   * Fetch results of a single partition, given facet arrays for that partition,
-   * and based on the matching documents and faceted search parameters.
-   * 
-   * @param arrays
-   *          facet arrays for the certain partition
-   * @param offset
-   *          offset in input arrays where partition starts
-   * @return temporary facet result, potentially, to be passed back to
-   *         <b>this</b> result handler for merging, or <b>null</b> in case that
-   *         constructor parameter, <code>facetRequest</code>, requests an
-   *         illegal FacetResult, like, e.g., a root node category path that
-   *         does not exist in constructor parameter <code>taxonomyReader</code>
-   *         .
-   * @throws IOException
-   *           on error
-   */
-  public abstract IntermediateFacetResult fetchPartitionResult(FacetArrays arrays, int offset) throws IOException;
-
-  /**
-   * Merge results of several facet partitions. Logic of the merge is undefined
-   * and open for interpretations. For example, a merge implementation could
-   * keep top K results. Passed {@link IntermediateFacetResult} must be ones
-   * that were created by this handler otherwise a {@link ClassCastException} is
-   * thrown. In addition, all passed {@link IntermediateFacetResult} must have
-   * the same {@link FacetRequest} otherwise an {@link IllegalArgumentException}
-   * is thrown.
-   * 
-   * @param tmpResults one or more temporary results created by <b>this</b>
-   *        handler.
-   * @return temporary facet result that represents to union, as specified by
-   *         <b>this</b> handler, of the input temporary facet results.
-   * @throws IOException on error.
-   * @throws ClassCastException if the temporary result passed was not created
-   *         by this handler
-   * @throws IllegalArgumentException if passed <code>facetResults</code> do not
-   *         have the same {@link FacetRequest}
-   * @see IntermediateFacetResult#getFacetRequest()
-   */
-  public abstract IntermediateFacetResult mergeResults(IntermediateFacetResult... tmpResults) 
-  throws IOException, ClassCastException, IllegalArgumentException;
-
-  /**
-   * Create a facet result from the temporary result.
-   * @param tmpResult temporary result to be rendered as a {@link FacetResult}
-   * @throws IOException on error.
-   */
-  public abstract FacetResult renderFacetResult(IntermediateFacetResult tmpResult) throws IOException ;
-
-  /**
-   * Perform any rearrangement as required on a facet result that has changed after
-   * it was rendered.
-   * <P>
-   * Possible use case: a sampling facets accumulator invoked another 
-   * other facets accumulator on a sample set of documents, obtained
-   * rendered facet results, fixed their counts, and now it is needed 
-   * to sort the results differently according to the fixed counts. 
-   * @param facetResult result to be rearranged.
-   * @see FacetResultNode#value
-   */
-  public abstract FacetResult rearrangeFacetResult(FacetResult facetResult);
-
-  /**
-   * Label results according to settings in {@link FacetRequest}, 
-   * such as {@link FacetRequest#getNumLabel()}. 
-   * Usually invoked by {@link FacetsAccumulator#accumulate(ScoredDocIDs)}
-   * @param facetResult facet result to be labeled. 
-   * @throws IOException on error 
-   */
-  public abstract void labelResult (FacetResult facetResult) throws IOException;
-
-  /** Return taxonomy reader used for current facets accumulation operation. */
-  public final TaxonomyReader getTaxonomyReader() {
-    return this.taxonomyReader;
-  }
-
-  /** Return the facet request served by this handler. */
-  public final FacetRequest getFacetRequest() {
-    return this.facetRequest;
-  }
-
-  /**
-   * Check if an array contains the partition which contains ordinal
-   * 
-   * @param ordinal
-   *          checked facet
-   * @param facetArrays
-   *          facet arrays for the certain partition
-   * @param offset
-   *          offset in input arrays where partition starts
-   */
-  protected boolean isSelfPartition (int ordinal, FacetArrays facetArrays, int offset) {
-    int partitionSize = facetArrays.arrayLength;
-    return ordinal / partitionSize == offset / partitionSize;
-  }
-
+  /** Computes the {@link FacetResult} for the given {@link FacetArrays}. */
+  public abstract FacetResult compute() throws IOException;
+  
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java Thu Feb  7 21:32:27 2013
@@ -1,14 +1,26 @@
 package org.apache.lucene.facet.search;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
-import org.apache.lucene.index.IndexReader;
-
-import org.apache.lucene.facet.search.params.FacetSearchParams;
+import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.apache.lucene.facet.index.params.CategoryListParams.OrdinalPolicy;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
 import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.params.FacetSearchParams;
+import org.apache.lucene.facet.search.params.FacetRequest.FacetArraysSource;
+import org.apache.lucene.facet.search.params.FacetRequest.ResultMode;
+import org.apache.lucene.facet.search.params.FacetRequest.SortBy;
+import org.apache.lucene.facet.search.params.FacetRequest.SortOrder;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.ParallelTaxonomyArrays;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.encoding.DGapVInt8IntDecoder;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -33,116 +45,130 @@ import org.apache.lucene.facet.taxonomy.
  * 
  * @lucene.experimental
  */
-public abstract class FacetsAccumulator {
-
-  /**
-   * Default threshold for using the complements optimization.
-   * If accumulating facets for a document set larger than this ratio of the index size than 
-   * perform the complement optimization.
-   * @see #setComplementThreshold(double) for more info on the complements optimization.  
-   */
-  public static final double DEFAULT_COMPLEMENT_THRESHOLD = 0.6;
-
-  /**
-   * Passing this to {@link #setComplementThreshold(double)} will disable using complement optimization.
-   */
-  public static final double DISABLE_COMPLEMENT = Double.POSITIVE_INFINITY; // > 1 actually
-
-  /**
-   * Passing this to {@link #setComplementThreshold(double)} will force using complement optimization.
-   */
-  public static final double FORCE_COMPLEMENT = 0; // <=0  
-
-  private double complementThreshold = DEFAULT_COMPLEMENT_THRESHOLD;  
+public class FacetsAccumulator {
 
   protected final TaxonomyReader taxonomyReader;
   protected final IndexReader indexReader;
+  protected final FacetArrays facetArrays;
   protected FacetSearchParams searchParams;
 
-  private boolean allowLabeling = true;
-
-  public FacetsAccumulator(FacetSearchParams searchParams,
-                            IndexReader indexReader,
-                            TaxonomyReader taxonomyReader) {
+  /**
+   * Initializes the accumulator with the given search params, index reader and
+   * taxonomy reader. This constructor creates the default {@link FacetArrays},
+   * which do not support reuse. If you want to use {@link ReusingFacetArrays},
+   * you should use the
+   * {@link #FacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader, FacetArrays)}
+   * constructor.
+   */
+  public FacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
+    this(searchParams, indexReader, taxonomyReader, null);
+  }
+  
+  /**
+   * Initializes the accumulator with the given parameters as well as
+   * {@link FacetArrays}. Note that the accumulator doesn't call
+   * {@link FacetArrays#free()}. If you require that (only makes sense if you
+   * use {@link ReusingFacetArrays}, you should do it after you've finished with
+   * the accumulator.
+   */
+  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;
     this.searchParams = searchParams;
   }
-
-  /**
-   * Accumulate facets over given documents, according to facet requests in effect.
-   * @param docids documents (and their scores) for which facets are Accumulated.
-   * @return Accumulated facets.  
-   * @throws IOException on error.
-   */
-  // internal API note: it was considered to move the docids into the constructor as well, 
-  // but this prevents nice extension capabilities, especially in the way that 
-  // Sampling Accumulator works with the (any) delegated accumulator.
-  public abstract List<FacetResult> accumulate(ScoredDocIDs docids) throws IOException;
-
+  
   /**
-   * Returns the complement threshold.
-   * @see #setComplementThreshold(double)
-   */
-  public double getComplementThreshold() {
-    return complementThreshold;
+   * Returns the {@link FacetsAggregator} to use for aggregating the categories
+   * found in the result documents. The default implementation returns
+   * {@link CountingFacetsAggregator}, or {@link FastCountingFacetsAggregator}
+   * if all categories can be decoded with {@link DGapVInt8IntDecoder}.
+   */
+  public FacetsAggregator getAggregator() {
+    if (FastCountingFacetsAggregator.verifySearchParams(searchParams)) {
+      return new FastCountingFacetsAggregator();
+    } else {
+      return new CountingFacetsAggregator();
+    }
   }
-
+  
   /**
-   * Set the complement threshold.
-   * This threshold will dictate whether the complements optimization is applied.
-   * The optimization is to count for less documents. It is useful when the same 
-   * FacetSearchParams are used for varying sets of documents. The first time 
-   * complements is used the "total counts" are computed - counting for all the 
-   * documents in the collection. Then, only the complementing set of documents
-   * is considered, and used to decrement from the overall counts, thereby 
-   * walking through less documents, which is faster.
-   * <p>
-   * For the default settings see {@link #DEFAULT_COMPLEMENT_THRESHOLD}.
-   * <p>
-   * To forcing complements in all cases pass {@link #FORCE_COMPLEMENT}.
-   * This is mostly useful for testing purposes, as forcing complements when only 
-   * tiny fraction of available documents match the query does not make sense and 
-   * would incur performance degradations.
-   * <p>
-   * To disable complements pass {@link #DISABLE_COMPLEMENT}.
-   * @param complementThreshold the complement threshold to set
-   * @see #getComplementThreshold()
+   * Creates a {@link FacetResultsHandler} that matches the given
+   * {@link FacetRequest}.
    */
-  public void setComplementThreshold(double complementThreshold) {
-    this.complementThreshold = complementThreshold;
-  }
+  protected FacetResultsHandler createFacetResultsHandler(FacetRequest fr) {
+    if (fr.getDepth() == 1 && fr.getSortOrder() == SortOrder.DESCENDING && fr.getSortBy() == SortBy.VALUE) {
+      FacetArraysSource fas = fr.getFacetArraysSource();
+      if (fas == FacetArraysSource.INT) {
+        return new IntFacetResultsHandler(taxonomyReader, fr, facetArrays);
+      }
+      
+      if (fas == FacetArraysSource.FLOAT) {
+        return new FloatFacetResultsHandler(taxonomyReader, fr, facetArrays);
+      }
+    }
 
-  /**
-   * Check if labeling is allowed for this accumulator.
-   * <p>
-   * By default labeling is allowed.
-   * This allows one accumulator to invoke other accumulators for accumulation
-   * but keep to itself the responsibility of labeling.
-   * This might br handy since labeling is a costly operation. 
-   * @return true of labeling is allowed for this accumulator
-   * @see #setAllowLabeling(boolean)
-   */
-  protected boolean isAllowLabeling() {
-    return allowLabeling;
+    if (fr.getResultMode() == ResultMode.PER_NODE_IN_TREE) {
+      return new TopKInEachNodeHandler(taxonomyReader, fr, facetArrays);
+    } 
+    return new TopKFacetResultsHandler(taxonomyReader, fr, facetArrays);
   }
 
-  /**
-   * Set whether labeling is allowed for this accumulator.
-   * @param allowLabeling new setting for allow labeling
-   * @see #isAllowLabeling()
-   */
-  protected void setAllowLabeling(boolean allowLabeling) {
-    this.allowLabeling = allowLabeling;
+  protected Set<CategoryListParams> getCategoryLists() {
+    if (searchParams.indexingParams.getAllCategoryListParams().size() == 1) {
+      return Collections.singleton(searchParams.indexingParams.getCategoryListParams(null));
+    }
+    
+    HashSet<CategoryListParams> clps = new HashSet<CategoryListParams>();
+    for (FacetRequest fr : searchParams.facetRequests) {
+      clps.add(searchParams.indexingParams.getCategoryListParams(fr.categoryPath));
+    }
+    return clps;
   }
 
-  /** check if all requests are complementable */
-  protected boolean mayComplement() {
-    for (FacetRequest freq:searchParams.facetRequests) {
-      if (!freq.supportsComplements()) {
-        return false;
+  /**
+   * Used by {@link FacetsCollector} to build the list of {@link FacetResult
+   * facet results} that match the {@link FacetRequest facet requests} that were
+   * given in the constructor.
+   * 
+   * @param matchingDocs
+   *          the documents that matched the query, per-segment.
+   */
+  public List<FacetResult> accumulate(List<MatchingDocs> matchingDocs) throws IOException {
+    // aggregate facets per category list (usually onle one category list)
+    FacetsAggregator aggregator = getAggregator();
+    for (CategoryListParams clp : getCategoryLists()) {
+      for (MatchingDocs md : matchingDocs) {
+        aggregator.aggregate(md, clp, facetArrays);
       }
     }
-    return true;
+    
+    ParallelTaxonomyArrays arrays = taxonomyReader.getParallelTaxonomyArrays();
+    
+    // compute top-K
+    final int[] children = arrays.children();
+    final int[] siblings = arrays.siblings();
+    List<FacetResult> res = new ArrayList<FacetResult>();
+    for (FacetRequest fr : searchParams.facetRequests) {
+      int rootOrd = taxonomyReader.getOrdinal(fr.categoryPath);
+      if (rootOrd == TaxonomyReader.INVALID_ORDINAL) { // category does not exist
+        continue;
+      }
+      CategoryListParams clp = searchParams.indexingParams.getCategoryListParams(fr.categoryPath);
+      OrdinalPolicy ordinalPolicy = clp .getOrdinalPolicy(fr.categoryPath.components[0]);
+      if (ordinalPolicy == OrdinalPolicy.NO_PARENTS) {
+        // rollup values
+        aggregator.rollupValues(rootOrd, children, siblings, facetArrays);
+      }
+      
+      FacetResultsHandler frh = createFacetResultsHandler(fr);
+      res.add(frh.compute());
+    }
+    return res;
   }
-}
\ No newline at end of file
+
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,49 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.apache.lucene.facet.index.params.CategoryListParams.OrdinalPolicy;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+
+/*
+ * 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.
+ */
+
+/**
+ * Aggregates categories that were found in result documents (specified by
+ * {@link MatchingDocs}). If the aggregator requires document scores too, it
+ * should return {@code true} from {@link #requiresDocScores()}.
+ * 
+ * @lucene.experimental
+ */
+public interface FacetsAggregator {
+  
+  /** Aggregate the facets found in the given matching documents. */
+  public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException;
+  
+  /**
+   * Rollup the values of the given ordinal. This method is called when a
+   * category was indexed with {@link OrdinalPolicy#NO_PARENTS}. The given
+   * ordinal is the requested category, and you should use the children and
+   * siblings arrays to traverse its sub-tree.
+   */
+  public void rollupValues(int ordinal, int[] children, int[] siblings, FacetArrays facetArrays);
+  
+  /** Returns {@code true} if this aggregator requires document scores. */
+  public boolean requiresDocScores();
+  
+}

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java Thu Feb  7 21:32:27 2013
@@ -1,15 +1,21 @@
 package org.apache.lucene.facet.search;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.facet.search.params.CountFacetRequest;
 import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.FixedBitSet;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -38,34 +44,199 @@ import org.apache.lucene.search.Collecto
  * @lucene.experimental
  */
 public abstract class FacetsCollector extends Collector {
+
+  private static final class DocsAndScoresCollector extends FacetsCollector {
+
+    private AtomicReaderContext context;
+    private Scorer scorer;
+    private FixedBitSet bits;
+    private int totalHits;
+    private float[] scores;
+    
+    public DocsAndScoresCollector(FacetsAccumulator accumulator) {
+      super(accumulator);
+    }
+    
+    @Override
+    protected final void finish() {
+      if (bits != null) {
+        matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, scores));
+        bits = null;
+        scores = null;
+        context = null;
+      }
+    }
+    
+    @Override
+    public final boolean acceptsDocsOutOfOrder() {
+      return false;
+    }
+
+    @Override
+    public final void collect(int doc) throws IOException {
+      bits.set(doc);
+      if (totalHits >= scores.length) {
+        float[] newScores = new float[ArrayUtil.oversize(totalHits + 1, 4)];
+        System.arraycopy(scores, 0, newScores, 0, totalHits);
+        scores = newScores;
+      }
+      scores[totalHits] = scorer.score();
+      totalHits++;
+    }
+
+    @Override
+    public final void setScorer(Scorer scorer) throws IOException {
+      this.scorer = scorer;
+    }
+    
+    @Override
+    public final void setNextReader(AtomicReaderContext context) throws IOException {
+      if (bits != null) {
+        matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, scores));
+      }
+      bits = new FixedBitSet(context.reader().maxDoc());
+      totalHits = 0;
+      scores = new float[64]; // some initial size
+      this.context = context;
+    }
+
+  }
+
+  private final static class DocsOnlyCollector extends FacetsCollector {
+
+    private AtomicReaderContext context;
+    private FixedBitSet bits;
+    private int totalHits;
+
+    public DocsOnlyCollector(FacetsAccumulator accumulator) {
+      super(accumulator);
+    }
+    
+    @Override
+    protected final void finish() {
+      if (bits != null) {
+        matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, null));
+        bits = null;
+        context = null;
+      }
+    }
+    
+    @Override
+    public final boolean acceptsDocsOutOfOrder() {
+      return true;
+    }
+
+    @Override
+    public final void collect(int doc) throws IOException {
+      totalHits++;
+      bits.set(doc);
+    }
+
+    @Override
+    public final void setScorer(Scorer scorer) throws IOException {}
+    
+    @Override
+    public final void setNextReader(AtomicReaderContext context) throws IOException {
+      if (bits != null) {
+        matchingDocs.add(new MatchingDocs(this.context, bits, totalHits, null));
+      }
+      bits = new FixedBitSet(context.reader().maxDoc());
+      totalHits = 0;
+      this.context = context;
+    }
+  }
+  
+  /**
+   * Holds the documents that were matched in the {@link AtomicReaderContext}.
+   * If scores were required, then {@code scores} is not null.
+   */
+  public final static class MatchingDocs {
+    
+    public final AtomicReaderContext context;
+    public final FixedBitSet bits;
+    public final float[] scores;
+    public final int totalHits;
+    
+    public MatchingDocs(AtomicReaderContext context, FixedBitSet bits, int totalHits, float[] scores) {
+      this.context = context;
+      this.bits = bits;
+      this.scores = scores;
+      this.totalHits = totalHits;
+    }
+  }
   
   /**
-   * Returns the most optimized {@link FacetsCollector} for the given search
-   * parameters. The returned {@link FacetsCollector} is guaranteed to satisfy
-   * the requested parameters.
-   * 
-   * @throws IllegalArgumentException
-   *           if there is no built-in collector that can satisfy the search
-   *           parameters.
+   * Creates a {@link FacetsCollector} with the default
+   * {@link FacetsAccumulator}.
    */
   public static FacetsCollector create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
-    if (CountingFacetsCollector.assertParams(fsp) == null) {
-      return new CountingFacetsCollector(fsp, taxoReader);
+    if (fsp.indexingParams.getPartitionSize() != Integer.MAX_VALUE) {
+      return create(new StandardFacetsAccumulator(fsp, indexReader, taxoReader));
     }
     
-    if (StandardFacetsCollector.assertParams(fsp) == null) {
-      return new StandardFacetsCollector(fsp, indexReader, taxoReader);
+    for (FacetRequest fr : fsp.facetRequests) {
+      if (!(fr instanceof CountFacetRequest)) {
+        return create(new StandardFacetsAccumulator(fsp, indexReader, taxoReader));
+      }
     }
     
-    throw new IllegalArgumentException("None of the built-in FacetsCollectors can handle the given search params");
+    return create(new FacetsAccumulator(fsp, indexReader, taxoReader));
+  }
+
+  /**
+   * Creates a {@link FacetsCollector} that satisfies the requirements of the
+   * given {@link FacetsAccumulator}.
+   */
+  public static FacetsCollector create(FacetsAccumulator accumulator) {
+    if (accumulator.getAggregator().requiresDocScores()) {
+      return new DocsAndScoresCollector(accumulator);
+    } else {
+      return new DocsOnlyCollector(accumulator);
+    }
+  }
+
+  private final FacetsAccumulator accumulator;
+  
+  protected final List<MatchingDocs> matchingDocs = new ArrayList<MatchingDocs>();
+
+  protected FacetsCollector(FacetsAccumulator accumulator) {
+    this.accumulator = accumulator;
   }
   
   /**
+   * Called when the Collector has finished, so that the last
+   * {@link MatchingDocs} can be added.
+   */
+  protected abstract void finish();
+  
+  /**
    * Returns a {@link FacetResult} per {@link FacetRequest} set in
    * {@link FacetSearchParams}. Note that if one of the {@link FacetRequest
    * requests} is for a {@link CategoryPath} that does not exist in the taxonomy,
    * no matching {@link FacetResult} will be returned.
    */
-  public abstract List<FacetResult> getFacetResults() throws IOException;
+  public final List<FacetResult> getFacetResults() throws IOException {
+    finish();
+    return accumulator.accumulate(matchingDocs);
+  }
+  
+  /**
+   * Returns the documents matched by the query, one {@link MatchingDocs} per
+   * visited segment.
+   */
+  public final List<MatchingDocs> getMatchingDocs() {
+    finish();
+    return matchingDocs;
+  }
+  
+  /**
+   * Allows to reuse the collector between search requests. This method simply
+   * clears all collected documents (and scores) information, and does not
+   * attempt to reuse allocated memory spaces.
+   */
+  public final void reset() {
+    finish();
+    matchingDocs.clear();
+  }
 
 }

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,122 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+
+import org.apache.lucene.facet.index.params.CategoryListParams;
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.params.FacetSearchParams;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.encoding.DGapVInt8IntDecoder;
+import org.apache.lucene.util.encoding.DGapVInt8IntEncoder;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link FacetsAggregator} which counts the number of times each category
+ * appears in the given set of documents. This aggregator reads the categories
+ * from the {@link BinaryDocValues} field defined by
+ * {@link CategoryListParams#field}, and assumes that the category ordinals were
+ * encoded with {@link DGapVInt8IntEncoder}.
+ * 
+ * @lucene.experimental
+ */
+public final class FastCountingFacetsAggregator implements FacetsAggregator {
+  
+  private final BytesRef buf = new BytesRef(32);
+  
+  /**
+   * Asserts that this {@link FacetsCollector} can handle the given
+   * {@link FacetSearchParams}. Returns {@code null} if true, otherwise an error
+   * message.
+   */
+  final static boolean verifySearchParams(FacetSearchParams fsp) {
+    // verify that all category lists were encoded with DGapVInt
+    for (FacetRequest fr : fsp.facetRequests) {
+      CategoryListParams clp = fsp.indexingParams.getCategoryListParams(fr.categoryPath);
+      if (clp.createEncoder().createMatchingDecoder().getClass() != DGapVInt8IntDecoder.class) {
+        return false;
+      }
+    }
+    
+    return true;
+  }
+
+  @Override
+  public final void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) 
+      throws IOException {
+    assert clp.createEncoder().createMatchingDecoder().getClass() == DGapVInt8IntDecoder.class 
+        : "this aggregator assumes ordinals were encoded as dgap+vint";
+    
+    final BinaryDocValues dv = matchingDocs.context.reader().getBinaryDocValues(clp.field);
+    if (dv == null) { // this reader does not have DocValues for the requested category list
+      return;
+    }
+    
+    final int length = matchingDocs.bits.length();
+    final int[] counts = facetArrays.getIntArray();
+    int doc = 0;
+    while (doc < length && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+      dv.get(doc, buf);
+      if (buf.length > 0) {
+        // this document has facets
+        final int upto = buf.offset + buf.length;
+        int ord = 0;
+        int offset = buf.offset;
+        int prev = 0;
+        while (offset < upto) {
+          byte b = buf.bytes[offset++];
+          if (b >= 0) {
+            prev = ord = ((ord << 7) | b) + prev;
+            ++counts[ord];
+            ord = 0;
+          } else {
+            ord = (ord << 7) | (b & 0x7F);
+          }
+        }
+      }
+      ++doc;
+    }
+  }
+  
+  private int rollupCounts(int ordinal, int[] children, int[] siblings, int[] counts) {
+    int count = 0;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      int childCount = counts[ordinal];
+      childCount += rollupCounts(children[ordinal], children, siblings, counts);
+      counts[ordinal] = childCount;
+      count += childCount;
+      ordinal = siblings[ordinal];
+    }
+    return count;
+  }
+
+  @Override
+  public final void rollupValues(int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
+    final int[] counts = facetArrays.getIntArray();
+    counts[ordinal] += rollupCounts(children[ordinal], children, siblings, counts);
+  }
+
+  @Override
+  public final boolean requiresDocScores() {
+    return false;
+  }
+  
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FloatFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FloatFacetResultsHandler.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FloatFacetResultsHandler.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/FloatFacetResultsHandler.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,80 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.results.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.PriorityQueue;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link DepthOneFacetResultsHandler} which fills the categories values from
+ * {@link FacetArrays#getFloatArray()}.
+ * 
+ * @lucene.experimental
+ */
+public final class FloatFacetResultsHandler extends DepthOneFacetResultsHandler {
+
+  private final float[] values;
+  
+  public FloatFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
+    super(taxonomyReader, facetRequest, facetArrays);
+    this.values = facetArrays.getFloatArray();
+  }
+
+  @Override
+  protected final double valueOf(int ordinal) {
+    return values[ordinal];
+  }
+
+  
+  @Override
+  protected final int addSiblings(int ordinal, int[] siblings, PriorityQueue<FacetResultNode> pq) {
+    FacetResultNode top = pq.top();
+    int numResults = 0;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      float value = values[ordinal];
+      if (value > top.value) {
+        top.value = value;
+        top.ordinal = ordinal;
+        top = pq.updateTop();
+        ++numResults;
+      }
+      ordinal = siblings[ordinal];
+    }
+    return numResults;
+  }
+  
+  @Override
+  protected final void addSiblings(int ordinal, int[] siblings, ArrayList<FacetResultNode> nodes) throws IOException {
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      float value = values[ordinal];
+      if (value > 0) {
+        FacetResultNode node = new FacetResultNode();
+        node.label = taxonomyReader.getPath(ordinal);
+        node.value = value;
+        nodes.add(node);
+      }
+      ordinal = siblings[ordinal];
+    }
+  }
+  
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/IntFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/IntFacetResultsHandler.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/IntFacetResultsHandler.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/IntFacetResultsHandler.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,79 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+
+import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.results.FacetResultNode;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.PriorityQueue;
+
+/*
+ * 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.
+ */
+
+/**
+ * A {@link DepthOneFacetResultsHandler} which fills the categories values from
+ * {@link FacetArrays#getIntArray()}.
+ * 
+ * @lucene.experimental
+ */
+public final class IntFacetResultsHandler extends DepthOneFacetResultsHandler {
+  
+  private final int[] values;
+  
+  public IntFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
+    super(taxonomyReader, facetRequest, facetArrays);
+    this.values = facetArrays.getIntArray();
+  }
+  
+  @Override
+  protected final double valueOf(int ordinal) {
+    return values[ordinal];
+  }
+  
+  @Override
+  protected final int addSiblings(int ordinal, int[] siblings, PriorityQueue<FacetResultNode> pq) {
+    FacetResultNode top = pq.top();
+    int numResults = 0;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      int value = values[ordinal];
+      if (value > top.value) {
+        top.value = value;
+        top.ordinal = ordinal;
+        top = pq.updateTop();
+        ++numResults;
+      }
+      ordinal = siblings[ordinal];
+    }
+    return numResults;
+  }
+  
+  @Override
+  protected final void addSiblings(int ordinal, int[] siblings, ArrayList<FacetResultNode> nodes) throws IOException {
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      int value = values[ordinal];
+      if (value > 0) {
+        FacetResultNode node = new FacetResultNode();
+        node.label = taxonomyReader.getPath(ordinal);
+        node.value = value;
+        nodes.add(node);
+      }
+      ordinal = siblings[ordinal];
+    }
+  }
+  
+}

Added: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/MatchingDocsAsScoredDocIDs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/MatchingDocsAsScoredDocIDs.java?rev=1443736&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/MatchingDocsAsScoredDocIDs.java (added)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/MatchingDocsAsScoredDocIDs.java Thu Feb  7 21:32:27 2013
@@ -0,0 +1,169 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/*
+ * 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 {@link MatchingDocs} as {@link ScoredDocIDs}.
+ * 
+ * @lucene.experimental
+ */
+public class MatchingDocsAsScoredDocIDs implements ScoredDocIDs {
+
+  // TODO remove this class once we get rid of ScoredDocIDs 
+
+  final List<MatchingDocs> matchingDocs;
+  final int size;
+  
+  public MatchingDocsAsScoredDocIDs(List<MatchingDocs> matchingDocs) {
+    this.matchingDocs = matchingDocs;
+    int totalSize = 0;
+    for (MatchingDocs md : matchingDocs) {
+      totalSize += md.totalHits;
+    }
+    this.size = totalSize;
+  }
+  
+  @Override
+  public ScoredDocIDsIterator iterator() throws IOException {
+    return new ScoredDocIDsIterator() {
+      
+      final Iterator<MatchingDocs> mdIter = matchingDocs.iterator();
+      
+      int scoresIdx = 0;
+      int doc = 0;
+      MatchingDocs current;
+      int currentLength;
+      boolean done = false;
+      
+      @Override
+      public boolean next() {
+        if (done) {
+          return false;
+        }
+        
+        while (current == null) {
+          if (!mdIter.hasNext()) {
+            done = true;
+            return false;
+          }
+          current = mdIter.next();
+          currentLength = current.bits.length();
+          doc = 0;
+          scoresIdx = 0;
+          
+          if (doc >= currentLength || (doc = current.bits.nextSetBit(doc)) == -1) {
+            current = null;
+          } else {
+            doc = -1; // we're calling nextSetBit later on
+          }
+        }
+        
+        ++doc;
+        if (doc >= currentLength || (doc = current.bits.nextSetBit(doc)) == -1) {
+          current = null;
+          return next();
+        }
+        
+        return true;
+      }
+      
+      @Override
+      public float getScore() {
+        return current.scores == null ? ScoredDocIDsIterator.DEFAULT_SCORE : current.scores[scoresIdx++];
+      }
+      
+      @Override
+      public int getDocID() {
+        return done ? DocIdSetIterator.NO_MORE_DOCS : doc + current.context.docBase;
+      }
+    };
+  }
+
+  @Override
+  public DocIdSet getDocIDs() {
+    return new DocIdSet() {
+      
+      final Iterator<MatchingDocs> mdIter = matchingDocs.iterator();
+      int doc = 0;
+      MatchingDocs current;
+      int currentLength;
+      boolean done = false;
+      
+      @Override
+      public DocIdSetIterator iterator() throws IOException {
+        return new DocIdSetIterator() {
+          
+          @Override
+          public int nextDoc() throws IOException {
+            if (done) {
+              return DocIdSetIterator.NO_MORE_DOCS;
+            }
+            
+            while (current == null) {
+              if (!mdIter.hasNext()) {
+                done = true;
+                return DocIdSetIterator.NO_MORE_DOCS;
+              }
+              current = mdIter.next();
+              currentLength = current.bits.length();
+              doc = 0;
+              
+              if (doc >= currentLength || (doc = current.bits.nextSetBit(doc)) == -1) {
+                current = null;
+              } else {
+                doc = -1; // we're calling nextSetBit later on
+              }
+            }
+            
+            ++doc;
+            if (doc >= currentLength || (doc = current.bits.nextSetBit(doc)) == -1) {
+              current = null;
+              return nextDoc();
+            }
+            
+            return doc + current.context.docBase;
+          }
+          
+          @Override
+          public int docID() {
+            return doc + current.context.docBase;
+          }
+          
+          @Override
+          public int advance(int target) throws IOException {
+            throw new UnsupportedOperationException("not supported");
+          }
+        };
+      }
+    };
+  }
+
+  @Override
+  public int size() {
+    return size;
+  }
+  
+}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/SamplingWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/SamplingWrapper.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/SamplingWrapper.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/SamplingWrapper.java Thu Feb  7 21:32:27 2013
@@ -4,6 +4,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.facet.partitions.search.PartitionsFacetResultsHandler;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.sampling.Sampler;
@@ -35,12 +36,12 @@ import org.apache.lucene.facet.search.sa
  * 
  * @lucene.experimental
  */
-public class SamplingWrapper extends FacetsAccumulator {
+public class SamplingWrapper extends StandardFacetsAccumulator {
 
-  private FacetsAccumulator delegee;
+  private StandardFacetsAccumulator delegee;
   private Sampler sampler;
 
-  public SamplingWrapper(FacetsAccumulator delegee, Sampler sampler) {
+  public SamplingWrapper(StandardFacetsAccumulator delegee, Sampler sampler) {
     super(delegee.searchParams, delegee.indexReader, delegee.taxonomyReader);
     this.delegee = delegee;
     this.sampler = sampler;
@@ -48,11 +49,6 @@ public class SamplingWrapper extends Fac
 
   @Override
   public List<FacetResult> accumulate(ScoredDocIDs docids) throws IOException {
-    // first let delegee accumulate without labeling at all (though
-    // currently it doesn't matter because we have to label all returned anyhow)
-    boolean origAllowLabeling = isAllowLabeling();
-    setAllowLabeling(false);
-
     // Replacing the original searchParams with the over-sampled (and without statistics-compute)
     FacetSearchParams original = delegee.searchParams;
     delegee.searchParams = sampler.overSampledSearchParams(original);
@@ -60,24 +56,20 @@ public class SamplingWrapper extends Fac
     SampleResult sampleSet = sampler.getSampleSet(docids);
 
     List<FacetResult> sampleRes = delegee.accumulate(sampleSet.docids);
-    setAllowLabeling(origAllowLabeling);
 
     List<FacetResult> fixedRes = new ArrayList<FacetResult>();
     for (FacetResult fres : sampleRes) {
       // for sure fres is not null because this is guaranteed by the delegee.
-      FacetResultsHandler frh = fres.getFacetRequest().createFacetResultsHandler(taxonomyReader);
+      PartitionsFacetResultsHandler frh = createFacetResultsHandler(fres.getFacetRequest());
       // fix the result of current request
-      sampler.getSampleFixer(indexReader, taxonomyReader, searchParams)
-          .fixResult(docids, fres); 
+      sampler.getSampleFixer(indexReader, taxonomyReader, searchParams).fixResult(docids, fres); 
       fres = frh.rearrangeFacetResult(fres); // let delegee's handler do any
       
       // Using the sampler to trim the extra (over-sampled) results
       fres = sampler.trimResult(fres);
       
       // final labeling if allowed (because labeling is a costly operation)
-      if (isAllowLabeling()) {
-        frh.labelResult(fres);
-      }
+      frh.labelResult(fres);
       fixedRes.add(fres); // add to final results
     }
 
@@ -96,14 +88,4 @@ public class SamplingWrapper extends Fac
     delegee.setComplementThreshold(complementThreshold);
   }
 
-  @Override
-  protected boolean isAllowLabeling() {
-    return delegee.isAllowLabeling();
-  }
-
-  @Override
-  protected void setAllowLabeling(boolean allowLabeling) {
-    delegee.setAllowLabeling(allowLabeling);
-  }
-
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIDs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIDs.java?rev=1443736&r1=1443735&r2=1443736&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIDs.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIDs.java Thu Feb  7 21:32:27 2013
@@ -25,7 +25,7 @@ import org.apache.lucene.search.DocIdSet
  * Document IDs with scores for each, driving facets accumulation. Document
  * scores are optionally used in the process of facets scoring.
  * 
- * @see FacetsAccumulator#accumulate(ScoredDocIDs)
+ * @see StandardFacetsAccumulator#accumulate(ScoredDocIDs)
  * @lucene.experimental
  */
 public interface ScoredDocIDs {