You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/01/18 19:31:23 UTC

svn commit: r1435287 [13/41] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/analysis/icu/ dev-tools/maven/ dev-tools/maven/lucene/benchmark/ dev-tools/maven/solr/ dev-tools/...

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java Fri Jan 18 18:30:54 2013
@@ -17,26 +17,20 @@ package org.apache.lucene.facet.index;
  * limitations under the License.
  */
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.facet.index.params.CategoryListParams;
-import org.apache.lucene.facet.index.params.DefaultFacetIndexingParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter.OrdinalMap;
 import org.apache.lucene.index.AtomicReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.DocValues.Source;
+import org.apache.lucene.index.DocValues.Type;
 import org.apache.lucene.index.FilterAtomicReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.encoding.IntDecoder;
 import org.apache.lucene.util.encoding.IntEncoder;
 
@@ -44,8 +38,8 @@ import org.apache.lucene.util.encoding.I
  * A {@link FilterAtomicReader} for updating facets ordinal references,
  * based on an ordinal map. You should use this code in conjunction with merging
  * taxonomies - after you merge taxonomies, you receive an {@link OrdinalMap}
- * which maps the 'old' payloads to the 'new' ones. You can use that map to
- * re-map the payloads which contain the facets information (ordinals) either
+ * which maps the 'old' ordinals to the 'new' ones. You can use that map to
+ * re-map the doc values which contain the facets information (ordinals) either
  * before or while merging the indexes.
  * <p>
  * For re-mapping the ordinals during index merge, do the following:
@@ -71,10 +65,10 @@ import org.apache.lucene.util.encoding.I
  * @lucene.experimental
  */
 public class OrdinalMappingAtomicReader extends FilterAtomicReader {
+  
   private final int[] ordinalMap;
-  // a little obtuse: but we dont need to create Term objects this way
-  private final Map<String,Map<BytesRef,CategoryListParams>> termMap = 
-      new HashMap<String,Map<BytesRef,CategoryListParams>>(1);
+  
+  private final Map<String,CategoryListParams> dvFieldMap = new HashMap<String,CategoryListParams>();
   
   /**
    * Wraps an AtomicReader, mapping ordinals according to the ordinalMap.
@@ -82,7 +76,7 @@ public class OrdinalMappingAtomicReader 
    * OrdinalMappingAtomicReader(in, ordinalMap, new DefaultFacetIndexingParams())}
    */
   public OrdinalMappingAtomicReader(AtomicReader in, int[] ordinalMap) {
-    this(in, ordinalMap, new DefaultFacetIndexingParams());
+    this(in, ordinalMap, FacetIndexingParams.ALL_PARENTS);
   }
   
   /**
@@ -93,132 +87,85 @@ public class OrdinalMappingAtomicReader 
     super(in);
     this.ordinalMap = ordinalMap;
     for (CategoryListParams params: indexingParams.getAllCategoryListParams()) {
-      Term term = params.getTerm();
-      Map<BytesRef,CategoryListParams> fieldMap = termMap.get(term.field());
-      if (fieldMap == null) {
-        fieldMap = new HashMap<BytesRef,CategoryListParams>(1);
-        termMap.put(term.field(), fieldMap);
-      }
-      fieldMap.put(term.bytes(), params);
+      dvFieldMap.put(params.field, params);
     }
   }
 
   @Override
-  public Fields getTermVectors(int docID) throws IOException {
-    Fields fields = super.getTermVectors(docID);
-    if (fields == null) {
-      return null;
-    } else {
-      return new OrdinalMappingFields(fields);
+  public DocValues docValues(String field) throws IOException {
+    DocValues inner = super.docValues(field);
+    if (inner == null) {
+      return inner;
     }
-  }
-
-  @Override
-  public Fields fields() throws IOException {
-    Fields fields = super.fields();
-    if (fields == null) {
-      return null;
+    
+    CategoryListParams clp = dvFieldMap.get(field);
+    if (clp == null) {
+      return inner;
     } else {
-      return new OrdinalMappingFields(fields);
+      return new OrdinalMappingDocValues(inner, clp);
     }
   }
   
-  private class OrdinalMappingFields extends FilterFields {
+  private class OrdinalMappingDocValues extends DocValues {
 
-    public OrdinalMappingFields(Fields in) {
-      super(in);
+    private final CategoryListParams clp;
+    private final DocValues delegate;
+    
+    public OrdinalMappingDocValues(DocValues delegate, CategoryListParams clp) {
+      this.delegate = delegate;
+      this.clp = clp;
     }
 
     @Override
-    public Terms terms(String field) throws IOException {
-      Terms terms = super.terms(field);
-      if (terms == null) {
-        return terms;
-      }
-      Map<BytesRef,CategoryListParams> termsMap = termMap.get(field);
-      if (termsMap == null) {
-        return terms;
-      } else {
-        return new OrdinalMappingTerms(terms, termsMap);
-      }
-    }
-  }
-  
-  private class OrdinalMappingTerms extends FilterTerms {
-    private final Map<BytesRef,CategoryListParams> termsMap;
-    
-    public OrdinalMappingTerms(Terms in, Map<BytesRef,CategoryListParams> termsMap) {
-      super(in);
-      this.termsMap = termsMap;
+    protected Source loadSource() throws IOException {
+      return new OrdinalMappingSource(getType(), clp, delegate.getSource());
     }
 
     @Override
-    public TermsEnum iterator(TermsEnum reuse) throws IOException {
-      // TODO: should we reuse the inner termsenum?
-      return new OrdinalMappingTermsEnum(super.iterator(reuse), termsMap);
-    }
-  }
-  
-  private class OrdinalMappingTermsEnum extends FilterTermsEnum {
-    private final Map<BytesRef,CategoryListParams> termsMap;
-    
-    public OrdinalMappingTermsEnum(TermsEnum in, Map<BytesRef,CategoryListParams> termsMap) {
-      super(in);
-      this.termsMap = termsMap;
+    protected Source loadDirectSource() throws IOException {
+      return new OrdinalMappingSource(getType(), clp, delegate.getDirectSource());
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      // TODO: we could reuse our D&P enum if we need
-      DocsAndPositionsEnum inner = super.docsAndPositions(liveDocs, reuse, flags);
-      if (inner == null) {
-        return inner;
-      }
-      
-      CategoryListParams params = termsMap.get(term());
-      if (params == null) {
-        return inner;
-      }
-      
-      return new OrdinalMappingDocsAndPositionsEnum(inner, params);
+    public Type getType() {
+      return Type.BYTES_VAR_STRAIGHT;
     }
+    
   }
   
-  private class OrdinalMappingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
+  private class OrdinalMappingSource extends Source {
+
     private final IntEncoder encoder;
     private final IntDecoder decoder;
-    private final ByteArrayOutputStream os = new ByteArrayOutputStream();
-    private final BytesRef payloadOut = new BytesRef();
-
-    public OrdinalMappingDocsAndPositionsEnum(DocsAndPositionsEnum in, CategoryListParams params) {
-      super(in);
-      encoder = params.createEncoder();
+    private final IntsRef ordinals = new IntsRef(32);
+    private final Source delegate;
+    
+    protected OrdinalMappingSource(Type type, CategoryListParams clp, Source delegate) {
+      super(type);
+      this.delegate = delegate;
+      encoder = clp.createEncoder();
       decoder = encoder.createMatchingDecoder();
     }
-
+    
+    @SuppressWarnings("synthetic-access")
     @Override
-    public BytesRef getPayload() throws IOException {
-      BytesRef payload = super.getPayload();
-      if (payload == null) {
-        return payload;
+    public BytesRef getBytes(int docID, BytesRef ref) {
+      ref = delegate.getBytes(docID, ref);
+      if (ref == null || ref.length == 0) {
+        return ref;
       } else {
-        InputStream is = new ByteArrayInputStream(payload.bytes, payload.offset, payload.length);
-        decoder.reInit(is);
-        os.reset();
-        encoder.reInit(os);
-        long ordinal;
-        while ((ordinal = decoder.decode()) != IntDecoder.EOS) {
-          int newOrdinal = ordinalMap[(int)ordinal];
-          encoder.encode(newOrdinal);      
+        decoder.decode(ref, ordinals);
+        
+        // map the ordinals
+        for (int i = 0; i < ordinals.length; i++) {
+          ordinals.ints[i] = ordinalMap[ordinals.ints[i]];
         }
-        encoder.close();
-        // TODO (Facet): avoid copy?
-        byte out[] = os.toByteArray();
-        payloadOut.bytes = out;
-        payloadOut.offset = 0;
-        payloadOut.length = out.length;
-        return payloadOut;
+        
+        encoder.encode(ordinals, ref);
+        return ref;
       }
     }
+    
   }
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelOrdinalPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelOrdinalPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelOrdinalPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelOrdinalPolicy.java Fri Jan 18 18:30:54 2013
@@ -46,6 +46,7 @@ public class NonTopLevelOrdinalPolicy im
    *            A relevant taxonomyWriter object, with which ordinals sent to
    *            {@link #shouldAdd(int)} are examined.
    */
+  @Override
   public void init(TaxonomyWriter taxonomyWriter) {
     this.taxonomyWriter = taxonomyWriter;
   }
@@ -55,6 +56,7 @@ public class NonTopLevelOrdinalPolicy im
    * determine if a parent is root, there's a need for
    * {@link TaxonomyWriter#getParent(int)}.
    */
+  @Override
   public boolean shouldAdd(int ordinal) {
     if (ordinal > TaxonomyReader.ROOT_ORDINAL) {
       try {

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelPathPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelPathPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelPathPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/NonTopLevelPathPolicy.java Fri Jan 18 18:30:54 2013
@@ -37,7 +37,8 @@ public class NonTopLevelPathPolicy imple
    * Filters out (returns false) CategoryPaths equal or less than
    * {@link TaxonomyReader#ROOT_ORDINAL}. true otherwise.
    */
+  @Override
   public boolean shouldAdd(CategoryPath categoryPath) {
-    return categoryPath.length() >= DEFAULT_MINIMAL_SUBPATH_LENGTH;
+    return categoryPath.length >= DEFAULT_MINIMAL_SUBPATH_LENGTH;
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/OrdinalPolicy.java Fri Jan 18 18:30:54 2013
@@ -2,7 +2,8 @@ package org.apache.lucene.facet.index.ca
 
 import java.io.Serializable;
 
-import org.apache.lucene.facet.index.streaming.CategoryParentsStream;
+import org.apache.lucene.facet.search.FacetsAccumulator;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 
 /*
@@ -23,17 +24,43 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Filtering category ordinals in {@link CategoryParentsStream}, where a given
- * category ordinal is added to the stream, and than its parents are being added
- * one after the other using {@link TaxonomyWriter#getParent(int)}. <br>
- * That loop should have a stop point - the default approach (excluding the
- * ROOT) is implemented in {@link DefaultOrdinalPolicy}.
+ * A policy for adding category parent ordinals to the list of ordinals that are
+ * encoded for a given document. The default {@link #ALL_PARENTS} policy always
+ * adds all parents, where {@link #NO_PARENTS} never adds any parents.
  * 
  * @lucene.experimental
  */
 public interface OrdinalPolicy extends Serializable {
 
   /**
+   * An {@link OrdinalPolicy} which never stores parent ordinals. Useful if you
+   * only want to store the exact categories that were added to the document.
+   * Note that this is a rather expert policy, which requires a matching
+   * {@link FacetsAccumulator} that computes the weight of the parent categories
+   * on-the-fly.
+   */
+  public static final OrdinalPolicy NO_PARENTS = new OrdinalPolicy() {
+    @Override
+    public boolean shouldAdd(int ordinal) { return false; }
+
+    @Override
+    public void init(TaxonomyWriter taxonomyWriter) {}
+  };
+
+  /**
+   * An {@link OrdinalPolicy} which stores all parent ordinals, except
+   * {@link TaxonomyReader#ROOT_ORDINAL}. This is the default
+   * {@link OrdinalPolicy} and works with the default {@link FacetsAccumulator}.
+   */
+  public static final OrdinalPolicy ALL_PARENTS = new OrdinalPolicy() {
+    @Override
+    public boolean shouldAdd(int ordinal) { return ordinal > TaxonomyReader.ROOT_ORDINAL; }
+    
+    @Override
+    public void init(TaxonomyWriter taxonomyWriter) {}
+  };
+  
+  /**
    * Check whether a given category ordinal should be added to the stream.
    * 
    * @param ordinal

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/categorypolicy/PathPolicy.java Fri Jan 18 18:30:54 2013
@@ -2,7 +2,7 @@ package org.apache.lucene.facet.index.ca
 
 import java.io.Serializable;
 
-import org.apache.lucene.facet.index.streaming.CategoryParentsStream;
+import org.apache.lucene.facet.index.DrillDownStream;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 
 /*
@@ -23,17 +23,25 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Filtering category paths in {@link CategoryParentsStream}, where a given
- * category is added to the stream, and than all its parents are being
- * added one after the other by successively removing the last component. <br>
- * That loop should have a stop point - the default approach (excluding the
- * ROOT) is implemented in {@link DefaultOrdinalPolicy}.
+ * Determines which {@link CategoryPath categories} should be added as terms to
+ * the {@link DrillDownStream}. The default approach is implemented by
+ * {@link #ALL_CATEGORIES}.
  * 
  * @lucene.experimental
  */
 public interface PathPolicy extends Serializable {
 
   /**
+   * A {@link PathPolicy} which adds all {@link CategoryPath} that have at least
+   * one component (i.e. {@link CategoryPath#length} &gt; 0) to the categories
+   * stream.
+   */
+  public static final PathPolicy ALL_CATEGORIES = new PathPolicy() {
+    @Override
+    public boolean shouldAdd(CategoryPath categoryPath) { return categoryPath.length > 0; }
+  };
+  
+  /**
    * Check whether a given category path should be added to the stream.
    * 
    * @param categoryPath

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/package.html?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/package.html (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/package.html Fri Jan 18 18:30:54 2013
@@ -23,7 +23,7 @@
 
 Attachment of 
 {@link org.apache.lucene.facet.taxonomy.CategoryPath CategoryPath}'s 
-or {@link org.apache.lucene.facet.index.attributes.CategoryAttribute CategoryAttribute}'s 
+or {@link org.apache.lucene.facet.associations.CategoryAssociation CategoryAssociation}'s 
 to a given document using a 
 {@link org.apache.lucene.facet.taxonomy.TaxonomyWriter Taxonomy}.
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java Fri Jan 18 18:30:54 2013
@@ -3,19 +3,14 @@ package org.apache.lucene.facet.index.pa
 import java.io.IOException;
 import java.io.Serializable;
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-
 import org.apache.lucene.facet.search.CategoryListIterator;
-import org.apache.lucene.facet.search.PayloadIntDecodingIterator;
-import org.apache.lucene.facet.search.TotalFacetCounts;
+import org.apache.lucene.facet.search.DocValuesCategoryListIterator;
 import org.apache.lucene.facet.util.PartitionsUtils;
-import org.apache.lucene.util.encoding.DGapIntEncoder;
+import org.apache.lucene.util.encoding.DGapVInt8IntEncoder;
 import org.apache.lucene.util.encoding.IntDecoder;
 import org.apache.lucene.util.encoding.IntEncoder;
 import org.apache.lucene.util.encoding.SortingIntEncoder;
 import org.apache.lucene.util.encoding.UniqueValuesIntEncoder;
-import org.apache.lucene.util.encoding.VInt8IntEncoder;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -41,39 +36,26 @@ import org.apache.lucene.util.encoding.V
  */
 public class CategoryListParams implements Serializable {
 
-  /** The default term used to store the facets information. */
-  public static final Term DEFAULT_TERM = new Term("$facets", "$fulltree$");
+  /** The default field used to store the facets information. */
+  public static final String DEFAULT_FIELD = "$facets";
 
-  private final Term term;
+  public final String field;
 
   private final int hashCode;
 
-  /**
-   * Constructs a default category list parameters object, using
-   * {@link #DEFAULT_TERM}.
-   */
+  /** Constructs a default category list parameters object, using {@link #DEFAULT_FIELD}. */
   public CategoryListParams() {
-    this(DEFAULT_TERM);
+    this(DEFAULT_FIELD);
   }
 
-  /**
-   * Constructs a category list parameters object, using the given {@link Term}.
-   * @param term who's payload hold the category-list.
-   */
-  public CategoryListParams(Term term) {
-    this.term = term;
+  /** Constructs a category list parameters object, using the given field. */
+  public CategoryListParams(String field) {
+    this.field = field;
     // Pre-compute the hashCode because these objects are immutable.  Saves
     // some time on the comparisons later.
-    this.hashCode = term.hashCode();
+    this.hashCode = field.hashCode();
   }
   
-  /** 
-   * A {@link Term} who's payload holds the category-list. 
-   */
-  public final Term getTerm() {
-    return term;
-  }
-
   /**
    * Allows to override how categories are encoded and decoded. A matching
    * {@link IntDecoder} is provided by the {@link IntEncoder}.
@@ -95,14 +77,9 @@ public class CategoryListParams implemen
    * counting facets.
    */
   public IntEncoder createEncoder() {
-    return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
+    return new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapVInt8IntEncoder()));
   }
 
-  /**
-   * Equality is defined by the 'term' that defines this category list.  
-   * Sub-classes should override this method if a more complex calculation
-   * is needed to ensure equality. 
-   */
   @Override
   public boolean equals(Object o) {
     if (o == this) {
@@ -118,32 +95,19 @@ public class CategoryListParams implemen
     // The above hashcodes might equal each other in the case of a collision,
     // so at this point only directly term equality testing will settle
     // the equality test.
-    return this.term.equals(other.term);
+    return field.equals(other.field);
   }
 
-  /**
-   * Hashcode is similar to {@link #equals(Object)}, in that it uses
-   * the term that defines this category list to derive the hashcode.
-   * Subclasses need to ensure that equality/hashcode is correctly defined,
-   * or there could be side-effects in the {@link TotalFacetCounts} caching 
-   * mechanism (as the filename for a Total Facet Counts array cache 
-   * is dependent on the hashCode, so it should consistently return the same
-   * hash for identity).
-   */
   @Override
   public int hashCode() {
     return this.hashCode;
   }
 
-  /**
-   * Create the category list iterator for the specified partition.
-   */
-  public CategoryListIterator createCategoryListIterator(IndexReader reader,
-      int partition) throws IOException {
-    String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
-    Term payloadTerm = new Term(term.field(), categoryListTermStr);
-    return new PayloadIntDecodingIterator(reader, payloadTerm,
-        createEncoder().createMatchingDecoder());
+  /** Create the {@link CategoryListIterator} for the specified partition. */
+  public CategoryListIterator createCategoryListIterator(int partition) throws IOException {
+    String categoryListTermStr = PartitionsUtils.partitionName(partition);
+    String docValuesField = field + categoryListTermStr;
+    return new DocValuesCategoryListIterator(docValuesField, createEncoder().createMatchingDecoder());
   }
   
 }
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java Fri Jan 18 18:30:54 2013
@@ -1,9 +1,11 @@
 package org.apache.lucene.facet.index.params;
 
-import java.io.Serializable;
+import java.util.Collections;
+import java.util.List;
 
 import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
 import org.apache.lucene.facet.index.categorypolicy.PathPolicy;
+import org.apache.lucene.facet.search.FacetArrays;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 
 /*
@@ -24,75 +26,193 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Parameters on how facets are to be written to the index. 
- * For example, which fields and terms are used to refer to the indexed posting list.
- * <P>
- * If non-default parameters were used during indexing, the same parameters
- * must also be passed during faceted search. This requirement is analogous
- * to the requirement during search to know which fields were indexed, and which
- * Analyzer was used on the text.
+ * Defines parameters that are needed for facets indexing. Note that this class
+ * does not have any setters. That's because overriding the default parameters
+ * is considered expert. If you wish to override them, simply extend this class
+ * and override the relevant getter.
+ * 
+ * <p>
+ * <b>NOTE:</b> This class is also used during faceted search in order to e.g.
+ * know which field holds the drill-down terms or the fulltree posting.
+ * Therefore this class should be initialized once and you should refrain from
+ * changing it. Also note that if you make any changes to it (e.g. suddenly
+ * deciding that drill-down terms should be read from a different field) and use
+ * it on an existing index, things may not work as expected.
  * 
  * @lucene.experimental
  */
-public interface FacetIndexingParams extends Serializable {
+public class FacetIndexingParams {
+  
+  // the default CLP, can be a singleton
+  protected static final CategoryListParams DEFAULT_CATEGORY_LIST_PARAMS = new CategoryListParams();
 
   /**
-   * The name of the category-list to put this category in, or null if this
-   * category should not be aggregatable.
-   * <P>
-   * By default, all categories are written to the same category list, but
-   * applications which know in advance that in some situations only parts
-   * of the category hierarchy needs to be counted can divide the categories
-   * into two or more different category lists.
-   * <P>
-   * If null is returned for a category, it means that this category should
-   * not appear in any category list, and thus counts for it cannot be
-   * aggregated. This category can still be used for drill-down, even though
-   * the count for it is not known.
+   * A {@link FacetIndexingParams} which fixes {@link OrdinalPolicy} to
+   * {@link OrdinalPolicy#NO_PARENTS}. This is a singleton equivalent to new
+   * {@link #FacetIndexingParams()}.
+   */
+  public static final FacetIndexingParams ALL_PARENTS = new FacetIndexingParams();
+  
+  /**
+   * The default delimiter with which {@link CategoryPath#components} are
+   * concatenated when written to the index, e.g. as drill-down terms. If you
+   * choose to override it by overiding {@link #getFacetDelimChar()}, you should
+   * make sure that you return a character that's not found in any path
+   * component.
    */
-  public CategoryListParams getCategoryListParams(CategoryPath category);
+  public static final char DEFAULT_FACET_DELIM_CHAR = '\uF749';
+  
+  private final OrdinalPolicy ordinalPolicy = OrdinalPolicy.ALL_PARENTS;
+  private final PathPolicy pathPolicy = PathPolicy.ALL_CATEGORIES;
+  private final int partitionSize = Integer.MAX_VALUE;
+
+  protected final CategoryListParams clParams;
+
+  /**
+   * Initializes new default params. You should use this constructor only if you
+   * intend to override any of the getters, otherwise you can use
+   * {@link #ALL_PARENTS} to save unnecessary object allocations.
+   */
+  public FacetIndexingParams() {
+    this(DEFAULT_CATEGORY_LIST_PARAMS);
+  }
+
+  /** Initializes new params with the given {@link CategoryListParams}. */
+  public FacetIndexingParams(CategoryListParams categoryListParams) {
+    clParams = categoryListParams;
+  }
 
   /**
-   * Return info about all category lists in the index.
+   * Returns the {@link CategoryListParams} for this {@link CategoryPath}. The
+   * default implementation returns the same {@link CategoryListParams} for all
+   * categories (even if {@code category} is {@code null}).
    * 
-   * @see #getCategoryListParams(CategoryPath)
+   * @see PerDimensionIndexingParams
    */
-  public Iterable<CategoryListParams> getAllCategoryListParams();
-
-  // TODO (Facet): Add special cases of exact/non-exact category term-text
+  public CategoryListParams getCategoryListParams(CategoryPath category) {
+    return clParams;
+  }
 
   /**
-   * Return the drilldown Term-Text which does not need to do any allocations.
-   * The number of chars set is returned.
+   * Copies the text required to execute a drill-down query on the given
+   * category to the given {@code char[]}, and returns the number of characters
+   * that were written.
    * <p>
-   * Note: Make sure <code>buffer</code> is large enough.
-   * @see CategoryPath#charsNeededForFullPath()
+   * <b>NOTE:</b> You should make sure that the {@code char[]} is large enough,
+   * by e.g. calling {@link CategoryPath#fullPathLength()}.
    */
-  public int drillDownTermText(CategoryPath path, char[] buffer);
-
+  public int drillDownTermText(CategoryPath path, char[] buffer) {
+    return path.copyFullPath(buffer, 0, getFacetDelimChar());
+  }
+  
   /**
-   * Get the partition size.
-   * Same value should be used during the life time of an index.
-   * At search time this value is compared with actual taxonomy size and their minimum is used.
+   * Returns the size of a partition. <i>Partitions</i> allow you to divide
+   * (hence, partition) the categories space into small sets to e.g. improve RAM
+   * consumption during faceted search. For instance, {@code partitionSize=100K}
+   * would mean that if your taxonomy index contains 420K categories, they will
+   * be divided into 5 groups and at search time a {@link FacetArrays} will be
+   * allocated at the size of the partition.
+   * 
+   * <p>
+   * This is real advanced setting and should be changed with care. By default,
+   * all categories are put in one partition. You should modify this setting if
+   * you have really large taxonomies (e.g. 1M+ nodes).
+   */
+  public int getPartitionSize() {
+    return partitionSize;
+  }
+  
+  /**
+   * Returns a list of all {@link CategoryListParams categoryListParams} that
+   * are used for facets indexing.
    */
-  public int getPartitionSize();
+  public List<CategoryListParams> getAllCategoryListParams() {
+    return Collections.singletonList(clParams);
+  }
 
-  /** 
-   * Get the policy for indexing category <b>paths</b>, 
-   * used for deciding how "high" to climb in taxonomy 
-   * from a category when ingesting its category paths. 
-   */
-  public PathPolicy getPathPolicy();
+  /**
+   * Returns the {@link OrdinalPolicy} that is used during indexing. By default
+   * returns {@link OrdinalPolicy#ALL_PARENTS} which means that the full
+   * hierarchy will be stored for every document.
+   */
+  public OrdinalPolicy getOrdinalPolicy() {
+    return ordinalPolicy;
+  }
 
-  /** 
-   * Get the policy for indexing category <b>ordinals</b>, 
-   * used for deciding how "high" to climb in taxonomy 
-   * from a category when ingesting its ordinals 
+  /**
+   * Returns the {@link PathPolicy} that is used during indexing. By default
+   * returns {@link PathPolicy#ALL_CATEGORIES} which means that the full
+   * hierarchy is added as drill-down terms for every document.
+   */
+  public PathPolicy getPathPolicy() {
+    return pathPolicy;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((clParams == null) ? 0 : clParams.hashCode());
+    result = prime * result + ((ordinalPolicy == null) ? 0 : ordinalPolicy.hashCode());
+    result = prime * result + partitionSize;
+    result = prime * result + ((pathPolicy == null) ? 0 : pathPolicy.hashCode());
+    
+    for (CategoryListParams clp : getAllCategoryListParams()) {
+      result ^= clp.hashCode();
+    }
+    
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof FacetIndexingParams)) {
+      return false;
+    }
+    FacetIndexingParams other = (FacetIndexingParams) obj;
+    if (clParams == null) {
+      if (other.clParams != null) {
+        return false;
+      }
+    } else if (!clParams.equals(other.clParams)) {
+      return false;
+    }
+    if (ordinalPolicy == null) {
+      if (other.ordinalPolicy != null) {
+        return false;
+      }
+    } else if (!ordinalPolicy.equals(other.ordinalPolicy)) {
+      return false;
+    }
+    if (partitionSize != other.partitionSize) {
+      return false;
+    }
+    if (pathPolicy == null) {
+      if (other.pathPolicy != null) {
+        return false;
+      }
+    } else if (!pathPolicy.equals(other.pathPolicy)) {
+      return false;
+    }
+    
+    Iterable<CategoryListParams> cLs = getAllCategoryListParams();
+    Iterable<CategoryListParams> otherCLs = other.getAllCategoryListParams();
+    
+    return cLs.equals(otherCLs);
+  }
+
+  /**
+   * Returns the delimiter character used internally for concatenating category
+   * path components, e.g. for drill-down terms.
    */
-  public OrdinalPolicy getOrdinalPolicy();
-  
-  /** 
-   * Get the delimiter character used internally for drill-down terms 
-   */ 
-  public char getFacetDelimChar();
+  public char getFacetDelimChar() {
+    return DEFAULT_FACET_DELIM_CHAR;
+  }
+
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java Fri Jan 18 18:30:54 2013
@@ -2,7 +2,9 @@ package org.apache.lucene.facet.index.pa
 
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 
@@ -24,79 +26,71 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * A FacetIndexingParams that utilizes different category lists, defined by the
- * dimension specified CategoryPaths (see
- * {@link PerDimensionIndexingParams#addCategoryListParams(CategoryPath, CategoryListParams)}
+ * A {@link FacetIndexingParams} that utilizes different category lists, defined
+ * by the dimension specified by a {@link CategoryPath category} (see
+ * {@link #PerDimensionIndexingParams(Map, CategoryListParams)}.
  * <p>
  * A 'dimension' is defined as the first or "zero-th" component in a
- * CategoryPath. For example, if a CategoryPath is defined as
- * "/Author/American/Mark Twain", then the dimension is "Author".
- * <p>
- * This class also uses the 'default' CategoryListParams (as specified by
- * {@link CategoryListParams#CategoryListParams()} when
- * {@link #getCategoryListParams(CategoryPath)} is called for a CategoryPath
- * whose dimension component has not been specifically defined.
+ * {@link CategoryPath}. For example, if a category is defined as
+ * "Author/American/Mark Twain", then the dimension would be "Author".
  * 
  * @lucene.experimental
  */
-public class PerDimensionIndexingParams extends DefaultFacetIndexingParams {
+public class PerDimensionIndexingParams extends FacetIndexingParams {
 
-  // "Root" or "first component" of a Category Path maps to a
-  // CategoryListParams
-  private final Map<String, CategoryListParams> clParamsMap = new HashMap<String, CategoryListParams>();
+  private final Map<String, CategoryListParams> clParamsMap;
 
   /**
-   * Construct with the default {@link CategoryListParams} as the default
-   * CategoryListParams for unspecified CategoryPaths.
+   * Initializes a new instance with the given dimension-to-params mapping. The
+   * dimension is considered as what's returned by
+   * {@link CategoryPath#components cp.components[0]}.
+   * 
+   * <p>
+   * <b>NOTE:</b> for any dimension whose {@link CategoryListParams} is not
+   * defined in the mapping, a default {@link CategoryListParams} will be used.
+   * 
+   * @see #PerDimensionIndexingParams(Map, CategoryListParams)
    */
-  public PerDimensionIndexingParams() {
-    this(new CategoryListParams());
+  public PerDimensionIndexingParams(Map<CategoryPath, CategoryListParams> paramsMap) {
+    this(paramsMap, DEFAULT_CATEGORY_LIST_PARAMS);
   }
 
   /**
-   * Construct with the included categoryListParams as the default
-   * CategoryListParams for unspecified CategoryPaths.
-   * 
-   * @param categoryListParams
-   *            the default categoryListParams to use
+   * Same as {@link #PerDimensionIndexingParams(Map)}, only the given
+   * {@link CategoryListParams} will be used for any dimension that is not
+   * specified in the given mapping.
    */
-  public PerDimensionIndexingParams(CategoryListParams categoryListParams) {
+  public PerDimensionIndexingParams(Map<CategoryPath, CategoryListParams> paramsMap, 
+      CategoryListParams categoryListParams) {
     super(categoryListParams);
+    clParamsMap = new HashMap<String,CategoryListParams>();
+    for (Entry<CategoryPath, CategoryListParams> e : paramsMap.entrySet()) {
+      clParamsMap.put(e.getKey().components[0], e.getValue());
+    }
   }
 
-  /**
-   * Get all the categoryListParams, including the default.
-   */
   @Override
-  public Iterable<CategoryListParams> getAllCategoryListParams() {
-    ArrayList<CategoryListParams> vals = 
-      new ArrayList<CategoryListParams>(clParamsMap.values());
-    for (CategoryListParams clp : super.getAllCategoryListParams()) {
-      vals.add(clp);
-    }
+  public List<CategoryListParams> getAllCategoryListParams() {
+    ArrayList<CategoryListParams> vals = new ArrayList<CategoryListParams>(clParamsMap.values());
+    vals.add(clParams); // add the default too
     return vals;
   }
 
   /**
-   * Get the CategoryListParams based on the dimension or "zero-th category"
-   * of the specified CategoryPath.
+   * Returns the {@link CategoryListParams} for the corresponding dimension
+   * which is returned by {@code category.getComponent(0)}. If {@code category}
+   * is {@code null}, or was not specified in the map given to the constructor,
+   * returns the default {@link CategoryListParams}.
    */
   @Override
   public CategoryListParams getCategoryListParams(CategoryPath category) {
     if (category != null) {
-      CategoryListParams clParams = clParamsMap.get(category.getComponent(0));
+      CategoryListParams clParams = clParamsMap.get(category.components[0]);
       if (clParams != null) {
         return clParams;
       }
     }
-    return super.getCategoryListParams(category);
+    return clParams;
   }
 
-  /**
-   * Add a CategoryListParams for a given CategoryPath's dimension or
-   * "zero-th" category.
-   */
-  public void addCategoryListParams(CategoryPath category, CategoryListParams clParams) {
-    clParamsMap.put(category.getComponent(0), clParams);
-  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/AdaptiveFacetsAccumulator.java Fri Jan 18 18:30:54 2013
@@ -2,18 +2,15 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Random;
-
-import org.apache.lucene.index.IndexReader;
 
 import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.results.FacetResultNode;
 import org.apache.lucene.facet.search.sampling.RandomSampler;
-import org.apache.lucene.facet.search.sampling.RepeatableSampler;
 import org.apache.lucene.facet.search.sampling.Sampler;
 import org.apache.lucene.facet.search.sampling.SamplingAccumulator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.index.IndexReader;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -53,20 +50,20 @@ public final class AdaptiveFacetsAccumul
    * Create an {@link AdaptiveFacetsAccumulator} 
    * @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader)
    */
-  public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
+  public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader, 
       TaxonomyReader taxonomyReader) {
     super(searchParams, indexReader, taxonomyReader);
   }
 
   /**
-   * Create an {@link AdaptiveFacetsAccumulator} 
-   * @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams, IndexReader, TaxonomyReader, 
-   *                               IntArrayAllocator, FloatArrayAllocator)
+   * Create an {@link AdaptiveFacetsAccumulator}
+   * 
+   * @see StandardFacetsAccumulator#StandardFacetsAccumulator(FacetSearchParams,
+   *      IndexReader, TaxonomyReader, FacetArrays)
    */
   public AdaptiveFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
-      TaxonomyReader taxonomyReader, IntArrayAllocator intArrayAllocator,
-      FloatArrayAllocator floatArrayAllocator) {
-    super(searchParams, indexReader, taxonomyReader, intArrayAllocator, floatArrayAllocator);
+      TaxonomyReader taxonomyReader, FacetArrays facetArrays) {
+    super(searchParams, indexReader, taxonomyReader, facetArrays);
   }
 
   /**

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java Fri Jan 18 18:30:54 2013
@@ -2,6 +2,9 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.util.IntsRef;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -20,50 +23,34 @@ import java.io.IOException;
  */
 
 /**
- * An interface for iterating over a "category list", i.e., the list of
- * categories per document.
+ * An interface for obtaining the category ordinals of documents.
+ * {@link #getOrdinals(int, IntsRef)} calls are done with document IDs that are
+ * local to the reader given to {@link #setNextReader(AtomicReaderContext)}.
  * <p>
- * <b>NOTE:</b>
- * <ul>
- * <li>This class operates as a key to a Map. Appropriate implementation of
- * <code>hashCode()</code> and <code>equals()</code> must be provided.
- * <li>{@link #init()} must be called before you consume any categories, or call
- * {@link #skipTo(int)}.
- * <li>{@link #skipTo(int)} must be called before any calls to
- * {@link #nextCategory()}.
- * <li>{@link #nextCategory()} returns values &lt; {@link Integer#MAX_VALUE}, so
- * you can use it as a stop condition.
- * </ul>
+ * <b>NOTE:</b> this class operates as a key to a map, and therefore you should
+ * implement {@code equals()} and {@code hashCode()} for proper behavior.
  * 
  * @lucene.experimental
  */
 public interface CategoryListIterator {
 
   /**
-   * Initializes the iterator. This method must be called before any calls to
-   * {@link #skipTo(int)}, and its return value indicates whether there are
-   * any relevant documents for this iterator. If it returns false, any call
-   * to {@link #skipTo(int)} will return false as well.<br>
-   * <b>NOTE:</b> calling this method twice may result in skipping over
-   * documents for some implementations. Also, calling it again after all
-   * documents were consumed may yield unexpected behavior.
-   */
-  public boolean init() throws IOException;
-
-  /**
-   * Skips forward to document docId. Returns true iff this document exists
-   * and has any categories. This method must be called before calling
-   * {@link #nextCategory()} for a particular document.<br>
-   * <b>NOTE:</b> Users should call this method with increasing docIds, and
-   * implementations can assume that this is the case.
+   * Sets the {@link AtomicReaderContext} for which
+   * {@link #getOrdinals(int, IntsRef)} calls will be made. Returns true iff any
+   * of the documents in this reader have category ordinals. This method must be
+   * called before any calls to {@link #getOrdinals(int, IntsRef)}.
    */
-  public boolean skipTo(int docId) throws IOException;
-
+  public boolean setNextReader(AtomicReaderContext context) throws IOException;
+  
   /**
-   * Returns the next category for the current document that is set through
-   * {@link #skipTo(int)}, or a number higher than {@link Integer#MAX_VALUE}.
-   * No assumptions can be made on the order of the categories.
+   * Stores the category ordinals of the given document ID in the given
+   * {@link IntsRef}, starting at position 0 upto {@link IntsRef#length}. Grows
+   * the {@link IntsRef} if it is not large enough.
+   * 
+   * <p>
+   * <b>NOTE:</b> if the requested document does not have category ordinals
+   * associated with it, {@link IntsRef#length} is set to zero.
    */
-  public long nextCategory() throws IOException;
-
+  public void getOrdinals(int docID, IntsRef ints) throws IOException;
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDown.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDown.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDown.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/DrillDown.java Fri Jan 18 18:30:54 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.search;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.BooleanClause.Occur;
@@ -29,7 +30,14 @@ import org.apache.lucene.facet.taxonomy.
  */
 
 /**
- * Creation of drill down term or query.
+ * Utility class for creating drill-down {@link Query queries} or {@link Term
+ * terms} over {@link CategoryPath}. This can be used to e.g. narrow down a
+ * user's search to selected categories.
+ * <p>
+ * <b>NOTE:</b> if you choose to create your own {@link Query} by calling
+ * {@link #term}, it is recommended to wrap it with {@link ConstantScoreQuery}
+ * and set the {@link ConstantScoreQuery#setBoost(float) boost} to {@code 0.0f},
+ * so that it does not affect the scores of the documents.
  * 
  * @lucene.experimental
  */
@@ -42,69 +50,60 @@ public final class DrillDown {
     return term(sParams.getFacetIndexingParams(), path);
   }
 
-  /**
-   * Return a term for drilling down into a category.
-   */
+  /** Return a drill-down {@link Term} for a category. */
   public static final Term term(FacetIndexingParams iParams, CategoryPath path) {
     CategoryListParams clp = iParams.getCategoryListParams(path);
-    char[] buffer = new char[path.charsNeededForFullPath()];
+    char[] buffer = new char[path.fullPathLength()];
     iParams.drillDownTermText(path, buffer);
-    return new Term(clp.getTerm().field(), String.valueOf(buffer));
+    return new Term(clp.field, String.valueOf(buffer));
   }
   
   /**
-   * Return a query for drilling down into all given categories (AND).
-   * @see #term(FacetSearchParams, CategoryPath)
-   * @see #query(FacetSearchParams, Query, CategoryPath...)
+   * Wraps a given {@link Query} as a drill-down query over the given
+   * categories, assuming all are required (e.g. {@code AND}). You can construct
+   * a query with different modes (such as {@code OR} or {@code AND} of
+   * {@code ORs}) by creating a {@link BooleanQuery} and call this method
+   * several times. Make sure to wrap the query in that case by
+   * {@link ConstantScoreQuery} and set the boost to 0.0f, so that it doesn't
+   * affect scoring.
+   * <p>
+   * <b>NOTE:</b> {@code baseQuery} can be {@code null}, in which case only the
+   * {@link Query} over the categories will is returned.
    */
-  public static final Query query(FacetIndexingParams iParams, CategoryPath... paths) {
-    if (paths==null || paths.length==0) {
+  public static final Query query(FacetIndexingParams iParams, Query baseQuery, CategoryPath... paths) {
+    if (paths == null || paths.length == 0) {
       throw new IllegalArgumentException("Empty category path not allowed for drill down query!");
     }
-    if (paths.length==1) {
-      return new TermQuery(term(iParams, paths[0]));
+    
+    final Query q;
+    if (paths.length == 1) {
+      q = new TermQuery(term(iParams, paths[0]));
+    } else {
+      BooleanQuery bq = new BooleanQuery(true); // disable coord
+      for (CategoryPath cp : paths) {
+        bq.add(new TermQuery(term(iParams, cp)), Occur.MUST);
+      }
+      q = bq;
     }
-    BooleanQuery res = new BooleanQuery();
-    for (CategoryPath cp : paths) {
-      res.add(new TermQuery(term(iParams, cp)), Occur.MUST);
-    }
-    return res;
-  }
-  
-  /**
-   * Return a query for drilling down into all given categories (AND).
-   * @see #term(FacetSearchParams, CategoryPath)
-   * @see #query(FacetSearchParams, Query, CategoryPath...)
-   */
-  public static final Query query(FacetSearchParams sParams, CategoryPath... paths) {
-    return query(sParams.getFacetIndexingParams(), paths);
-  }
 
-  /**
-   * Turn a base query into a drilling-down query for all given category paths (AND).
-   * @see #query(FacetIndexingParams, CategoryPath...)
-   */
-  public static final Query query(FacetIndexingParams iParams, Query baseQuery, CategoryPath... paths) {
-    BooleanQuery res = new BooleanQuery();
-    res.add(baseQuery, Occur.MUST);
-    res.add(query(iParams, paths), Occur.MUST);
-    return res;
+    final ConstantScoreQuery drillDownQuery = new ConstantScoreQuery(q);
+    drillDownQuery.setBoost(0.0f);
+
+    if (baseQuery == null) {
+      return drillDownQuery;
+    } else {
+      BooleanQuery res = new BooleanQuery(true);
+      res.add(baseQuery, Occur.MUST);
+      res.add(drillDownQuery, Occur.MUST);
+      return res;
+    }
   }
 
   /**
-   * Turn a base query into a drilling-down query for all given category paths (AND).
-   * @see #query(FacetSearchParams, CategoryPath...)
+   * @see #query(FacetIndexingParams, Query, CategoryPath...)
    */
   public static final Query query(FacetSearchParams sParams, Query baseQuery, CategoryPath... paths) {
     return query(sParams.getFacetIndexingParams(), baseQuery, paths);
   }
 
-  /**
-   * Turn a base query into a drilling-down query using the default {@link FacetSearchParams}  
-   * @see #query(FacetSearchParams, Query, CategoryPath...)
-   */
-  public static final Query query(Query baseQuery, CategoryPath... paths) {
-    return query(new FacetSearchParams(), baseQuery, paths);
-  }
-
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetArrays.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetArrays.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetArrays.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetArrays.java Fri Jan 18 18:30:54 2013
@@ -18,74 +18,66 @@ package org.apache.lucene.facet.search;
  */
 
 /**
- * Provider of arrays used for facet operations such as counting.
+ * Provider of arrays used for facets aggregation. Returns either an
+ * {@code int[]} or {@code float[]} of the specified array length. When the
+ * arrays are no longer needed, you should call {@link #free()}, so that e.g.
+ * they will be reclaimed.
+ * 
+ * <p>
+ * <b>NOTE:</b> if you need to reuse the allocated arrays between search
+ * requests, use {@link ReusingFacetArrays}.
+ * 
+ * <p>
+ * <b>NOTE:</b> this class is not thread safe. You typically allocate it per
+ * search.
  * 
  * @lucene.experimental
  */
 public class FacetArrays {
 
-  private int[] intArray;
-  private float[] floatArray;
-  private IntArrayAllocator intArrayAllocator;
-  private FloatArrayAllocator floatArrayAllocator;
-  private int arraysLength;
-
-  /**
-   * Create a FacetArrays with certain array allocators.
-   * @param intArrayAllocator allocator for int arrays.
-   * @param floatArrayAllocator allocator for float arrays.
-   */
-  public FacetArrays(IntArrayAllocator intArrayAllocator,
-                      FloatArrayAllocator floatArrayAllocator) {
-    this.intArrayAllocator = intArrayAllocator;
-    this.floatArrayAllocator = floatArrayAllocator;
+  private int[] ints;
+  private float[] floats;
+  
+  public final int arrayLength;
+
+  /** Arrays will be allocated at the specified length. */
+  public FacetArrays(int arrayLength) {
+    this.arrayLength = arrayLength;
   }
-
-  /**
-   * Notify allocators that they can free arrays allocated 
-   * on behalf of this FacetArrays object. 
-   */
-  public void free() {
-    if (intArrayAllocator!=null) {
-      intArrayAllocator.free(intArray);
-      // Should give up handle to the array now
-      // that it is freed.
-      intArray = null;
-    }
-    if (floatArrayAllocator!=null) {
-      floatArrayAllocator.free(floatArray);
-      // Should give up handle to the array now
-      // that it is freed.
-      floatArray = null;
-    }
-    arraysLength = 0;
+  
+  protected float[] newFloatArray() {
+    return new float[arrayLength];
   }
-
+  
+  protected int[] newIntArray() {
+    return new int[arrayLength];
+  }
+  
+  protected void doFree(float[] floats, int[] ints) {
+  }
+  
   /**
-   * Obtain an int array, e.g. for facet counting. 
+   * Notifies that the arrays obtained from {@link #getIntArray()}
+   * or {@link #getFloatArray()} are no longer needed and can be freed.
    */
-  public int[] getIntArray() {
-    if (intArray == null) {
-      intArray = intArrayAllocator.allocate();
-      arraysLength = intArray.length;
-    }
-    return intArray;
+  public final void free() {
+    doFree(floats, ints);
+    ints = null;
+    floats = null;
   }
 
-  /** Obtain a float array, e.g. for evaluating facet association values. */
-  public float[] getFloatArray() {
-    if (floatArray == null) {
-      floatArray = floatArrayAllocator.allocate();
-      arraysLength = floatArray.length;
+  public final int[] getIntArray() {
+    if (ints == null) {
+      ints = newIntArray();
     }
-    return floatArray;
+    return ints;
   }
 
-  /**
-   * Return the arrays length
-   */
-  public int getArraysLength() {
-    return arraysLength;
+  public final float[] getFloatArray() {
+    if (floats == null) {
+      floats = newFloatArray();
+    }
+    return floats;
   }
 
-}
\ No newline at end of file
+}

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/FacetResultsHandler.java Fri Jan 18 18:30:54 2013
@@ -154,7 +154,7 @@ public abstract class FacetResultsHandle
    *          offset in input arrays where partition starts
    */
   protected boolean isSelfPartition (int ordinal, FacetArrays facetArrays, int offset) {
-    int partitionSize = facetArrays.getArraysLength();
+    int partitionSize = facetArrays.arrayLength;
     return ordinal / partitionSize == offset / partitionSize;
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/ScoredDocIdCollector.java Fri Jan 18 18:30:54 2013
@@ -62,15 +62,18 @@ public abstract class ScoredDocIdCollect
     }
 
     @Override
-    public ScoredDocIDsIterator scoredDocIdsIterator() {
+    protected ScoredDocIDsIterator scoredDocIdsIterator() {
       return new ScoredDocIDsIterator() {
 
         private DocIdSetIterator docIdsIter = docIds.iterator();
         private int nextDoc;
 
+        @Override
         public int getDocID() { return nextDoc; }
+        @Override
         public float getScore() { return defaultScore; }
 
+        @Override
         public boolean next() {
           try {
             nextDoc = docIdsIter.nextDoc();
@@ -126,16 +129,19 @@ public abstract class ScoredDocIdCollect
     }
 
     @Override
-    public ScoredDocIDsIterator scoredDocIdsIterator() {
+    protected ScoredDocIDsIterator scoredDocIdsIterator() {
       return new ScoredDocIDsIterator() {
 
         private DocIdSetIterator docIdsIter = docIds.iterator();
         private int nextDoc;
         private int scoresIdx = -1;
 
+        @Override
         public int getDocID() { return nextDoc; }
+        @Override
         public float getScore() { return scores[scoresIdx]; }
 
+        @Override
         public boolean next() {
           try {
             nextDoc = docIdsIter.nextDoc();
@@ -183,8 +189,7 @@ public abstract class ScoredDocIdCollect
    *        do not require scoring, it is better to set it to <i>false</i>.
    */
   public static ScoredDocIdCollector create(int maxDoc, boolean enableScoring) {
-    return enableScoring   ? new ScoringDocIdCollector(maxDoc)
-                          : new NonScoringDocIdCollector(maxDoc);
+    return enableScoring ? new ScoringDocIdCollector(maxDoc) : new NonScoringDocIdCollector(maxDoc);
   }
 
   private ScoredDocIdCollector(int maxDoc) {
@@ -192,25 +197,29 @@ public abstract class ScoredDocIdCollect
     docIds = new FixedBitSet(maxDoc);
   }
 
+  protected abstract ScoredDocIDsIterator scoredDocIdsIterator() throws IOException;
+
   /** Returns the default score used when scoring is disabled. */
   public abstract float getDefaultScore();
 
   /** Set the default score. Only applicable if scoring is disabled. */
   public abstract void setDefaultScore(float defaultScore);
 
-  public abstract ScoredDocIDsIterator scoredDocIdsIterator() throws IOException;
 
   public ScoredDocIDs getScoredDocIDs() {
     return new ScoredDocIDs() {
 
+      @Override
       public ScoredDocIDsIterator iterator() throws IOException {
         return scoredDocIdsIterator();
       }
 
+      @Override
       public DocIdSet getDocIDs() {
         return docIds;
       }
 
+      @Override
       public int size() {
         return numDocIds;
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java Fri Jan 18 18:30:54 2013
@@ -3,21 +3,25 @@ package org.apache.lucene.facet.search;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
-import org.apache.lucene.index.IndexReader;
-
+import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
-import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.params.FacetRequest;
+import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.results.IntermediateFacetResult;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 import org.apache.lucene.facet.util.PartitionsUtils;
 import org.apache.lucene.facet.util.ScoredDocIdsUtils;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -62,8 +66,7 @@ public class StandardFacetsAccumulator e
 
   private static final Logger logger = Logger.getLogger(StandardFacetsAccumulator.class.getName());
 
-  protected final IntArrayAllocator intArrayAllocator;
-  protected final FloatArrayAllocator floatArrayAllocator;
+  protected final FacetArrays facetArrays;
 
   protected int partitionSize;
   protected int maxPartitions;
@@ -74,31 +77,25 @@ public class StandardFacetsAccumulator e
   private Object accumulateGuard;
 
   public StandardFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
-      TaxonomyReader taxonomyReader, IntArrayAllocator intArrayAllocator,
-      FloatArrayAllocator floatArrayAllocator) {
-    
+      TaxonomyReader taxonomyReader, FacetArrays facetArrays) {
     super(searchParams,indexReader,taxonomyReader);
-    int realPartitionSize = intArrayAllocator == null || floatArrayAllocator == null 
-              ? PartitionsUtils.partitionSize(searchParams, taxonomyReader) : -1; // -1 if not needed.
-    this.intArrayAllocator = intArrayAllocator != null 
-        ? intArrayAllocator
-        // create a default one if null was provided
-        : new IntArrayAllocator(realPartitionSize, 1);
-    this.floatArrayAllocator = floatArrayAllocator != null 
-        ? floatArrayAllocator
-        // create a default one if null provided
-        : new FloatArrayAllocator(realPartitionSize, 1);
+    
+    if (facetArrays == null) {
+      throw new IllegalArgumentException("facetArrays cannot be null");
+    }
+    
+    this.facetArrays = facetArrays;
     // can only be computed later when docids size is known
     isUsingComplements = false;
-    partitionSize = PartitionsUtils.partitionSize(searchParams, taxonomyReader);
+    partitionSize = PartitionsUtils.partitionSize(searchParams.getFacetIndexingParams(), taxonomyReader);
     maxPartitions = (int) Math.ceil(this.taxonomyReader.getSize() / (double) partitionSize);
     accumulateGuard = new Object();
   }
 
-  public StandardFacetsAccumulator(FacetSearchParams searchParams, IndexReader indexReader,
-      TaxonomyReader taxonomyReader) {
-    
-    this(searchParams, indexReader, taxonomyReader, null, null);
+  public StandardFacetsAccumulator(FacetSearchParams searchParams,
+      IndexReader indexReader, TaxonomyReader taxonomyReader) {
+    this(searchParams, indexReader, taxonomyReader, new FacetArrays(
+        PartitionsUtils.partitionSize(searchParams.getFacetIndexingParams(), taxonomyReader)));
   }
 
   @Override
@@ -114,9 +111,8 @@ public class StandardFacetsAccumulator e
 
       if (isUsingComplements) {
         try {
-          totalFacetCounts = TotalFacetCountsCache.getSingleton()
-            .getTotalCounts(indexReader, taxonomyReader,
-                searchParams.getFacetIndexingParams(), searchParams.getClCache());
+          totalFacetCounts = TotalFacetCountsCache.getSingleton().getTotalCounts(indexReader, taxonomyReader, 
+              searchParams.getFacetIndexingParams());
           if (totalFacetCounts != null) {
             docids = ScoredDocIdsUtils.getComplementSet(docids, indexReader);
           } else {
@@ -142,18 +138,12 @@ public class StandardFacetsAccumulator e
           isUsingComplements = false;
         } catch (Exception e) {
           // give up: this should not happen!
-          IOException ioEx = new IOException(
-              "PANIC: Got unexpected exception while trying to get/calculate total counts: "
-              +e.getMessage());
-          ioEx.initCause(e);
-          throw ioEx;
+          throw new IOException("PANIC: Got unexpected exception while trying to get/calculate total counts", e);
         }
       }
 
       docids = actualDocsToAccumulate(docids);
 
-      FacetArrays facetArrays = new FacetArrays(intArrayAllocator, floatArrayAllocator);
-
       HashMap<FacetRequest, IntermediateFacetResult> fr2tmpRes = new HashMap<FacetRequest, IntermediateFacetResult>();
 
       try {
@@ -165,19 +155,21 @@ public class StandardFacetsAccumulator e
           int offset = part * partitionSize;
 
           // for each partition we go over all requests and handle
-          // each, where
-          // the request maintains the merged result.
-          // In this implementation merges happen after each
-          // partition,
+          // each, where the request maintains the merged result.
+          // In this implementation merges happen after each partition,
           // but other impl could merge only at the end.
+          final HashSet<FacetRequest> handledRequests = new HashSet<FacetRequest>();
           for (FacetRequest fr : searchParams.getFacetRequests()) {
-            FacetResultsHandler frHndlr = fr.createFacetResultsHandler(taxonomyReader);
-            IntermediateFacetResult res4fr = frHndlr.fetchPartitionResult(facetArrays, offset);
-            IntermediateFacetResult oldRes = fr2tmpRes.get(fr);
-            if (oldRes != null) {
-              res4fr = frHndlr.mergeResults(oldRes, res4fr);
-            }
-            fr2tmpRes.put(fr, res4fr);
+            // Handle and merge only facet requests which were not already handled.  
+            if (handledRequests.add(fr)) {
+              FacetResultsHandler frHndlr = fr.createFacetResultsHandler(taxonomyReader);
+              IntermediateFacetResult res4fr = frHndlr.fetchPartitionResult(facetArrays, offset);
+              IntermediateFacetResult oldRes = fr2tmpRes.get(fr);
+              if (oldRes != null) {
+                res4fr = frHndlr.mergeResults(oldRes, res4fr);
+              }
+              fr2tmpRes.put(fr, res4fr);
+            } 
           }
         }
       } finally {
@@ -188,11 +180,11 @@ public class StandardFacetsAccumulator e
       List<FacetResult> res = new ArrayList<FacetResult>();
       for (FacetRequest fr : searchParams.getFacetRequests()) {
         FacetResultsHandler frHndlr = fr.createFacetResultsHandler(taxonomyReader);
-        IntermediateFacetResult tmpResult = fr2tmpRes.get(fr); 
+        IntermediateFacetResult tmpResult = fr2tmpRes.get(fr);
         if (tmpResult == null) {
           continue; // do not add a null to the list.
         }
-        FacetResult facetRes = frHndlr.renderFacetResult(tmpResult); 
+        FacetResult facetRes = frHndlr.renderFacetResult(tmpResult);
         // final labeling if allowed (because labeling is a costly operation)
         if (isAllowLabeling()) {
           frHndlr.labelResult(facetRes);
@@ -222,18 +214,15 @@ public class StandardFacetsAccumulator e
 
   /** Check if it is worth to use complements */
   protected boolean shouldComplement(ScoredDocIDs docids) {
-    return 
-      mayComplement() && 
-      (docids.size() > indexReader.numDocs() * getComplementThreshold()) ;
+    return mayComplement() && (docids.size() > indexReader.numDocs() * getComplementThreshold()) ;
   }
 
   /**
    * Iterate over the documents for this partition and fill the facet arrays with the correct
    * count/complement count/value.
-   * @throws IOException If there is a low-level I/O error.
    */
-  private final void fillArraysForPartition(ScoredDocIDs docids,
-      FacetArrays facetArrays, int partition) throws IOException {
+  private final void fillArraysForPartition(ScoredDocIDs docids, FacetArrays facetArrays, int partition) 
+      throws IOException {
     
     if (isUsingComplements) {
       initArraysByTotalCounts(facetArrays, partition, docids.size());
@@ -241,39 +230,59 @@ public class StandardFacetsAccumulator e
       facetArrays.free(); // to get a cleared array for this partition
     }
 
-    HashMap<CategoryListIterator, Aggregator> categoryLists = getCategoryListMap(
-        facetArrays, partition);
+    HashMap<CategoryListIterator, Aggregator> categoryLists = getCategoryListMap(facetArrays, partition);
 
+    IntsRef ordinals = new IntsRef(32); // a reasonable start capacity for most common apps
     for (Entry<CategoryListIterator, Aggregator> entry : categoryLists.entrySet()) {
-      CategoryListIterator categoryList = entry.getKey();
-      if (!categoryList.init()) {
-        continue;
-      }
-
-      Aggregator categorator = entry.getValue();
-      ScoredDocIDsIterator iterator = docids.iterator();
+      final ScoredDocIDsIterator iterator = docids.iterator();
+      final CategoryListIterator categoryListIter = entry.getKey();
+      final Aggregator aggregator = entry.getValue();
+      Iterator<AtomicReaderContext> contexts = indexReader.leaves().iterator();
+      AtomicReaderContext current = null;
+      int maxDoc = -1;
       while (iterator.next()) {
         int docID = iterator.getDocID();
-        if (!categoryList.skipTo(docID)) {
-          continue;
+        if (docID >= maxDoc) {
+          boolean iteratorDone = false;
+          do { // find the segment which contains this document
+            if (!contexts.hasNext()) {
+              throw new RuntimeException("ScoredDocIDs contains documents outside this reader's segments !?");
+            }
+            current = contexts.next();
+            maxDoc = current.docBase + current.reader().maxDoc();
+            if (docID < maxDoc) { // segment has docs, check if it has categories
+              boolean validSegment = categoryListIter.setNextReader(current);
+              validSegment &= aggregator.setNextReader(current);
+              if (!validSegment) { // if categoryList or aggregtor say it's an invalid segment, skip all docs
+                while (docID < maxDoc && iterator.next()) {
+                  docID = iterator.getDocID();
+                }
+                if (docID < maxDoc) {
+                  iteratorDone = true;
+                }
+              }
+            }
+          } while (docID >= maxDoc);
+          if (iteratorDone) { // iterator finished, terminate the loop
+            break;
+          }
         }
-        categorator.setNextDoc(docID, iterator.getScore());
-        long ordinal;
-        while ((ordinal = categoryList.nextCategory()) <= Integer.MAX_VALUE) {
-          categorator.aggregate((int) ordinal);
+        docID -= current.docBase;
+        categoryListIter.getOrdinals(docID, ordinals);
+        if (ordinals.length == 0) {
+          continue; // document does not have category ordinals
         }
+        aggregator.aggregate(docID, iterator.getScore(), ordinals);
       }
     }
   }
 
-  /**
-   * Init arrays for partition by total counts, optionally applying a factor
-   */
+  /** Init arrays for partition by total counts, optionally applying a factor */
   private final void initArraysByTotalCounts(FacetArrays facetArrays, int partition, int nAccumulatedDocs) {
     int[] intArray = facetArrays.getIntArray();
     totalFacetCounts.fillTotalCountsForPartition(intArray, partition);
     double totalCountsFactor = getTotalCountsFactor();
-    // fix total counts, but only if the effect of this would be meaningfull. 
+    // fix total counts, but only if the effect of this would be meaningful. 
     if (totalCountsFactor < 0.99999) {
       int delta = nAccumulatedDocs + 1;
       for (int i = 0; i < intArray.length; i++) {
@@ -312,20 +321,17 @@ public class StandardFacetsAccumulator e
     
     HashMap<CategoryListIterator, Aggregator> categoryLists = new HashMap<CategoryListIterator, Aggregator>();
 
+    FacetIndexingParams indexingParams = searchParams.getFacetIndexingParams();
     for (FacetRequest facetRequest : searchParams.getFacetRequests()) {
-      Aggregator categoryAggregator = facetRequest.createAggregator(
-          isUsingComplements, facetArrays, indexReader,  taxonomyReader);
+      Aggregator categoryAggregator = facetRequest.createAggregator(isUsingComplements, facetArrays, taxonomyReader);
 
-      CategoryListIterator cli = 
-        facetRequest.createCategoryListIterator(indexReader, taxonomyReader, searchParams, partition);
+      CategoryListIterator cli = indexingParams.getCategoryListParams(facetRequest.categoryPath).createCategoryListIterator(partition);
       
       // get the aggregator
       Aggregator old = categoryLists.put(cli, categoryAggregator);
 
       if (old != null && !old.equals(categoryAggregator)) {
-        // TODO (Facet): create a more meaningful RE class, and throw it.
-        throw new RuntimeException(
-        "Overriding existing category list with different aggregator. THAT'S A NO NO!");
+        throw new RuntimeException("Overriding existing category list with different aggregator");
       }
       // if the aggregator is the same we're covered
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/search/TopKFacetResultsHandler.java Fri Jan 18 18:30:54 2013
@@ -6,10 +6,10 @@ import java.util.ArrayList;
 import org.apache.lucene.facet.search.params.FacetRequest;
 import org.apache.lucene.facet.search.results.FacetResult;
 import org.apache.lucene.facet.search.results.FacetResultNode;
-import org.apache.lucene.facet.search.results.MutableFacetResultNode;
 import org.apache.lucene.facet.search.results.IntermediateFacetResult;
-import org.apache.lucene.facet.taxonomy.ChildrenArrays;
+import org.apache.lucene.facet.search.results.MutableFacetResultNode;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+import org.apache.lucene.facet.taxonomy.directory.ParallelTaxonomyArrays;
 import org.apache.lucene.facet.util.ResultSortUtils;
 
 /*
@@ -55,11 +55,11 @@ public class TopKFacetResultsHandler ext
   public IntermediateFacetResult fetchPartitionResult(FacetArrays facetArrays, int offset)
   throws IOException {
     TopKFacetResult res = null;
-    int ordinal = taxonomyReader.getOrdinal(facetRequest.getCategoryPath());
+    int ordinal = taxonomyReader.getOrdinal(facetRequest.categoryPath);
     if (ordinal != TaxonomyReader.INVALID_ORDINAL) {
       double value = 0;  
       if (isSelfPartition(ordinal, facetArrays, offset)) {
-        int partitionSize = facetArrays.getArraysLength();
+        int partitionSize = facetArrays.arrayLength;
         value = facetRequest.getValueOf(facetArrays, ordinal % partitionSize);
       }
       
@@ -79,7 +79,7 @@ public class TopKFacetResultsHandler ext
   @Override
   public IntermediateFacetResult mergeResults(IntermediateFacetResult... tmpResults) throws IOException {
     
-    int ordinal = taxonomyReader.getOrdinal(facetRequest.getCategoryPath());
+    int ordinal = taxonomyReader.getOrdinal(facetRequest.categoryPath);
     MutableFacetResultNode resNode = new MutableFacetResultNode(ordinal, 0);
     
     int totalFacets = 0;
@@ -121,11 +121,11 @@ public class TopKFacetResultsHandler ext
    */
   private int heapDescendants(int ordinal, Heap<FacetResultNode> pq,
       MutableFacetResultNode parentResultNode, FacetArrays facetArrays, int offset) throws IOException {
-    int partitionSize = facetArrays.getArraysLength();
+    int partitionSize = facetArrays.arrayLength;
     int endOffset = offset + partitionSize;
-    ChildrenArrays childrenArray = taxonomyReader.getChildrenArrays();
-    int[] youngestChild = childrenArray.getYoungestChildArray();
-    int[] olderSibling = childrenArray.getOlderSiblingArray();
+    ParallelTaxonomyArrays childrenArray = taxonomyReader.getParallelTaxonomyArrays();
+    int[] children = childrenArray.children();
+    int[] siblings = childrenArray.siblings();
     FacetResultNode reusable = null;
     int localDepth = 0;
     int depth = facetRequest.getDepth();
@@ -134,9 +134,9 @@ public class TopKFacetResultsHandler ext
     
     int tosOrdinal; // top of stack element
     
-    int yc = youngestChild[ordinal];
+    int yc = children[ordinal];
     while (yc >= endOffset) {
-      yc = olderSibling[yc];
+      yc = siblings[yc];
     }
     // make use of the fact that TaxonomyReader.INVALID_ORDINAL == -1, < endOffset
     // and it, too, can stop the loop.
@@ -161,7 +161,7 @@ public class TopKFacetResultsHandler ext
         // need to proceed to its sibling
         localDepth--;
         // change element now on top of stack to its sibling.
-        ordinalStack[localDepth] = olderSibling[ordinalStack[localDepth]];
+        ordinalStack[localDepth] = siblings[ordinalStack[localDepth]];
         continue;
       }
       // top of stack is not invalid, this is the first time we see it on top of stack.
@@ -187,9 +187,9 @@ public class TopKFacetResultsHandler ext
       }
       if (localDepth < depth) {
         // push kid of current tos
-        yc = youngestChild[tosOrdinal];
+        yc = children[tosOrdinal];
         while (yc >= endOffset) {
-          yc = olderSibling[yc];
+          yc = siblings[yc];
         }
         ordinalStack[++localDepth] = yc;
       } else { // localDepth == depth; current tos exhausted its possible children, mark this by pushing INVALID_ORDINAL