You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ha...@apache.org on 2013/08/13 06:06:27 UTC

svn commit: r1513336 [7/11] - in /lucene/dev/branches/lucene3069/lucene: ./ analysis/ analysis/common/ analysis/common/src/java/org/apache/lucene/analysis/charfilter/ analysis/common/src/java/org/apache/lucene/analysis/hunspell/ analysis/common/src/jav...

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeAccumulator.java Tue Aug 13 04:06:18 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.facet.range;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 import org.apache.lucene.facet.params.FacetSearchParams;
@@ -26,10 +27,8 @@ 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.taxonomy.CategoryPath;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.NumericDocValues;
 
 /** Uses a {@link NumericDocValues} and accumulates
@@ -51,38 +50,34 @@ public class RangeAccumulator extends Fa
 
   final List<RangeSet> requests = new ArrayList<RangeSet>();
 
-  public RangeAccumulator(FacetSearchParams fsp, IndexReader reader) {
-    super(fsp, reader, null, null);
-
-    for(FacetRequest fr : fsp.facetRequests) {
-
+  public RangeAccumulator(FacetRequest... facetRequests) {
+    this(Arrays.asList(facetRequests));
+  }
+  
+  public RangeAccumulator(List<FacetRequest> facetRequests) {
+    super(new FacetSearchParams(facetRequests));
+    for (FacetRequest fr : facetRequests) {
       if (!(fr instanceof RangeFacetRequest)) {
-        throw new IllegalArgumentException("only RangeFacetRequest is supported; got " + fsp.facetRequests.get(0).getClass());
+        throw new IllegalArgumentException("this accumulator only supports RangeFacetRequest; got " + fr);
       }
 
       if (fr.categoryPath.length != 1) {
         throw new IllegalArgumentException("only flat (dimension only) CategoryPath is allowed");
       }
-
+      
       RangeFacetRequest<?> rfr = (RangeFacetRequest<?>) fr;
-
-      requests.add(new RangeSet(rfr.ranges, rfr.categoryPath.components[0]));
+      requests.add(new RangeSet(rfr.ranges, fr.categoryPath.components[0]));
     }
   }
 
   @Override
-  public FacetsAggregator getAggregator() {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
   public List<FacetResult> accumulate(List<MatchingDocs> matchingDocs) throws IOException {
 
     // TODO: test if this is faster (in the past it was
     // faster to do MachingDocs on the inside) ... see
     // patches on LUCENE-4965):
     List<FacetResult> results = new ArrayList<FacetResult>();
-    for(int i=0;i<requests.size();i++) {
+    for (int i = 0; i < requests.size(); i++) {
       RangeSet ranges = requests.get(i);
 
       int[] counts = new int[ranges.ranges.length];
@@ -100,7 +95,7 @@ public class RangeAccumulator extends Fa
           // (really, a specialized case of the interval
           // tree)
           // TODO: use interval tree instead of linear search:
-          for(int j=0;j<ranges.ranges.length;j++) {
+          for (int j = 0; j < ranges.ranges.length; j++) {
             if (ranges.ranges[j].accept(v)) {
               counts[j]++;
             }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetRequest.java Tue Aug 13 04:06:18 2013
@@ -19,11 +19,10 @@ package org.apache.lucene.facet.range;
 
 import java.util.List;
 
-import org.apache.lucene.facet.search.Aggregator;
-import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.FacetRequest;
+import org.apache.lucene.facet.search.FacetsAggregator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /**
  * Facet request for dynamic ranges based on a
@@ -36,6 +35,7 @@ public class RangeFacetRequest<T extends
 
   public final Range[] ranges;
 
+  @SuppressWarnings("unchecked")
   public RangeFacetRequest(String field, T...ranges) {
     super(new CategoryPath(field), 1);
     this.ranges = ranges;
@@ -47,18 +47,8 @@ public class RangeFacetRequest<T extends
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public double getValueOf(FacetArrays arrays, int ordinal) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public FacetArraysSource getFacetArraysSource() {
-    throw new UnsupportedOperationException();
+  public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
+    return null;
   }
   
 }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RandomSampler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RandomSampler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RandomSampler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RandomSampler.java Tue Aug 13 04:06:18 2013
@@ -3,9 +3,9 @@ package org.apache.lucene.facet.sampling
 import java.io.IOException;
 import java.util.Random;
 
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.ScoredDocIDsIterator;
-import org.apache.lucene.facet.util.ScoredDocIdsUtils;
+import org.apache.lucene.facet.old.ScoredDocIDs;
+import org.apache.lucene.facet.old.ScoredDocIDsIterator;
+import org.apache.lucene.facet.old.ScoredDocIdsUtils;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RepeatableSampler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RepeatableSampler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RepeatableSampler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/RepeatableSampler.java Tue Aug 13 04:06:18 2013
@@ -5,12 +5,11 @@ import java.util.Arrays;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.lucene.facet.old.ScoredDocIDs;
+import org.apache.lucene.facet.old.ScoredDocIDsIterator;
+import org.apache.lucene.facet.old.ScoredDocIdsUtils;
 import org.apache.lucene.util.PriorityQueue;
 
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.ScoredDocIDsIterator;
-import org.apache.lucene.facet.util.ScoredDocIdsUtils;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SampleFixer.java Tue Aug 13 04:06:18 2013
@@ -2,9 +2,9 @@ package org.apache.lucene.facet.sampling
 
 import java.io.IOException;
 
+import org.apache.lucene.facet.old.ScoredDocIDs;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
-import org.apache.lucene.facet.search.ScoredDocIDs;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/Sampler.java Tue Aug 13 04:06:18 2013
@@ -4,14 +4,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.facet.old.ScoredDocIDs;
+import org.apache.lucene.facet.params.FacetIndexingParams;
 import org.apache.lucene.facet.params.FacetSearchParams;
-import org.apache.lucene.facet.search.Aggregator;
-import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetResultNode;
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.search.FacetsAggregator;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -196,15 +195,9 @@ public abstract class Sampler {
     return res;
   }
   
-  /**
-   * Wrapping a facet request for over sampling.
-   * Implementation detail: even if the original request is a count request, no 
-   * statistics will be computed for it as the wrapping is not a count request.
-   * This is ok, as the sampling accumulator is later computing the statistics
-   * over the original requests.
-   */
-  private static class OverSampledFacetRequest extends FacetRequest {
-    final FacetRequest orig;
+  /** Wrapping a facet request for over sampling. */
+  public static class OverSampledFacetRequest extends FacetRequest {
+    public final FacetRequest orig;
     public OverSampledFacetRequest(FacetRequest orig, int num) {
       super(orig.categoryPath, num);
       this.orig = orig;
@@ -215,19 +208,8 @@ public abstract class Sampler {
     }
     
     @Override
-    public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
-        throws IOException {
-      return orig.createAggregator(useComplements, arrays, taxonomy);
-    }
-
-    @Override
-    public FacetArraysSource getFacetArraysSource() {
-      return orig.getFacetArraysSource();
-    }
-
-    @Override
-    public double getValueOf(FacetArrays arrays, int idx) {
-      return orig.getValueOf(arrays, idx);
+    public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
+      return orig.createFacetsAggregator(fip);
     }
   }
 

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingAccumulator.java Tue Aug 13 04:06:18 2013
@@ -4,14 +4,15 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
+import org.apache.lucene.facet.old.ScoredDocIDs;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.partitions.PartitionsFacetResultsHandler;
 import org.apache.lucene.facet.sampling.Sampler.SampleResult;
 import org.apache.lucene.facet.search.FacetArrays;
+import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.search.FacetResult;
 import org.apache.lucene.facet.search.FacetsAccumulator;
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.IndexReader;
 
@@ -38,10 +39,10 @@ import org.apache.lucene.index.IndexRead
  * Note two major differences between this class and {@link SamplingWrapper}:
  * <ol>
  * <li>Latter can wrap any other {@link FacetsAccumulator} while this class
- * directly extends {@link StandardFacetsAccumulator}.</li>
+ * directly extends {@link OldFacetsAccumulator}.</li>
  * <li>This class can effectively apply sampling on the complement set of
  * matching document, thereby working efficiently with the complement
- * optimization - see {@link StandardFacetsAccumulator#getComplementThreshold()}
+ * optimization - see {@link OldFacetsAccumulator#getComplementThreshold()}
  * .</li>
  * </ol>
  * <p>
@@ -52,7 +53,7 @@ import org.apache.lucene.index.IndexRead
  * @see Sampler
  * @lucene.experimental
  */
-public class SamplingAccumulator extends StandardFacetsAccumulator {
+public class SamplingAccumulator extends OldFacetsAccumulator {
   
   private double samplingRatio = -1d;
   private final Sampler sampler;
@@ -90,7 +91,8 @@ public class SamplingAccumulator extends
     List<FacetResult> results = new ArrayList<FacetResult>();
     for (FacetResult fres : sampleRes) {
       // for sure fres is not null because this is guaranteed by the delegee.
-      PartitionsFacetResultsHandler frh = createFacetResultsHandler(fres.getFacetRequest());
+      FacetRequest fr = fres.getFacetRequest();
+      PartitionsFacetResultsHandler frh = createFacetResultsHandler(fr, createOrdinalValueResolver(fr));
       if (samplerFixer != null) {
         // fix the result of current request
         samplerFixer.fixResult(docids, fres, samplingRatio);
@@ -106,7 +108,7 @@ public class SamplingAccumulator extends
       // final labeling if allowed (because labeling is a costly operation)
       if (fres.getFacetResultNode().ordinal == TaxonomyReader.INVALID_ORDINAL) {
         // category does not exist, add an empty result
-        results.add(emptyResult(fres.getFacetResultNode().ordinal, fres.getFacetRequest()));
+        results.add(emptyResult(fres.getFacetResultNode().ordinal, fr));
       } else {
         frh.labelResult(fres);
         results.add(fres);

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingParams.java Tue Aug 13 04:06:18 2013
@@ -32,19 +32,19 @@ public class SamplingParams {
   
   /**
    * Default ratio between size of sample to original size of document set.
-   * @see Sampler#getSampleSet(org.apache.lucene.facet.search.ScoredDocIDs)
+   * @see Sampler#getSampleSet(org.apache.lucene.facet.old.ScoredDocIDs)
    */
   public static final double DEFAULT_SAMPLE_RATIO = 0.01;
   
   /**
    * Default maximum size of sample.
-   * @see Sampler#getSampleSet(org.apache.lucene.facet.search.ScoredDocIDs)
+   * @see Sampler#getSampleSet(org.apache.lucene.facet.old.ScoredDocIDs)
    */
   public static final int DEFAULT_MAX_SAMPLE_SIZE = 10000;
   
   /**
    * Default minimum size of sample.
-   * @see Sampler#getSampleSet(org.apache.lucene.facet.search.ScoredDocIDs)
+   * @see Sampler#getSampleSet(org.apache.lucene.facet.old.ScoredDocIDs)
    */
   public static final int DEFAULT_MIN_SAMPLE_SIZE = 100;
   
@@ -65,7 +65,7 @@ public class SamplingParams {
   /**
    * Return the maxSampleSize.
    * In no case should the resulting sample size exceed this value.  
-   * @see Sampler#getSampleSet(org.apache.lucene.facet.search.ScoredDocIDs)
+   * @see Sampler#getSampleSet(org.apache.lucene.facet.old.ScoredDocIDs)
    */
   public final int getMaxSampleSize() {
     return maxSampleSize;
@@ -74,7 +74,7 @@ public class SamplingParams {
   /**
    * Return the minSampleSize.
    * In no case should the resulting sample size be smaller than this value.  
-   * @see Sampler#getSampleSet(org.apache.lucene.facet.search.ScoredDocIDs)
+   * @see Sampler#getSampleSet(org.apache.lucene.facet.old.ScoredDocIDs)
    */
   public final int getMinSampleSize() {
     return minSampleSize;
@@ -82,7 +82,7 @@ public class SamplingParams {
 
   /**
    * @return the sampleRatio
-   * @see Sampler#getSampleSet(org.apache.lucene.facet.search.ScoredDocIDs)
+   * @see Sampler#getSampleSet(org.apache.lucene.facet.old.ScoredDocIDs)
    */
   public final double getSampleRatio() {
     return sampleRatio;

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/SamplingWrapper.java Tue Aug 13 04:06:18 2013
@@ -4,12 +4,13 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.facet.old.OldFacetsAccumulator;
+import org.apache.lucene.facet.old.ScoredDocIDs;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.partitions.PartitionsFacetResultsHandler;
 import org.apache.lucene.facet.sampling.Sampler.SampleResult;
+import org.apache.lucene.facet.search.FacetRequest;
 import org.apache.lucene.facet.search.FacetResult;
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.StandardFacetsAccumulator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
@@ -38,12 +39,12 @@ import org.apache.lucene.facet.taxonomy.
  * 
  * @lucene.experimental
  */
-public class SamplingWrapper extends StandardFacetsAccumulator {
+public class SamplingWrapper extends OldFacetsAccumulator {
 
-  private StandardFacetsAccumulator delegee;
+  private OldFacetsAccumulator delegee;
   private Sampler sampler;
 
-  public SamplingWrapper(StandardFacetsAccumulator delegee, Sampler sampler) {
+  public SamplingWrapper(OldFacetsAccumulator delegee, Sampler sampler) {
     super(delegee.searchParams, delegee.indexReader, delegee.taxonomyReader);
     this.delegee = delegee;
     this.sampler = sampler;
@@ -68,7 +69,8 @@ public class SamplingWrapper extends Sta
     
     for (FacetResult fres : sampleRes) {
       // for sure fres is not null because this is guaranteed by the delegee.
-      PartitionsFacetResultsHandler frh = createFacetResultsHandler(fres.getFacetRequest());
+      FacetRequest fr = fres.getFacetRequest();
+      PartitionsFacetResultsHandler frh = createFacetResultsHandler(fr, createOrdinalValueResolver(fr));
       if (sampleFixer != null) {
         // fix the result of current request
         sampleFixer.fixResult(docids, fres, sampleSet.actualSampleRatio); 
@@ -83,7 +85,7 @@ public class SamplingWrapper extends Sta
       // final labeling if allowed (because labeling is a costly operation)
       if (fres.getFacetResultNode().ordinal == TaxonomyReader.INVALID_ORDINAL) {
         // category does not exist, add an empty result
-        results.add(emptyResult(fres.getFacetResultNode().ordinal, fres.getFacetRequest()));
+        results.add(emptyResult(fres.getFacetResultNode().ordinal, fr));
       } else {
         frh.labelResult(fres);
         results.add(fres);

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/sampling/TakmiSampleFixer.java Tue Aug 13 04:06:18 2013
@@ -2,11 +2,11 @@ package org.apache.lucene.facet.sampling
 
 import java.io.IOException;
 
+import org.apache.lucene.facet.old.ScoredDocIDs;
+import org.apache.lucene.facet.old.ScoredDocIDsIterator;
 import org.apache.lucene.facet.params.FacetSearchParams;
 import org.apache.lucene.facet.search.DrillDownQuery;
 import org.apache.lucene.facet.search.FacetResultNode;
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.ScoredDocIDsIterator;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.DocsEnum;

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountFacetRequest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountFacetRequest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountFacetRequest.java Tue Aug 13 04:06:18 2013
@@ -1,8 +1,7 @@
 package org.apache.lucene.facet.search;
 
-import org.apache.lucene.facet.complements.ComplementCountingAggregator;
+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
@@ -33,23 +32,8 @@ public class CountFacetRequest extends F
   }
 
   @Override
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) {
-    // we rely on that, if needed, result is cleared by arrays!
-    int[] a = arrays.getIntArray();
-    if (useComplements) {
-      return new ComplementCountingAggregator(a);
-    }
-    return new CountingAggregator(a);
-  }
-
-  @Override
-  public double getValueOf(FacetArrays arrays, int ordinal) {
-    return arrays.getIntArray()[ordinal];
-  }
-
-  @Override
-  public FacetArraysSource getFacetArraysSource() {
-    return FacetArraysSource.INT;
+  public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
+    return CountingFacetsAggregator.create(fip.getCategoryListParams(categoryPath));
   }
   
 }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/CountingFacetsAggregator.java Tue Aug 13 04:06:18 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.facet.encoding.DGapVInt8IntDecoder;
 import org.apache.lucene.facet.params.CategoryListParams;
 import org.apache.lucene.facet.search.FacetsCollector.MatchingDocs;
 import org.apache.lucene.util.IntsRef;
@@ -34,6 +35,18 @@ import org.apache.lucene.util.IntsRef;
  */
 public class CountingFacetsAggregator extends IntRollupFacetsAggregator {
   
+  /**
+   * Returns a {@link FacetsAggregator} suitable for counting categories given
+   * the {@link CategoryListParams}.
+   */
+  public static FacetsAggregator create(CategoryListParams clp) {
+    if (clp.createEncoder().createMatchingDecoder().getClass() == DGapVInt8IntDecoder.class) {
+      return new FastCountingFacetsAggregator();
+    } else {
+      return new CountingFacetsAggregator();
+    }
+  }
+
   private final IntsRef ordinals = new IntsRef(32);
   
   @Override

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DepthOneFacetResultsHandler.java Tue Aug 13 04:06:18 2013
@@ -9,6 +9,7 @@ import java.util.Comparator;
 import org.apache.lucene.facet.search.FacetRequest.SortOrder;
 import org.apache.lucene.facet.taxonomy.ParallelTaxonomyArrays;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.util.CollectionUtil;
 import org.apache.lucene.util.PriorityQueue;
 
 /*
@@ -31,12 +32,11 @@ import org.apache.lucene.util.PriorityQu
 /**
  * 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}.
+ * value, in descending order.
  * 
  * @lucene.experimental
  */
-public abstract class DepthOneFacetResultsHandler extends FacetResultsHandler {
+public class DepthOneFacetResultsHandler extends FacetResultsHandler {
   
   private static class FacetResultNodeQueue extends PriorityQueue<FacetResultNode> {
     
@@ -51,40 +51,19 @@ public abstract class DepthOneFacetResul
     
     @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;
+      return a.compareTo(b)  < 0;
     }
     
   }
 
-  public DepthOneFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays) {
-    super(taxonomyReader, facetRequest, facetArrays);
+  public DepthOneFacetResultsHandler(TaxonomyReader taxonomyReader, FacetRequest facetRequest, FacetArrays facetArrays, 
+      OrdinalValueResolver resolver) {
+    super(taxonomyReader, facetRequest, resolver, 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";
   }
 
-  /** 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.  Returns the total
-   * number of siblings.
-   */
-  protected abstract int addSiblings(int ordinal, int[] siblings, PriorityQueue<FacetResultNode> pq);
-  
   @Override
   public final FacetResult compute() throws IOException {
     ParallelTaxonomyArrays arrays = taxonomyReader.getParallelTaxonomyArrays();
@@ -93,23 +72,28 @@ public abstract class DepthOneFacetResul
     
     int rootOrd = taxonomyReader.getOrdinal(facetRequest.categoryPath);
         
-    FacetResultNode root = new FacetResultNode(rootOrd, valueOf(rootOrd));
+    FacetResultNode root = new FacetResultNode(rootOrd, resolver.valueOf(rootOrd));
     root.label = facetRequest.categoryPath;
     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>() {
+      int ordinal = children[rootOrd];
+      while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+        double value = resolver.valueOf(ordinal);
+        if (value > 0) {
+          FacetResultNode node = new FacetResultNode(ordinal, value);
+          node.label = taxonomyReader.getPath(ordinal);
+          nodes.add(node);
+        }
+        ordinal = siblings[ordinal];
+      }
+
+      CollectionUtil.introSort(nodes, Collections.reverseOrder(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;
+          return o1.compareTo(o2);
         }
-      });
+      }));
       
       root.subResults = nodes;
       return new FacetResult(facetRequest, root, nodes.size());
@@ -117,7 +101,21 @@ public abstract class DepthOneFacetResul
     
     // 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 numSiblings = addSiblings(children[rootOrd], siblings, pq);
+    int ordinal = children[rootOrd];
+    FacetResultNode top = pq.top();
+    int numSiblings = 0;
+    while (ordinal != TaxonomyReader.INVALID_ORDINAL) {
+      double value = resolver.valueOf(ordinal);
+      if (value > 0) {
+        ++numSiblings;
+        if (value > top.value) {
+          top.value = value;
+          top.ordinal = ordinal;
+          top = pq.updateTop();
+        }
+      }
+      ordinal = siblings[ordinal];
+    }
 
     // pop() the least (sentinel) elements
     int pqsize = pq.size();

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSideways.java Tue Aug 13 04:06:18 2013
@@ -25,7 +25,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.params.FacetSearchParams;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesFacetFields;
+import org.apache.lucene.facet.sortedset.SortedSetDocValuesReaderState;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
@@ -70,11 +73,26 @@ public class DrillSideways {
 
   protected final IndexSearcher searcher;
   protected final TaxonomyReader taxoReader;
-
-  /** Create a new {@code DrillSideways} instance. */
+  protected final SortedSetDocValuesReaderState state;
+  
+  /**
+   * Create a new {@code DrillSideways} instance, assuming the categories were
+   * indexed with {@link FacetFields}.
+   */
   public DrillSideways(IndexSearcher searcher, TaxonomyReader taxoReader) {
     this.searcher = searcher;
     this.taxoReader = taxoReader;
+    this.state = null;
+  }
+  
+  /**
+   * Create a new {@code DrillSideways} instance, assuming the categories were
+   * indexed with {@link SortedSetDocValuesFacetFields}.
+   */
+  public DrillSideways(IndexSearcher searcher, SortedSetDocValuesReaderState state) {
+    this.searcher = searcher;
+    this.taxoReader = null;
+    this.state = state;
   }
 
   /** Moves any drill-downs that don't have a corresponding
@@ -440,13 +458,21 @@ public class DrillSideways {
   /** Override this to use a custom drill-down {@link
    *  FacetsAccumulator}. */
   protected FacetsAccumulator getDrillDownAccumulator(FacetSearchParams fsp) throws IOException {
-    return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader);
+    if (taxoReader != null) {
+      return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader, null);
+    } else {
+      return FacetsAccumulator.create(fsp, state, null);
+    }
   }
 
   /** Override this to use a custom drill-sideways {@link
    *  FacetsAccumulator}. */
   protected FacetsAccumulator getDrillSidewaysAccumulator(String dim, FacetSearchParams fsp) throws IOException {
-    return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader);
+    if (taxoReader != null) {
+      return FacetsAccumulator.create(fsp, searcher.getIndexReader(), taxoReader, null);
+    } else {
+      return FacetsAccumulator.create(fsp, state, null);
+    }
   }
 
   /** Override this and return true if your collector

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/DrillSidewaysQuery.java Tue Aug 13 04:06:18 2013
@@ -130,7 +130,7 @@ class DrillSidewaysQuery extends Query {
             continue;
           }
           for(int i=0;i<drillDownTerms[dim].length;i++) {
-            if (termsEnum.seekExact(drillDownTerms[dim][i].bytes(), false)) {
+            if (termsEnum.seekExact(drillDownTerms[dim][i].bytes())) {
               DocsEnum docsEnum = termsEnum.docs(null, null, 0);
               if (docsEnum != null) {
                 dims[dim].docsEnums[i] = docsEnum;

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetRequest.java Tue Aug 13 04:06:18 2013
@@ -1,9 +1,9 @@
 package org.apache.lucene.facet.search;
 
-import java.io.IOException;
-
+import org.apache.lucene.facet.params.CategoryListParams.OrdinalPolicy;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.range.RangeFacetRequest;
 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
@@ -23,90 +23,70 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Request to accumulate facet information for a specified facet and possibly 
- * also some of its descendants, upto a specified depth.
+ * Defines an aggregation request for a category. Allows specifying the
+ * {@link #numResults number of child categories} to return as well as
+ * {@link #getSortOrder() which} categories to consider the "top" (highest or
+ * lowest ranking ones).
  * <p>
- * The facet request additionally defines what information should 
- * be computed within the facet results, if and how should results
- * be ordered, etc.
- * <P>
- * An example facet request is to look at all sub-categories of "Author", and
- * return the 10 with the highest counts (sorted by decreasing count). 
+ * If the category being aggregated is hierarchical, you can also specify the
+ * {@link #setDepth(int) depth} up which to aggregate child categories as well
+ * as how the result should be {@link #setResultMode(ResultMode) constructed}.
  * 
  * @lucene.experimental
  */
 public abstract class FacetRequest {
   
   /**
-   * Result structure manner of applying request's limits such as
-   * {@link FacetRequest#getNumLabel()} and {@link FacetRequest#numResults}.
-   * Only relevant when {@link FacetRequest#getDepth()} is &gt; 1.
+   * When {@link FacetRequest#getDepth()} is greater than 1, defines the
+   * structure of the result as well as how constraints such as
+   * {@link FacetRequest#numResults} and {@link FacetRequest#getNumLabel()} are
+   * applied.
    */
   public enum ResultMode { 
-    /** Limits are applied per node, and the result has a full tree structure. */
+    /**
+     * Constraints are applied per node, and the result has a full tree
+     * structure. Default result mode.
+     */
     PER_NODE_IN_TREE, 
     
-    /** Limits are applied globally, on total number of results, and the result has a flat structure. */
+    /**
+     * Constraints are applied globally, on total number of results, and the
+     * result has a flat structure.
+     */
     GLOBAL_FLAT
   }
   
   /**
-   * Specifies which array of {@link FacetArrays} should be used to resolve
-   * values. When set to {@link #INT} or {@link #FLOAT}, allows creating an
-   * optimized {@link FacetResultsHandler}, which does not call
-   * {@link FacetRequest#getValueOf(FacetArrays, int)} for every ordinals.
-   * <p>
-   * If set to {@link #BOTH}, the {@link FacetResultsHandler} will use
-   * {@link FacetRequest#getValueOf(FacetArrays, int)} to resolve ordinal
-   * values, although it is recommended that you consider writing a specialized
-   * {@link FacetResultsHandler}.
+   * Defines which categories to return. If {@link #DESCENDING} (the default),
+   * the highest {@link FacetRequest#numResults} weighted categories will be
+   * returned, otherwise the lowest ones.
    */
-  public enum FacetArraysSource { INT, FLOAT, BOTH }
-  
-  /** Requested sort order for the results. */
   public enum SortOrder { ASCENDING, DESCENDING }
-  
-  /**
-   * Default depth for facets accumulation.
-   * @see #getDepth()
-   */
-  public static final int DEFAULT_DEPTH = 1;
-  
-  /**
-   * Default result mode
-   * @see #getResultMode()
-   */
-  public static final ResultMode DEFAULT_RESULT_MODE = ResultMode.PER_NODE_IN_TREE;
-  
+
+  /** The category being aggregated in this facet request. */
   public final CategoryPath categoryPath;
+  
+  /** The number of child categories to return for {@link #categoryPath}. */
   public final int numResults;
   
   private int numLabel;
-  private int depth;
-  private SortOrder sortOrder;
+  private int depth = 1;
+  private SortOrder sortOrder = SortOrder.DESCENDING;
+  private ResultMode resultMode = ResultMode.PER_NODE_IN_TREE;
   
-  /**
-   * Computed at construction, this hashCode is based on two final members
-   * {@link CategoryPath} and <code>numResults</code>
-   */
+  // Computed at construction; based on categoryPath and numResults.
   private final int hashCode;
   
-  private ResultMode resultMode = DEFAULT_RESULT_MODE;
-  
   /**
-   * Initialize the request with a given path, and a requested number of facets
-   * results. By default, all returned results would be labeled - to alter this
-   * default see {@link #setNumLabel(int)}.
-   * <p>
-   * <b>NOTE:</b> if <code>numResults</code> is given as
-   * <code>Integer.MAX_VALUE</code> than all the facet results would be
-   * returned, without any limit.
-   * <p>
-   * <b>NOTE:</b> it is assumed that the given {@link CategoryPath} is not
-   * modified after construction of this object. Otherwise, some things may not
-   * function properly, e.g. {@link #hashCode()}.
+   * Constructor with the given category to aggregate and the number of child
+   * categories to return.
    * 
-   * @throws IllegalArgumentException if numResults is &le; 0
+   * @param path
+   *          the category to aggregate. Cannot be {@code null}.
+   * @param numResults
+   *          the number of child categories to return. If set to
+   *          {@code Integer.MAX_VALUE}, all immediate child categories will be
+   *          returned. Must be greater than 0.
    */
   public FacetRequest(CategoryPath path, int numResults) {
     if (numResults <= 0) {
@@ -118,152 +98,116 @@ public abstract class FacetRequest {
     categoryPath = path;
     this.numResults = numResults;
     numLabel = numResults;
-    depth = DEFAULT_DEPTH;
-    sortOrder = SortOrder.DESCENDING;
-    
     hashCode = categoryPath.hashCode() ^ this.numResults;
   }
   
   /**
-   * Create an aggregator for this facet request. Aggregator action depends on
-   * request definition. For a count request, it will usually increment the
-   * count for that facet.
-   * 
-   * @param useComplements
-   *          whether the complements optimization is being used for current
-   *          computation.
-   * @param arrays
-   *          provider for facet arrays in use for current computation.
-   * @param taxonomy
-   *          reader of taxonomy in effect.
-   * @throws IOException If there is a low-level I/O error.
+   * Returns the {@link FacetsAggregator} which can aggregate the categories of
+   * this facet request. The aggregator is expected to aggregate category values
+   * into {@link FacetArrays}. If the facet request does not support that, e.g.
+   * {@link RangeFacetRequest}, it can return {@code null}. Note though that
+   * such requests require a dedicated {@link FacetsAccumulator}.
    */
-  public Aggregator createAggregator(boolean useComplements, FacetArrays arrays, TaxonomyReader taxonomy) 
-      throws IOException {
-    throw new UnsupportedOperationException("this FacetRequest does not support this type of Aggregator anymore; " +
-        "you should override FacetsAccumulator to return the proper FacetsAggregator");
-  }
+  public abstract FacetsAggregator createFacetsAggregator(FacetIndexingParams fip);
   
   @Override
   public boolean equals(Object o) {
     if (o instanceof FacetRequest) {
-      FacetRequest that = (FacetRequest)o;
-      return that.hashCode == this.hashCode &&
+      FacetRequest that = (FacetRequest) o;
+     return that.hashCode == this.hashCode &&
           that.categoryPath.equals(this.categoryPath) &&
           that.numResults == this.numResults &&
           that.depth == this.depth &&
           that.resultMode == this.resultMode &&
-          that.numLabel == this.numLabel;
+          that.numLabel == this.numLabel &&
+          that.sortOrder == this.sortOrder;
     }
     return false;
   }
   
   /**
-   * How deeply to look under the given category. If the depth is 0,
-   * only the category itself is counted. If the depth is 1, its immediate
-   * children are also counted, and so on. If the depth is Integer.MAX_VALUE,
-   * all the category's descendants are counted.<br>
+   * How deeply to look under {@link #categoryPath}. By default, only its
+   * immediate children are aggregated (depth=1). If set to
+   * {@code Integer.MAX_VALUE}, the entire sub-tree of the category will be
+   * aggregated.
+   * <p>
+   * <b>NOTE:</b> setting depth to 0 means that only the category itself should
+   * be aggregated. In that case, make sure to index the category with
+   * {@link OrdinalPolicy#ALL_PARENTS}, unless it is not the root category (the
+   * dimension), in which case {@link OrdinalPolicy#ALL_BUT_DIMENSION} is fine
+   * too.
    */
   public final int getDepth() {
-    // TODO add AUTO_EXPAND option  
+    // TODO an AUTO_EXPAND option could be useful  
     return depth;
   }
   
   /**
-   * Returns the {@link FacetArraysSource} this {@link FacetRequest} uses in
-   * {@link #getValueOf(FacetArrays, int)}.
-   */
-  public abstract FacetArraysSource getFacetArraysSource();
-  
-  /**
-   * If getNumLabel() &lt; getNumResults(), only the first getNumLabel() results
-   * will have their category paths calculated, and the rest will only be
-   * available as ordinals (category numbers) and will have null paths.
-   * <P>
-   * If Integer.MAX_VALUE is specified, all results are labled.
-   * <P>
-   * The purpose of this parameter is to avoid having to run the whole faceted
-   * search again when the user asks for more values for the facet; The
-   * application can ask (getNumResults()) for more values than it needs to
-   * show, but keep getNumLabel() only the number it wants to immediately show.
-   * The slow-down caused by finding more values is negligible, because the
-   * slowest part - finding the categories' paths, is avoided.
+   * Allows to specify the number of categories to label. By default all
+   * returned categories are labeled.
    * <p>
-   * Depending on the {@link #getResultMode() LimitsMode}, this limit is applied
-   * globally or per results node. In the global mode, if this limit is 3, only
-   * 3 top results would be labeled. In the per-node mode, if this limit is 3, 3
-   * top children of {@link #categoryPath the target category} would be labeled,
-   * as well as 3 top children of each of them, and so forth, until the depth
-   * defined by {@link #getDepth()}.
-   * 
-   * @see #getResultMode()
+   * This allows an app to request a large number of results to return, while
+   * labeling them on-demand (e.g. when the UI requests to show more
+   * categories).
    */
   public final int getNumLabel() {
     return numLabel;
   }
   
-  /** Return the requested result mode. */
+  /** Return the requested result mode (defaults to {@link ResultMode#PER_NODE_IN_TREE}. */
   public final ResultMode getResultMode() {
     return resultMode;
   }
   
-  /** Return the requested order of results. */
+  /** Return the requested order of results (defaults to {@link SortOrder#DESCENDING}. */
   public final SortOrder getSortOrder() {
     return sortOrder;
   }
   
-  /**
-   * Return the value of a category used for facets computations for this
-   * request. For a count request this would be the count for that facet, i.e.
-   * an integer number. but for other requests this can be the result of a more
-   * complex operation, and the result can be any double precision number.
-   * Having this method with a general name <b>value</b> which is double
-   * precision allows to have more compact API and code for handling counts and
-   * perhaps other requests (such as for associations) very similarly, and by
-   * the same code and API, avoiding code duplication.
-   * 
-   * @param arrays
-   *          provider for facet arrays in use for current computation.
-   * @param idx
-   *          an index into the count arrays now in effect in
-   *          <code>arrays</code>. E.g., for ordinal number <i>n</i>, with
-   *          partition, of size <i>partitionSize</i>, now covering <i>n</i>,
-   *          <code>getValueOf</code> would be invoked with <code>idx</code>
-   *          being <i>n</i> % <i>partitionSize</i>.
-   */
-  // TODO perhaps instead of getValueOf we can have a postProcess(FacetArrays)
-  // That, together with getFacetArraysSource should allow ResultHandlers to
-  // efficiently obtain the values from the arrays directly
-  public abstract double getValueOf(FacetArrays arrays, int idx);
-  
   @Override
   public int hashCode() {
     return hashCode; 
   }
   
+  /**
+   * Sets the depth up to which to aggregate facets.
+   * 
+   * @see #getDepth()
+   */
   public void setDepth(int depth) {
     this.depth = depth;
   }
   
+  /**
+   * Sets the number of categories to label.
+   * 
+   * @see #getNumLabel()
+   */
   public void setNumLabel(int numLabel) {
     this.numLabel = numLabel;
   }
   
   /**
-   * @param resultMode the resultMode to set
+   * Sets the {@link ResultMode} for this request.
+   * 
    * @see #getResultMode()
    */
   public void setResultMode(ResultMode resultMode) {
     this.resultMode = resultMode;
   }
-  
+
+  /**
+   * Sets the {@link SortOrder} for this request.
+   * 
+   * @see #getSortOrder()
+   */
   public void setSortOrder(SortOrder sortOrder) {
     this.sortOrder = sortOrder;
   }
   
   @Override
   public String toString() {
-    return categoryPath.toString()+" nRes="+numResults+" nLbl="+numLabel;
+    return categoryPath.toString() + " nRes=" + numResults + " nLbl=" + numLabel;
   }
   
 }

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResult.java Tue Aug 13 04:06:18 2013
@@ -7,6 +7,7 @@ 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;
@@ -96,6 +97,11 @@ public class FacetResult {
         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) {
@@ -104,7 +110,10 @@ public class FacetResult {
             FacetResultNode parentNode = null;
             while (parent.length > 0 && (parentNode = mergedNodes.get(parent)) == null) {
               int parentOrd = taxoReader.getOrdinal(parent);
-              double parentValue = arrays != null ? fr.getFacetRequest().getValueOf(arrays, parentOrd) : -1;
+              double parentValue = -1;
+              if (arrays != null) {
+                parentValue = resolver.valueOf(parentOrd);
+              }
               parentNode = new FacetResultNode(parentOrd, parentValue);
               parentNode.label = parent;
               parentNode.subResults = new ArrayList<FacetResultNode>();
@@ -153,12 +162,7 @@ public class FacetResult {
         }
         FacetRequest dummy = new FacetRequest(min, frs.get(0).getFacetRequest().numResults) {
           @Override
-          public double getValueOf(FacetArrays arrays, int idx) {
-            throw new UnsupportedOperationException("not supported by this request");
-          }
-          
-          @Override
-          public FacetArraysSource getFacetArraysSource() {
+          public FacetsAggregator createFacetsAggregator(FacetIndexingParams fip) {
             throw new UnsupportedOperationException("not supported by this request");
           }
         };

Modified: lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultNode.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAccumulator.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsAggregator.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FacetsCollector.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/FastCountingFacetsAggregator.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/IntRollupFacetsAggregator.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/PerCategoryListAggregator.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetRequest.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/SumScoreFacetsAggregator.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java Tue Aug 13 04:06:18 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/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java?rev=1513336&r1=1513335&r2=1513336&view=diff
==============================================================================
--- lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java (original)
+++ lucene/dev/branches/lucene3069/lucene/facet/src/java/org/apache/lucene/facet/search/TopKInEachNodeHandler.java Tue Aug 13 04:06:18 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++;
       }
     }