You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC

svn commit: r1512909 [16/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-too...

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java Sun Aug 11 12:19:13 2013
@@ -1,5 +1,17 @@
 package org.apache.lucene.facet.search;
 
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.CollectionUtil;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -24,6 +36,143 @@ package org.apache.lucene.facet.search;
  */
 public class FacetResult {
   
+  private static FacetResultNode addIfNotExist(Map<CategoryPath, FacetResultNode> nodes, FacetResultNode node) {
+    FacetResultNode n = nodes.get(node.label);
+    if (n == null) {
+      nodes.put(node.label, node);
+      n = node;
+    }
+    return n;
+  }
+
+  /**
+   * A utility for merging multiple {@link FacetResult} of the same
+   * (hierarchical) dimension into a single {@link FacetResult}, to reconstruct
+   * the hierarchy. The results are merged according to the following rules:
+   * <ul>
+   * <li>If two results share the same dimension (first component in their
+   * {@link CategoryPath}), they are merged.
+   * <li>If a result is missing ancestors in the other results, e.g. A/B/C but
+   * no corresponding A or A/B, these nodes are 'filled' with their label,
+   * ordinal and value (obtained from the respective {@link FacetArrays}).
+   * <li>If a result does not share a dimension with other results, it is
+   * returned as is.
+   * </ul>
+   * <p>
+   * <b>NOTE:</b> the returned results are not guaranteed to be in the same
+   * order of the input ones.
+   * 
+   * @param results
+   *          the results to merge
+   * @param taxoReader
+   *          the {@link TaxonomyReader} to use when creating missing ancestor
+   *          nodes
+   * @param dimArrays
+   *          a mapping from a dimension to the respective {@link FacetArrays}
+   *          from which to pull the nodes values
+   */
+  public static List<FacetResult> mergeHierarchies(List<FacetResult> results, TaxonomyReader taxoReader,
+      Map<String, FacetArrays> dimArrays) throws IOException {
+    final Map<String, List<FacetResult>> dims = new HashMap<String,List<FacetResult>>();
+    for (FacetResult fr : results) {
+      String dim = fr.getFacetRequest().categoryPath.components[0];
+      List<FacetResult> frs = dims.get(dim);
+      if (frs == null) {
+        frs = new ArrayList<FacetResult>();
+        dims.put(dim, frs);
+      }
+      frs.add(fr);
+    }
+
+    final List<FacetResult> res = new ArrayList<FacetResult>();
+    for (List<FacetResult> frs : dims.values()) {
+      FacetResult mergedResult = frs.get(0);
+      if (frs.size() > 1) {
+        CollectionUtil.introSort(frs, new Comparator<FacetResult>() {
+          @Override
+          public int compare(FacetResult fr1, FacetResult fr2) {
+            return fr1.getFacetRequest().categoryPath.compareTo(fr2.getFacetRequest().categoryPath);
+          }
+        });
+        Map<CategoryPath, FacetResultNode> mergedNodes = new HashMap<CategoryPath,FacetResultNode>();
+        FacetArrays arrays = dimArrays != null ? dimArrays.get(frs.get(0).getFacetRequest().categoryPath.components[0]) : null;
+        for (FacetResult fr : frs) {
+          FacetRequest freq = fr.getFacetRequest();
+          OrdinalValueResolver resolver = null;
+          if (arrays != null) {
+            resolver = freq.createFacetsAggregator(FacetIndexingParams.DEFAULT).createOrdinalValueResolver(freq, arrays);
+          }
+          FacetResultNode frn = fr.getFacetResultNode();
+          FacetResultNode merged = mergedNodes.get(frn.label);
+          if (merged == null) {
+            CategoryPath parent = frn.label.subpath(frn.label.length - 1);
+            FacetResultNode childNode = frn;
+            FacetResultNode parentNode = null;
+            while (parent.length > 0 && (parentNode = mergedNodes.get(parent)) == null) {
+              int parentOrd = taxoReader.getOrdinal(parent);
+              double parentValue = -1;
+              if (arrays != null) {
+                parentValue = resolver.valueOf(parentOrd);
+              }
+              parentNode = new FacetResultNode(parentOrd, parentValue);
+              parentNode.label = parent;
+              parentNode.subResults = new ArrayList<FacetResultNode>();
+              parentNode.subResults.add(childNode);
+              mergedNodes.put(parent, parentNode);
+              childNode = parentNode;
+              parent = parent.subpath(parent.length - 1);
+            }
+
+            // at least one parent was added, so link the final (existing)
+            // parent with the child
+            if (parent.length > 0) {
+              if (!(parentNode.subResults instanceof ArrayList)) {
+                parentNode.subResults = new ArrayList<FacetResultNode>(parentNode.subResults);
+              }
+              parentNode.subResults.add(childNode);
+            }
+
+            // for missing FRNs, add new ones with label and value=-1
+            // first time encountered this label, add it and all its children to
+            // the map.
+            mergedNodes.put(frn.label, frn);
+            for (FacetResultNode child : frn.subResults) {
+              addIfNotExist(mergedNodes, child);
+            }
+          } else {
+            if (!(merged.subResults instanceof ArrayList)) {
+              merged.subResults = new ArrayList<FacetResultNode>(merged.subResults);
+            }
+            for (FacetResultNode sub : frn.subResults) {
+              // make sure sub wasn't already added
+              sub = addIfNotExist(mergedNodes, sub);
+              if (!merged.subResults.contains(sub)) {
+                merged.subResults.add(sub);
+              }
+            }
+          }
+        }
+        
+        // find the 'first' node to put on the FacetResult root
+        CategoryPath min = null;
+        for (CategoryPath cp : mergedNodes.keySet()) {
+          if (min == null || cp.compareTo(min) < 0) {
+            min = cp;
+          }
+        }
+        FacetRequest dummy = new FacetRequest(min, frs.get(0).getFacetRequest().numResults) {
+          @Override
+          public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
+            throw new UnsupportedOperationException("not supported by this request");
+          }
+        };
+        mergedResult = new FacetResult(dummy, mergedNodes.get(min), -1);
+      }
+      res.add(mergedResult);
+    }
+    return res;
+  }
+
   private final FacetRequest facetRequest;
   private final FacetResultNode rootNode;
   private final int numValidDescendants;

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java Sun Aug 11 12:19:13 2013
@@ -28,10 +28,13 @@ import org.apache.lucene.facet.taxonomy.
  * Result of faceted search for a certain taxonomy node. This class serves as a
  * bin of different attributes of the result node, such as its {@link #ordinal}
  * as well as {@link #label}. You are not expected to modify those values.
+ * <p>
+ * This class implements {@link Comparable} for easy comparisons of result
+ * nodes, e.g. when sorting or computing top-K nodes.
  * 
  * @lucene.experimental
  */
-public class FacetResultNode {
+public class FacetResultNode implements Comparable<FacetResultNode> {
 
   public static final List<FacetResultNode> EMPTY_SUB_RESULTS = Collections.emptyList();
   
@@ -71,6 +74,15 @@ public class FacetResultNode {
     this.ordinal = ordinal;
     this.value = value;
   }
+
+  @Override
+  public int compareTo(FacetResultNode o) {
+    int res = Double.compare(value, o.value);
+    if (res == 0) {
+      res = ordinal - o.ordinal;
+    }
+    return res;
+  }
   
   @Override
   public String toString() {

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java Sun Aug 11 12:19:13 2013
@@ -29,15 +29,17 @@ import org.apache.lucene.facet.taxonomy.
 public abstract class FacetResultsHandler {
 
   public final TaxonomyReader taxonomyReader;
-
   public final FacetRequest facetRequest;
   
+  protected final OrdinalValueResolver resolver;
   protected final FacetArrays facetArrays;
 
-  public FacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
+  public FacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, OrdinalValueResolver resolver, 
+      FacetArrays facetArrays) {
     this.taxonomyReader = taxonomyReader;
     this.facetRequest = facetRequest;
     this.facetArrays = facetArrays;
+    this.resolver = resolver;
   }
 
   /** Computes the {@link FacetResult} for the given {@link FacetArrays}. */

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java Sun Aug 11 12:19:13 2013
@@ -2,20 +2,16 @@ 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.facet.encoding.DGapVInt8IntDecoder;
-import org.apache.lucene.facet.params.CategoryListParams;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
+import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.params.CategoryListParams.OrdinalPolicy;
-import org.apache.lucene.facet.search.FacetRequest.FacetArraysSource;
-import org.apache.lucene.facet.search.FacetRequest.ResultMode;
-import org.apache.lucene.facet.search.FacetRequest.SortOrder;
+import org.apache.lucene.facet.range.RangeAccumulator;
+import org.apache.lucene.facet.range.RangeFacetRequest;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
-import org.apache.lucene.facet.taxonomy.ParallelTaxonomyArrays;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesAccumulator;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.IndexReader;
 
@@ -37,48 +33,117 @@ import org.apache.lucene.index.IndexRead
  */
 
 /**
- * Driver for Accumulating facets of faceted search requests over given
- * documents.
+ * Accumulates the facets defined in the {@link FacetSearchParams}.
  * 
  * @lucene.experimental
  */
-public class FacetsAccumulator {
+public abstract class FacetsAccumulator {
 
-  public final TaxonomyReader taxonomyReader;
-  public final IndexReader indexReader;
-  public final FacetArrays facetArrays;
-  public FacetSearchParams searchParams;
+  // TODO this should be final, but currently SamplingAccumulator modifies the params.
+  // need to review the class and if it's resolved, make it final
+  public /*final*/ FacetSearchParams searchParams;
+
+  /** Constructor with the given search params. */
+  protected FacetsAccumulator(FacetSearchParams fsp) {
+    this.searchParams = fsp;
+  }
 
   /**
-   * 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.
+   * Creates a {@link FacetsAccumulator} for the given facet requests. This
+   * method supports {@link RangeAccumulator} and
+   * {@link TaxonomyFacetsAccumulator} by dividing the facet requests into
+   * {@link RangeFacetRequest} and the rest.
+   * <p>
+   * If both types of facet requests are used, it returns a
+   * {@link MultiFacetsAccumulator} and the facet results returned from
+   * {@link #accumulate(List)} may not be in the same order as the given facet
+   * requests.
+   * 
+   * @param fsp
+   *          the search params define the facet requests and the
+   *          {@link FacetIndexingParams}
+   * @param indexReader
+   *          the {@link IndexReader} used for search
+   * @param taxoReader
+   *          the {@link TaxonomyReader} used for search
+   * @param arrays
+   *          the {@link FacetArrays} which the accumulator should use to store
+   *          the categories weights in. Can be {@code null}.
    */
-  public FacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, TaxonomyReader taxonomyReader) {
-    this(searchParams, indexReader, taxonomyReader, new FacetArrays(taxonomyReader.getSize()));
-  }
+  public static FacetsAccumulator create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader, 
+      FacetArrays arrays) {
+    if (fsp.indexingParams.getPartitionSize() != Integer.MAX_VALUE) {
+      return new OldFacetsAccumulator(fsp, indexReader, taxoReader, arrays);
+    }
+    
+    List<FacetRequest> rangeRequests = new ArrayList<FacetRequest>();
+    List<FacetRequest> nonRangeRequests = new ArrayList<FacetRequest>();
+    for (FacetRequest fr : fsp.facetRequests) {
+      if (fr instanceof RangeFacetRequest) {
+        rangeRequests.add(fr);
+      } else {
+        nonRangeRequests.add(fr);
+      }
+    }
 
+    if (rangeRequests.isEmpty()) {
+      return new TaxonomyFacetsAccumulator(fsp, indexReader, taxoReader, arrays);
+    } else if (nonRangeRequests.isEmpty()) {
+      return new RangeAccumulator(rangeRequests);
+    } else {
+      FacetSearchParams searchParams = new FacetSearchParams(fsp.indexingParams, nonRangeRequests);
+      FacetsAccumulator accumulator = new TaxonomyFacetsAccumulator(searchParams, indexReader, taxoReader, arrays);
+      RangeAccumulator rangeAccumulator = new RangeAccumulator(rangeRequests);
+      return MultiFacetsAccumulator.wrap(accumulator, rangeAccumulator);
+    }
+  }
+  
   /**
-   * Creates an appropriate {@link FacetsAccumulator},
-   * returning {@link FacetsAccumulator} when all requests
-   * are {@link CountFacetRequest} and only one partition is
-   * in use, otherwise {@link StandardFacetsAccumulator}.
+   * Creates a {@link FacetsAccumulator} for the given facet requests. This
+   * method supports {@link RangeAccumulator} and
+   * {@link SortedSetDocValuesAccumulator} by dividing the facet requests into
+   * {@link RangeFacetRequest} and the rest.
+   * <p>
+   * If both types of facet requests are used, it returns a
+   * {@link MultiFacetsAccumulator} and the facet results returned from
+   * {@link #accumulate(List)} may not be in the same order as the given facet
+   * requests.
+   * 
+   * @param fsp
+   *          the search params define the facet requests and the
+   *          {@link FacetIndexingParams}
+   * @param state
+   *          the {@link SortedSetDocValuesReaderState} needed for accumulating
+   *          the categories
+   * @param arrays
+   *          the {@link FacetArrays} which the accumulator should use to
+   *          store the categories weights in. Can be {@code null}.
    */
-  public static FacetsAccumulator create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
+  public static FacetsAccumulator create(FacetSearchParams fsp, SortedSetDocValuesReaderState state, FacetArrays arrays) throws IOException {
     if (fsp.indexingParams.getPartitionSize() != Integer.MAX_VALUE) {
-      return new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
+      throw new IllegalArgumentException("only default partition size is supported by this method: " + fsp.indexingParams.getPartitionSize());
     }
     
+    List<FacetRequest> rangeRequests = new ArrayList<FacetRequest>();
+    List<FacetRequest> nonRangeRequests = new ArrayList<FacetRequest>();
     for (FacetRequest fr : fsp.facetRequests) {
-      if (!(fr instanceof CountFacetRequest)) {
-        return new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
+      if (fr instanceof RangeFacetRequest) {
+        rangeRequests.add(fr);
+      } else {
+        nonRangeRequests.add(fr);
       }
     }
     
-    return new FacetsAccumulator(fsp, indexReader, taxoReader);
+    if (rangeRequests.isEmpty()) {
+      return new SortedSetDocValuesAccumulator(state, fsp, arrays);
+    } else if (nonRangeRequests.isEmpty()) {
+      return new RangeAccumulator(rangeRequests);
+    } else {
+      FacetSearchParams searchParams = new FacetSearchParams(fsp.indexingParams, nonRangeRequests);
+      FacetsAccumulator accumulator = new SortedSetDocValuesAccumulator(state, searchParams, arrays);
+      RangeAccumulator rangeAccumulator = new RangeAccumulator(rangeRequests);
+      return MultiFacetsAccumulator.wrap(accumulator, rangeAccumulator);
+    }
   }
   
   /** Returns an empty {@link FacetResult}. */
@@ -89,69 +154,6 @@ public class FacetsAccumulator {
   }
   
   /**
-   * 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) {
-    this.facetArrays = facetArrays;
-    this.indexReader = indexReader;
-    this.taxonomyReader = taxonomyReader;
-    this.searchParams = searchParams;
-  }
-  
-  /**
-   * 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();
-    }
-  }
-  
-  /**
-   * Creates a {@link FacetResultsHandler} that matches the given
-   * {@link FacetRequest}.
-   */
-  protected FacetResultsHandler createFacetResultsHandler(FacetRequest fr) {
-    if (fr.getDepth() == 1 && fr.getSortOrder() == SortOrder.DESCENDING) {
-      FacetArraysSource fas = fr.getFacetArraysSource();
-      if (fas == FacetArraysSource.INT) {
-        return new IntFacetResultsHandler(taxonomyReader, fr, facetArrays);
-      }
-      
-      if (fas == FacetArraysSource.FLOAT) {
-        return new FloatFacetResultsHandler(taxonomyReader, fr, facetArrays);
-      }
-    }
-
-    if (fr.getResultMode() == ResultMode.PER_NODE_IN_TREE) {
-      return new TopKInEachNodeHandler(taxonomyReader, fr, facetArrays);
-    } 
-    return new TopKFacetResultsHandler(taxonomyReader, fr, facetArrays);
-  }
-
-  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;
-  }
-
-  /**
    * 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.
@@ -159,44 +161,12 @@ public class FacetsAccumulator {
    * @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);
-      }
-    }
-    
-    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
-        // Add empty FacetResult
-        res.add(emptyResult(rootOrd, fr));
-        continue;
-      }
-      CategoryListParams clp = searchParams.indexingParams.getCategoryListParams(fr.categoryPath);
-      if (fr.categoryPath.length > 0) { // someone might ask to aggregate the ROOT category
-        OrdinalPolicy ordinalPolicy = clp.getOrdinalPolicy(fr.categoryPath.components[0]);
-        if (ordinalPolicy == OrdinalPolicy.NO_PARENTS) {
-          // rollup values
-          aggregator.rollupValues(fr, rootOrd, children, siblings, facetArrays);
-        }
-      }
-      
-      FacetResultsHandler frh = createFacetResultsHandler(fr);
-      res.add(frh.compute());
-    }
-    return res;
-  }
+  public abstract List<FacetResult> accumulate(List<MatchingDocs> matchingDocs) throws IOException;
 
-  public boolean requiresDocScores() {
-    return getAggregator().requiresDocScores();
-  }
+  /**
+   * Used by {@link FacetsCollector} to determine if document scores need to be
+   * collected in addition to matching documents.
+   */
+  public abstract boolean requiresDocScores();
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java Sun Aug 11 12:19:13 2013
@@ -46,4 +46,11 @@ public interface FacetsAggregator {
   /** Returns {@code true} if this aggregator requires document scores. */
   public boolean requiresDocScores();
   
+  /**
+   * Creates the appropriate {@link OrdinalValueResolver} for this aggregator
+   * and the given {@link FacetRequest}. The request is passed so that compound
+   * aggregators can return the correct {@link OrdinalValueResolver}.
+   */
+  public OrdinalValueResolver createOrdinalValueResolver(FacetRequest facetRequest, FacetArrays arrays);
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java Sun Aug 11 12:19:13 2013
@@ -167,7 +167,7 @@ public abstract class FacetsCollector ex
    * FacetsAccumulator} from {@link FacetsAccumulator#create}.
    */
   public static FacetsCollector create(FacetSearchParams fsp, IndexReader indexReader, TaxonomyReader taxoReader) {
-    return create(FacetsAccumulator.create(fsp, indexReader, taxoReader));
+    return create(FacetsAccumulator.create(fsp, indexReader, taxoReader, null));
   }
 
   /**

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java Sun Aug 11 12:19:13 2013
@@ -5,7 +5,6 @@ import java.io.IOException;
 import org.apache.lucene.facet.encoding.DGapVInt8IntDecoder;
 import org.apache.lucene.facet.encoding.DGapVInt8IntEncoder;
 import org.apache.lucene.facet.params.CategoryListParams;
-import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.util.BytesRef;
@@ -40,23 +39,6 @@ public final class FastCountingFacetsAgg
   
   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 {

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java Sun Aug 11 12:19:13 2013
@@ -4,6 +4,7 @@ import java.io.IOException;
 
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.search.OrdinalValueResolver.IntValueResolver;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
@@ -60,4 +61,9 @@ public abstract class IntRollupFacetsAgg
     return false;
   }
   
+  @Override
+  public OrdinalValueResolver createOrdinalValueResolver(FacetRequest facetRequest, FacetArrays arrays) {
+    return new IntValueResolver(arrays);
+  }
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java Sun Aug 11 12:19:13 2013
@@ -27,6 +27,9 @@ import org.apache.lucene.facet.search.Fa
 /**
  * A {@link FacetsAggregator} which invokes the proper aggregator per
  * {@link CategoryListParams}.
+ * {@link #rollupValues(FacetRequest, int, int[], int[], FacetArrays)} is
+ * delegated to the proper aggregator which handles the
+ * {@link CategoryListParams} the given {@link FacetRequest} belongs to.
  */
 public class PerCategoryListAggregator implements FacetsAggregator {
   
@@ -58,5 +61,11 @@ public class PerCategoryListAggregator i
     }
     return false;
   }
+
+  @Override
+  public OrdinalValueResolver createOrdinalValueResolver(FacetRequest facetRequest, FacetArrays arrays) {
+    CategoryListParams clp = fip.getCategoryListParams(facetRequest.categoryPath);
+    return aggregators.get(clp).createOrdinalValueResolver(facetRequest, arrays);
+  }
   
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java Sun Aug 11 12:19:13 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.facet.search;
 
+import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -34,19 +34,8 @@ public class SumScoreFacetRequest extend
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) {
-    assert !useComplements : "complements are not supported by this FacetRequest";
-    return new ScoringAggregator(arrays.getFloatArray());
+  public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
+    return new SumScoreFacetsAggregator();
   }
-
-  @Override
-  public double getValueOf(FacetArrays arrays, int ordinal) {
-    return arrays.getFloatArray()[ordinal];
-  }
-
-  @Override
-  public FacetArraysSource getFacetArraysSource() {
-    return FacetArraysSource.FLOAT;
-  }
-
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java Sun Aug 11 12:19:13 2013
@@ -4,6 +4,7 @@ import java.io.IOException;
 
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
+import org.apache.lucene.facet.search.OrdinalValueResolver.FloatValueResolver;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.util.IntsRef;
 
@@ -76,5 +77,10 @@ public class SumScoreFacetsAggregator im
   public boolean requiresDocScores() {
     return true;
   }
+
+  @Override
+  public OrdinalValueResolver createOrdinalValueResolver(FacetRequest facetRequest, FacetArrays arrays) {
+    return new FloatValueResolver(arrays);
+  }
   
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java Sun Aug 11 12:19:13 2013
@@ -34,16 +34,10 @@ import org.apache.lucene.facet.util.Resu
  */
 public class TopKFacetResultsHandler extends PartitionsFacetResultsHandler {
   
-  /**
-   * Construct top-K results handler.
-   * 
-   * @param taxonomyReader
-   *          taxonomy reader
-   * @param facetRequest
-   *          facet request being served
-   */
-  public TopKFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
-    super(taxonomyReader, facetRequest, facetArrays);
+  /** Construct top-K results handler. */
+  public TopKFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, 
+      OrdinalValueResolver resolver, FacetArrays facetArrays) {
+    super(taxonomyReader, facetRequest, resolver, facetArrays);
   }
   
   // fetch top K for specific partition. 
@@ -56,7 +50,7 @@ public class TopKFacetResultsHandler ext
       double value = 0;  
       if (isSelfPartition(ordinal, facetArrays, offset)) {
         int partitionSize = facetArrays.arrayLength;
-        value = facetRequest.getValueOf(facetArrays, ordinal % partitionSize);
+        value = resolver.valueOf(ordinal % partitionSize);
       }
       
       FacetResultNode parentResultNode = new FacetResultNode(ordinal, value);
@@ -158,7 +152,7 @@ public class TopKFacetResultsHandler ext
       // collect it, if belongs to current partition, and then push its kids on itself, if applicable
       if (tosOrdinal >= offset) { // tosOrdinal resides in current partition
         int relativeOrdinal = tosOrdinal % partitionSize;
-        double value = facetRequest.getValueOf(facetArrays, relativeOrdinal);
+        double value = resolver.valueOf(relativeOrdinal);
         if (value != 0 && !Double.isNaN(value)) {
           // Count current ordinal -- the TOS
           if (reusable == null) {

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java Sun Aug 11 12:19:13 2013
@@ -62,8 +62,9 @@ import org.apache.lucene.util.PriorityQu
  */
 public class TopKInEachNodeHandler extends PartitionsFacetResultsHandler {
 
-  public TopKInEachNodeHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
-    super(taxonomyReader, facetRequest, facetArrays);
+  public TopKInEachNodeHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, OrdinalValueResolver resolver, 
+      FacetArrays facetArrays) {
+    super(taxonomyReader, facetRequest, resolver, facetArrays);
   }
 
   /**
@@ -112,8 +113,8 @@ public class TopKInEachNodeHandler exten
     // this will grow into the returned IntermediateFacetResult
     IntToObjectMap<AACO> AACOsOfOnePartition = new IntToObjectMap<AACO>();
 
-    int partitionSize = facetArrays.arrayLength; // all partitions, except, possibly, the last,
-    // have the same length. Hence modulo is OK.
+    // all partitions, except, possibly, the last, have the same length. Hence modulo is OK.
+    int partitionSize = facetArrays.arrayLength;
 
     int depth = facetRequest.getDepth();
 
@@ -123,7 +124,7 @@ public class TopKInEachNodeHandler exten
           facetRequest, AACOsOfOnePartition);
       if (isSelfPartition(rootNode, facetArrays, offset)) {
         tempFRWH.isRootNodeIncluded = true;
-        tempFRWH.rootNodeValue = this.facetRequest.getValueOf(facetArrays, rootNode % partitionSize);
+        tempFRWH.rootNodeValue = resolver.valueOf(rootNode % partitionSize);
       }
       return tempFRWH;
     }
@@ -267,7 +268,7 @@ public class TopKInEachNodeHandler exten
 
         while (tosOrdinal >= offset) { // while tosOrdinal belongs to the given partition; here, too, we use the fact
           // that TaxonomyReader.INVALID_ORDINAL == -1 < offset
-          double value = facetRequest.getValueOf(facetArrays, tosOrdinal % partitionSize);
+          double value = resolver.valueOf(tosOrdinal % partitionSize);
           if (value != 0) { // the value of yc is not 0, it is to be considered.  
             totalNumOfDescendantsConsidered++;
 
@@ -338,7 +339,7 @@ public class TopKInEachNodeHandler exten
         facetRequest, AACOsOfOnePartition);
     if (isSelfPartition(rootNode, facetArrays, offset)) {
       tempFRWH.isRootNodeIncluded = true;
-      tempFRWH.rootNodeValue = this.facetRequest.getValueOf(facetArrays, rootNode % partitionSize);
+      tempFRWH.rootNodeValue = resolver.valueOf(rootNode % partitionSize);
     }
     tempFRWH.totalNumOfFacetsConsidered = totalNumOfDescendantsConsidered;
     return tempFRWH;
@@ -374,7 +375,7 @@ public class TopKInEachNodeHandler exten
     int ret = 0;
     if (offset <= ordinal) {
       // ordinal belongs to the current partition
-      if (0 != facetRequest.getValueOf(facetArrays, ordinal % partitionSize)) {
+      if (0 != resolver.valueOf(ordinal % partitionSize)) {
         ret++;
       }
     }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesAccumulator.java Sun Aug 11 12:19:13 2013
@@ -32,17 +32,19 @@ import org.apache.lucene.facet.search.Fa
 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.search.TaxonomyFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.index.AtomicReader;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
 import org.apache.lucene.index.MultiDocValues;
+import org.apache.lucene.index.MultiDocValues.MultiSortedSetDocValues;
+import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 
-/** A {@link FacetsAccumulator} that uses previously
+/** A {@link TaxonomyFacetsAccumulator} that uses previously
  *  indexed {@link SortedSetDocValuesFacetFields} to perform faceting,
  *  without require a separate taxonomy index.  Faceting is
  *  a bit slower (~25%), and there is added cost on every
@@ -55,25 +57,34 @@ public class SortedSetDocValuesAccumulat
   final SortedSetDocValuesReaderState state;
   final SortedSetDocValues dv;
   final String field;
-
-  public SortedSetDocValuesAccumulator(FacetSearchParams fsp, SortedSetDocValuesReaderState state) throws IOException {
-    super(fsp, null, null, new FacetArrays((int) state.getDocValues().getValueCount()));
+  final FacetArrays facetArrays;
+  
+  /** Constructor with the given facet search params. */
+  public SortedSetDocValuesAccumulator(SortedSetDocValuesReaderState state, FacetSearchParams fsp) 
+      throws IOException {
+    this(state, fsp, null);
+  }
+  
+  public SortedSetDocValuesAccumulator(SortedSetDocValuesReaderState state, FacetSearchParams fsp, FacetArrays arrays) 
+      throws IOException {
+    super(fsp);
     this.state = state;
     this.field = state.getField();
+    this.facetArrays = arrays == null ? new FacetArrays(state.getSize()) : arrays;
     dv = state.getDocValues();
 
     // Check params:
-    for(FacetRequest request : fsp.facetRequests) {
-      if (!(request instanceof CountFacetRequest)) {
-        throw new IllegalArgumentException("this collector only supports CountFacetRequest; got " + request);
+    for (FacetRequest fr : fsp.facetRequests) {
+      if (!(fr instanceof CountFacetRequest)) {
+        throw new IllegalArgumentException("this accumulator only supports CountFacetRequest; got " + fr);
       }
-      if (request.categoryPath.length != 1) {
-        throw new IllegalArgumentException("this collector only supports depth 1 CategoryPath; got " + request.categoryPath);
+      if (fr.categoryPath.length != 1) {
+        throw new IllegalArgumentException("this accumulator only supports 1-level CategoryPath; got " + fr.categoryPath);
       }
-      if (request.getDepth() != 1) {
-        throw new IllegalArgumentException("this collector only supports depth=1; got " + request.getDepth());
+      if (fr.getDepth() != 1) {
+        throw new IllegalArgumentException("this accumulator only supports depth=1; got " + fr.getDepth());
       }
-      String dim = request.categoryPath.components[0];
+      String dim = fr.categoryPath.components[0];
 
       SortedSetDocValuesReaderState.OrdRange ordRange = state.getOrdRange(dim);
       if (ordRange == null) {
@@ -82,121 +93,123 @@ public class SortedSetDocValuesAccumulat
     }
   }
 
-  @Override
-  public FacetsAggregator getAggregator() {
+  /** Keeps highest count results. */
+  static class TopCountPQ extends PriorityQueue<FacetResultNode> {
+    public TopCountPQ(int topN) {
+      super(topN, false);
+    }
+
+    @Override
+    protected boolean lessThan(FacetResultNode a, FacetResultNode b) {
+      if (a.value < b.value) {
+        return true;
+      } else if (a.value > b.value) {
+        return false;
+      } else {
+        return a.ordinal > b.ordinal;
+      }
+    }
+  }
 
-    return new FacetsAggregator() {
+  static class SortedSetAggregator {
 
-      @Override
-      public void aggregate(MatchingDocs matchingDocs, CategoryListParams clp, FacetArrays facetArrays) throws IOException {
+    private final SortedSetDocValuesReaderState state;
+    private final String field;
+    private final SortedSetDocValues dv;
+    
+    public SortedSetAggregator(String field, SortedSetDocValuesReaderState state, SortedSetDocValues dv) {
+      this.field = field;
+      this.state = state;
+      this.dv = dv;
+    }
+    
+    public void aggregate(MatchingDocs matchingDocs, FacetArrays facetArrays) throws IOException {
 
-        SortedSetDocValues segValues = matchingDocs.context.reader().getSortedSetDocValues(field);
-        if (segValues == null) {
-          return;
-        }
+      AtomicReader reader = matchingDocs.context.reader();
 
-        final int[] counts = facetArrays.getIntArray();
-        final int maxDoc = matchingDocs.context.reader().maxDoc();
-        assert maxDoc == matchingDocs.bits.length();
-
-        if (dv instanceof MultiSortedSetDocValues) {
-          MultiDocValues.OrdinalMap ordinalMap = ((MultiSortedSetDocValues) dv).mapping;
-          int segOrd = matchingDocs.context.ord;
-
-          int numSegOrds = (int) segValues.getValueCount();
-
-          if (matchingDocs.totalHits < numSegOrds/10) {
-            // Remap every ord to global ord as we iterate:
-            int doc = 0;
-            while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
-              segValues.setDocument(doc);
-              int term = (int) segValues.nextOrd();
-              while (term != SortedSetDocValues.NO_MORE_ORDS) {
-                counts[(int) ordinalMap.getGlobalOrd(segOrd, term)]++;
-                term = (int) segValues.nextOrd();
-              }
-              ++doc;
-            }
-          } else {
+      // LUCENE-5090: make sure the provided reader context "matches"
+      // the top-level reader passed to the
+      // SortedSetDocValuesReaderState, else cryptic
+      // AIOOBE can happen:
+      if (ReaderUtil.getTopLevelContext(matchingDocs.context).reader() != state.origReader) {
+        throw new IllegalStateException("the SortedSetDocValuesReaderState provided to this class does not match the reader being searched; you must create a new SortedSetDocValuesReaderState every time you open a new IndexReader");
+      }
+      
+      SortedSetDocValues segValues = reader.getSortedSetDocValues(field);
+      if (segValues == null) {
+        return;
+      }
 
-            // First count in seg-ord space:
-            final int[] segCounts = new int[numSegOrds];
-            int doc = 0;
-            while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
-              segValues.setDocument(doc);
-              int term = (int) segValues.nextOrd();
-              while (term != SortedSetDocValues.NO_MORE_ORDS) {
-                segCounts[term]++;
-                term = (int) segValues.nextOrd();
-              }
-              ++doc;
-            }
+      final int[] counts = facetArrays.getIntArray();
+      final int maxDoc = reader.maxDoc();
+      assert maxDoc == matchingDocs.bits.length();
 
-            // Then, migrate to global ords:
-            for(int ord=0;ord<numSegOrds;ord++) {
-              int count = segCounts[ord];
-              if (count != 0) {
-                counts[(int) ordinalMap.getGlobalOrd(segOrd, ord)] += count;
-              }
+      if (dv instanceof MultiSortedSetDocValues) {
+        MultiDocValues.OrdinalMap ordinalMap = ((MultiSortedSetDocValues) dv).mapping;
+        int segOrd = matchingDocs.context.ord;
+
+        int numSegOrds = (int) segValues.getValueCount();
+
+        if (matchingDocs.totalHits < numSegOrds/10) {
+          // Remap every ord to global ord as we iterate:
+          int doc = 0;
+          while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+            segValues.setDocument(doc);
+            int term = (int) segValues.nextOrd();
+            while (term != SortedSetDocValues.NO_MORE_ORDS) {
+              counts[(int) ordinalMap.getGlobalOrd(segOrd, term)]++;
+              term = (int) segValues.nextOrd();
             }
+            ++doc;
           }
         } else {
-          // No ord mapping (e.g., single segment index):
-          // just aggregate directly into counts:
 
+          // First count in seg-ord space:
+          final int[] segCounts = new int[numSegOrds];
           int doc = 0;
           while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
             segValues.setDocument(doc);
             int term = (int) segValues.nextOrd();
             while (term != SortedSetDocValues.NO_MORE_ORDS) {
-              counts[term]++;
+              segCounts[term]++;
               term = (int) segValues.nextOrd();
             }
             ++doc;
           }
-        }
-      }
 
-      @Override
-      public void rollupValues(FacetRequest fr, int ordinal, int[] children, int[] siblings, FacetArrays facetArrays) {
-        // Nothing to do here: we only support flat (dim +
-        // label) facets, and in accumulate we sum up the
-        // count for the dimension.
-      }
+          // Then, migrate to global ords:
+          for(int ord=0;ord<numSegOrds;ord++) {
+            int count = segCounts[ord];
+            if (count != 0) {
+              counts[(int) ordinalMap.getGlobalOrd(segOrd, ord)] += count;
+            }
+          }
+        }
+      } else {
+        // No ord mapping (e.g., single segment index):
+        // just aggregate directly into counts:
 
-      @Override
-      public boolean requiresDocScores() {
-        return false;
+        int doc = 0;
+        while (doc < maxDoc && (doc = matchingDocs.bits.nextSetBit(doc)) != -1) {
+          segValues.setDocument(doc);
+          int term = (int) segValues.nextOrd();
+          while (term != SortedSetDocValues.NO_MORE_ORDS) {
+            counts[term]++;
+            term = (int) segValues.nextOrd();
+          }
+          ++doc;
+        }
       }
-    };
-  }
-
-  /** Keeps highest count results. */
-  static class TopCountPQ extends PriorityQueue<FacetResultNode> {
-    public TopCountPQ(int topN) {
-      super(topN, false);
     }
 
-    @Override
-    protected boolean lessThan(FacetResultNode a, FacetResultNode b) {
-      if (a.value < b.value) {
-        return true;
-      } else if (a.value > b.value) {
-        return false;
-      } else {
-        return a.ordinal > b.ordinal;
-      }
-    }
   }
-
+  
   @Override
   public List<FacetResult> accumulate(List<MatchingDocs> matchingDocs) throws IOException {
 
-    FacetsAggregator aggregator = getAggregator();
-    for (CategoryListParams clp : getCategoryLists()) {
-      for (MatchingDocs md : matchingDocs) {
-        aggregator.aggregate(md, clp, facetArrays);
-      }
+    SortedSetAggregator aggregator = new SortedSetAggregator(field, state, dv);
+    for (MatchingDocs md : matchingDocs) {
+      aggregator.aggregate(md, facetArrays);
     }
 
     // compute top-K
@@ -206,7 +219,7 @@ public class SortedSetDocValuesAccumulat
 
     BytesRef scratch = new BytesRef();
 
-    for(FacetRequest request : searchParams.facetRequests) {
+    for (FacetRequest request : searchParams.facetRequests) {
       String dim = request.categoryPath.components[0];
       SortedSetDocValuesReaderState.OrdRange ordRange = state.getOrdRange(dim);
       // checked in ctor:
@@ -303,4 +316,10 @@ public class SortedSetDocValuesAccumulat
 
     return results;
   }
+  
+  @Override
+  public boolean requiresDocScores() {
+    return false;
+  }
+  
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesReaderState.java Sun Aug 11 12:19:13 2013
@@ -51,6 +51,7 @@ public final class SortedSetDocValuesRea
   private final String field;
   private final AtomicReader topReader;
   private final int valueCount;
+  final IndexReader origReader;
   final char separator;
   final String separatorRegex;
 
@@ -91,6 +92,7 @@ public final class SortedSetDocValuesRea
     this.field = fip.getCategoryListParams(null).field + FACET_FIELD_EXTENSION;
     this.separator = fip.getFacetDelimChar();
     this.separatorRegex = Pattern.quote(Character.toString(separator));
+    this.origReader = reader;
 
     // We need this to create thread-safe MultiSortedSetDV
     // per collector:

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/CategoryPath.java Sun Aug 11 12:19:13 2013
@@ -17,6 +17,8 @@ package org.apache.lucene.facet.taxonomy
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
+
 import java.util.Arrays;
 import java.util.regex.Pattern;
 
@@ -28,6 +30,18 @@ import java.util.regex.Pattern;
  */
 public class CategoryPath implements Comparable<CategoryPath> {
 
+  /*
+   * copied from DocumentWriterPerThread -- if a CategoryPath is resolved to a
+   * drill-down term which is encoded to a larger term than that length, it is
+   * silently dropped! Therefore we limit the number of characters to MAX/4 to
+   * be on the safe side.
+   */
+  /**
+   * The maximum number of characters a {@link CategoryPath} can have. That is
+   * {@link CategoryPath#toString(char)} length must not exceed that limit.
+   */
+  public final static int MAX_CATEGORY_PATH_LENGTH = (BYTE_BLOCK_SIZE - 2) / 4;
+
   /** An empty {@link CategoryPath} */
   public static final CategoryPath EMPTY = new CategoryPath();
 
@@ -63,10 +77,18 @@ public class CategoryPath implements Com
   /** Construct from the given path components. */
   public CategoryPath(final String... components) {
     assert components.length > 0 : "use CategoryPath.EMPTY to create an empty path";
+    long len = 0;
     for (String comp : components) {
       if (comp == null || comp.isEmpty()) {
         throw new IllegalArgumentException("empty or null components not allowed: " + Arrays.toString(components));
       }
+      len += comp.length();
+    }
+    len += components.length - 1; // add separators
+    if (len > MAX_CATEGORY_PATH_LENGTH) {
+      throw new IllegalArgumentException("category path exceeds maximum allowed path length: max="
+          + MAX_CATEGORY_PATH_LENGTH + " len=" + len
+          + " path=" + Arrays.toString(components).substring(0, 30) + "...");
     }
     this.components = components;
     length = components.length;
@@ -74,6 +96,12 @@ public class CategoryPath implements Com
 
   /** Construct from a given path, separating path components with {@code delimiter}. */
   public CategoryPath(final String pathString, final char delimiter) {
+    if (pathString.length() > MAX_CATEGORY_PATH_LENGTH) {
+      throw new IllegalArgumentException("category path exceeds maximum allowed path length: max="
+              + MAX_CATEGORY_PATH_LENGTH + " len=" + pathString.length()
+              + " path=" + pathString.substring(0, 30) + "...");
+    }
+
     String[] comps = pathString.split(Pattern.quote(Character.toString(delimiter)));
     if (comps.length == 1 && comps[0].isEmpty()) {
       components = null;

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/DirectoryTaxonomyWriter.java Sun Aug 11 12:19:13 2013
@@ -428,7 +428,7 @@ public class DirectoryTaxonomyWriter imp
         Terms terms = ctx.reader().terms(Consts.FULL);
         if (terms != null) {
           termsEnum = terms.iterator(termsEnum);
-          if (termsEnum.seekExact(catTerm, true)) {
+          if (termsEnum.seekExact(catTerm)) {
             // liveDocs=null because the taxonomy has no deletes
             docs = termsEnum.docs(null, docs, 0 /* freqs not required */);
             // if the term was found, we know it has exactly one document.

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/package.html Sun Aug 11 12:19:13 2013
@@ -21,28 +21,33 @@
 <body>
 	<h1>Taxonomy of Categories</h1>
 	
-	Facets are defined using a hierarchy of categories, known as a
-	<i>Taxonomy</i>.
-	
-	<br>
-	For example, in a book store application, a Taxonomy could have the
-	following hierarchy:
-	<p>
+	Facets are defined using a hierarchy of categories, known as a <i>Taxonomy</i>.
+	For example, the taxonomy of a book store application might have the following structure:
 	<ul>
-		<li>Author</li>
-		<ul>
-		<li>Mark Twain</li>
-		<li>J. K. Rowling</li>
-		</ul>
+		<li>Author
+			<ul>
+			<li>Mark Twain</li>
+			<li>J. K. Rowling</li>
+			</ul>
+		</li>
 	</ul>
 	<ul>
-		<li>Date</li>
-		<ul>
-		<li>2010</li>
-		<li>2009</li>
-		</ul>
+		<li>Date
+			<ul>
+			<li>2010
+				<ul>
+				<li>March</li>
+				<li>April</li>
+				</ul>
+			</li>
+			<li>2009</li>
+			</ul>
+		</li>
 	</ul>
 	
-	The <i>Taxonomy</i> translates category-paths into category-ordinal and vice versa.
+	The <i>Taxonomy</i> translates category-paths into interger identifiers (often termed <i>ordinals</i>) and vice versa.
+	The category <code>Author/Mark Twain</code> adds two nodes to the taxonomy: <code>Author</code> and 
+	<code>Author/Mark Twain</code>, each is assigned a different ordinal. The taxonomy maintains the invariant that a 
+	node always has an ordinal that is &lt; all its children.
 </body>
 </html>
\ No newline at end of file

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CategoryPathUtils.java Sun Aug 11 12:19:13 2013
@@ -39,14 +39,14 @@ class CategoryPathUtils {
    * {@link #serialize(CategoryPath, CharBlockArray)}.
    */
   public static int hashCodeOfSerialized(CharBlockArray charBlockArray, int offset) {
-    int length = (short) charBlockArray.charAt(offset++);
+    int length = charBlockArray.charAt(offset++);
     if (length == 0) {
       return 0;
     }
     
     int hash = length;
     for (int i = 0; i < length; i++) {
-      int len = (short) charBlockArray.charAt(offset++);
+      int len = charBlockArray.charAt(offset++);
       hash = hash * 31 + charBlockArray.subSequence(offset, offset + len).hashCode();
       offset += len;
     }
@@ -67,7 +67,7 @@ class CategoryPathUtils {
     }
     
     for (int i = 0; i < cp.length; i++) {
-      int len = (short) charBlockArray.charAt(offset++);
+      int len = charBlockArray.charAt(offset++);
       if (len != cp.components[i].length()) {
         return false;
       }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/FacetsPayloadMigrationReader.java Sun Aug 11 12:19:13 2013
@@ -90,7 +90,7 @@ public class FacetsPayloadMigrationReade
           Terms terms = fields.terms(term.field());
           if (terms != null) {
             TermsEnum te = terms.iterator(null); // no use for reusing
-            if (te.seekExact(term.bytes(), true)) {
+            if (te.seekExact(term.bytes())) {
               // we're not expected to be called for deleted documents
               dpe = te.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
             }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/org/apache/lucene/facet/util/TaxonomyMergeUtils.java Sun Aug 11 12:19:13 2013
@@ -45,7 +45,7 @@ public class TaxonomyMergeUtils {
     // merge the taxonomies
     destTaxWriter.addTaxonomy(srcTaxDir, map);
     int ordinalMap[] = map.getMap();
-    DirectoryReader reader = DirectoryReader.open(srcIndexDir, -1);
+    DirectoryReader reader = DirectoryReader.open(srcIndexDir);
     List<AtomicReaderContext> leaves = reader.leaves();
     int numReaders = leaves.size();
     AtomicReader wrappedLeaves[] = new AtomicReader[numReaders];

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/java/overview.html?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/java/overview.html (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/java/overview.html Sun Aug 11 12:19:13 2013
@@ -15,12 +15,10 @@
  limitations under the License.
 -->
 <html>
-  <head>
-    <title>
-      facet
-    </title>
-  </head>
-  <body>
-  Provides faceted indexing and search capabilities (checkout the <a href="org/apache/lucene/facet/doc-files/userguide.html">userguide</a>).
-  </body>
+<head><title>facet</title></head>
+<body>
+Provides faceted indexing and search capabilities. Checkout <a href="http://shaierera.blogspot.com/2012/11/lucene-facets-part-1.html">this</a>
+and <a href="http://shaierera.blogspot.com/2012/11/lucene-facets-part-2.html">this</a> blog posts for some overview on the facets module
+as well as source code examples <a href="../demo">here</a>.
+</body>
 </html>

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/associations/AssociationsFacetRequestTest.java Sun Aug 11 12:19:13 2013
@@ -1,8 +1,6 @@
 package org.apache.lucene.facet.associations;
 
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -10,9 +8,9 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.facet.FacetTestCase;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.FacetResult;
-import org.apache.lucene.facet.search.FacetsAccumulator;
 import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.search.FacetsCollector;
+import org.apache.lucene.facet.search.TaxonomyFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
@@ -69,14 +67,18 @@ public class AssociationsFacetRequestTes
     AssociationsFacetFields assocFacetFields = new AssociationsFacetFields(taxoWriter);
     
     // index documents, 50% have only 'b' and all have 'a'
-    for (int i = 0; i < 100; i++) {
+    for (int i = 0; i < 110; i++) {
       Document doc = new Document();
       CategoryAssociationsContainer associations = new CategoryAssociationsContainer();
-      associations.setAssociation(aint, new CategoryIntAssociation(2));
-      associations.setAssociation(afloat, new CategoryFloatAssociation(0.5f));
-      if (i % 2 == 0) { // 50
-        associations.setAssociation(bint, new CategoryIntAssociation(3));
-        associations.setAssociation(bfloat, new CategoryFloatAssociation(0.2f));
+      // every 11th document is added empty, this used to cause the association
+      // aggregators to go into an infinite loop
+      if (i % 11 != 0) {
+        associations.setAssociation(aint, new CategoryIntAssociation(2));
+        associations.setAssociation(afloat, new CategoryFloatAssociation(0.5f));
+        if (i % 2 == 0) { // 50
+          associations.setAssociation(bint, new CategoryIntAssociation(3));
+          associations.setAssociation(bfloat, new CategoryFloatAssociation(0.2f));
+        }
       }
       assocFacetFields.addFields(doc, associations);
       writer.addDocument(doc);
@@ -103,12 +105,12 @@ public class AssociationsFacetRequestTes
     
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
-        new AssociationIntSumFacetRequest(aint, 10),
-        new AssociationIntSumFacetRequest(bint, 10));
+        new SumIntAssociationFacetRequest(aint, 10),
+        new SumIntAssociationFacetRequest(bint, 10));
     
     Query q = new MatchAllDocsQuery();
     
-    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+    TaxonomyFacetsAccumulator fa = new TaxonomyFacetsAccumulator(fsp, reader, taxo) {
       @Override
       public FacetsAggregator getAggregator() {
         return new SumIntAssociationFacetsAggregator();
@@ -135,12 +137,12 @@ public class AssociationsFacetRequestTes
     
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
-        new AssociationFloatSumFacetRequest(afloat, 10),
-        new AssociationFloatSumFacetRequest(bfloat, 10));
+        new SumFloatAssociationFacetRequest(afloat, 10),
+        new SumFloatAssociationFacetRequest(bfloat, 10));
     
     Query q = new MatchAllDocsQuery();
     
-    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
+    TaxonomyFacetsAccumulator fa = new TaxonomyFacetsAccumulator(fsp, reader, taxo) {
       @Override
       public FacetsAggregator getAggregator() {
         return new SumFloatAssociationFacetsAggregator();
@@ -167,27 +169,14 @@ public class AssociationsFacetRequestTes
     
     // facet requests for two facets
     FacetSearchParams fsp = new FacetSearchParams(
-        new AssociationIntSumFacetRequest(aint, 10),
-        new AssociationIntSumFacetRequest(bint, 10),
-        new AssociationFloatSumFacetRequest(afloat, 10),
-        new AssociationFloatSumFacetRequest(bfloat, 10));
+        new SumIntAssociationFacetRequest(aint, 10),
+        new SumIntAssociationFacetRequest(bint, 10),
+        new SumFloatAssociationFacetRequest(afloat, 10),
+        new SumFloatAssociationFacetRequest(bfloat, 10));
     
     Query q = new MatchAllDocsQuery();
     
-    final SumIntAssociationFacetsAggregator sumInt = new SumIntAssociationFacetsAggregator();
-    final SumFloatAssociationFacetsAggregator sumFloat = new SumFloatAssociationFacetsAggregator();
-    final Map<CategoryPath,FacetsAggregator> aggregators = new HashMap<CategoryPath,FacetsAggregator>();
-    aggregators.put(aint, sumInt);
-    aggregators.put(bint, sumInt);
-    aggregators.put(afloat, sumFloat);
-    aggregators.put(bfloat, sumFloat);
-    FacetsAccumulator fa = new FacetsAccumulator(fsp, reader, taxo) {
-      @Override
-      public FacetsAggregator getAggregator() {
-        return new MultiAssociationsFacetsAggregator(aggregators);
-      }
-    };
-    FacetsCollector fc = FacetsCollector.create(fa);
+    FacetsCollector fc = FacetsCollector.create(fsp, reader, taxo);
     
     IndexSearcher searcher = newSearcher(reader);
     searcher.search(q, fc);
@@ -200,6 +189,6 @@ public class AssociationsFacetRequestTes
     assertEquals("Wrong count for category 'b'!",10f, (float) res.get(3).getFacetResultNode().value, 0.00001);
     
     taxo.close();
-  }  
+  }
   
 }

Modified: lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java (original)
+++ lucene/dev/branches/lucene4956/lucene/facet/src/test/org/apache/lucene/facet/complements/TestFacetsAccumulatorWithComplement.java Sun Aug 11 12:19:13 2013
@@ -4,13 +4,13 @@ import java.io.IOException;
 import java.util.List;
 
 import org.apache.lucene.facet.FacetTestBase;
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
 import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.CountFacetRequest;
 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.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiReader;
@@ -112,8 +112,8 @@ public class TestFacetsAccumulatorWithCo
   /** compute facets with certain facet requests and docs */
   private List<FacetResult> findFacets(boolean withComplement) throws IOException {
     FacetSearchParams fsp = new FacetSearchParams(fip, new CountFacetRequest(new CategoryPath("root","a"), 10));
-    StandardFacetsAccumulator sfa = new StandardFacetsAccumulator(fsp, indexReader, taxoReader);
-    sfa.setComplementThreshold(withComplement ? StandardFacetsAccumulator.FORCE_COMPLEMENT : StandardFacetsAccumulator.DISABLE_COMPLEMENT);
+    OldFacetsAccumulator sfa = new OldFacetsAccumulator(fsp, indexReader, taxoReader);
+    sfa.setComplementThreshold(withComplement ? OldFacetsAccumulator.FORCE_COMPLEMENT : OldFacetsAccumulator.DISABLE_COMPLEMENT);
     FacetsCollector fc = FacetsCollector.create(sfa);
     searcher.search(new MatchAllDocsQuery(), fc);
     
@@ -121,8 +121,8 @@ public class TestFacetsAccumulatorWithCo
     
     // Results are ready, printing them...
     int i = 0;
-    for (FacetResult facetResult : res) {
-      if (VERBOSE) {
+    if (VERBOSE) {
+      for (FacetResult facetResult : res) {
         System.out.println("Res "+(i++)+": "+facetResult);
       }
     }