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

svn commit: r1432065 [2/5] - in /lucene/dev/branches/lucene4547: ./ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ lucene/analysis/kuromoji/src/tools/java/org/apache/lucene/analysis/ja/...

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsFacetFields.java Fri Jan 11 14:39:45 2013
@@ -7,7 +7,7 @@ import java.util.Map;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.facet.index.CategoryListBuilder;
+import org.apache.lucene.facet.index.CountingListBuilder;
 import org.apache.lucene.facet.index.DrillDownStream;
 import org.apache.lucene.facet.index.FacetFields;
 import org.apache.lucene.facet.index.params.CategoryListParams;
@@ -15,6 +15,8 @@ import org.apache.lucene.facet.index.par
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -94,15 +96,16 @@ public class AssociationsFacetFields ext
     return categoryLists;
   }
   
-  /**
-   * Returns a {@link CategoryListBuilder} for encoding the given categories and
-   * associations.
-   */
   @Override
-  protected CategoryListBuilder getCategoryListBuilder(CategoryListParams categoryListParams, 
-      Iterable<CategoryPath> categories) {
-    return new AssociationsCategoryListBuilder((CategoryAssociationsContainer) categories, categoryListParams, 
-        indexingParams, taxonomyWriter);
+  protected Map<String,BytesRef> getCategoryListData(CategoryListParams categoryListParams, IntsRef ordinals,
+      Iterable<CategoryPath> categories) throws IOException {
+    AssociationsListBuilder associations = new AssociationsListBuilder((CategoryAssociationsContainer) categories);
+    CountingListBuilder counting = new CountingListBuilder(categoryListParams, indexingParams, taxonomyWriter);
+    // CountingListBuilder modifies the ordinals array, by e.g. adding parent ordinals, sorting etc.
+    // Therefore first build the associations list and only afterwards the counting list.
+    final Map<String,BytesRef> res = associations.build(ordinals, categories);
+    res.putAll(counting.build(ordinals, categories));
+    return res;
   }
   
   @Override

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/AssociationsPayloadIterator.java Fri Jan 11 14:39:45 2013
@@ -53,20 +53,22 @@ public abstract class AssociationsPayloa
   }
 
   /**
-   * Skip to the requested document. Returns true iff the document has categort
-   * association values and they were read successfully.
+   * Skip to the requested document. Returns true iff the document has category
+   * association values and they were read successfully. Associations are
+   * handled through {@link #handleAssociation(int, CategoryAssociation)} by
+   * extending classes.
    */
-  public boolean setNextDoc(int docId) throws IOException {
+  protected final boolean setNextDoc(int docID) throws IOException {
     if (!hasAssociations) { // there are no associations at all
       return false;
     }
 
-    if (!pi.setdoc(docId)) { // no associations for the requested document
+    BytesRef bytes = pi.getPayload(docID);
+    if (bytes == null) { // no associations for the requested document
       return false;
     }
-
-    BytesRef associations = pi.getPayload();
-    ByteArrayDataInput in = new ByteArrayDataInput(associations.bytes, associations.offset, associations.length);
+    
+    ByteArrayDataInput in = new ByteArrayDataInput(bytes.bytes, bytes.offset, bytes.length);
     while (!in.eof()) {
       int ordinal = in.readInt();
       association.deserialize(in);

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryAssociationsContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryAssociationsContainer.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryAssociationsContainer.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryAssociationsContainer.java Fri Jan 11 14:39:45 2013
@@ -55,5 +55,10 @@ public class CategoryAssociationsContain
   public void clear() {
     categoryAssociations.clear();
   }
+
+  @Override
+  public String toString() {
+    return categoryAssociations.toString();
+  }
   
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryFloatAssociation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryFloatAssociation.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryFloatAssociation.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryFloatAssociation.java Fri Jan 11 14:39:45 2013
@@ -71,5 +71,10 @@ public class CategoryFloatAssociation im
   public float getValue() {
     return value;
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + value + ")";
+  }
   
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryIntAssociation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryIntAssociation.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryIntAssociation.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/CategoryIntAssociation.java Fri Jan 11 14:39:45 2013
@@ -72,4 +72,9 @@ public class CategoryIntAssociation impl
     return value;
   }
   
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "(" + value + ")";
+  }
+  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/FloatAssociationsPayloadIterator.java Fri Jan 11 14:39:45 2013
@@ -40,23 +40,17 @@ public class FloatAssociationsPayloadIte
   protected void handleAssociation(int ordinal, CategoryFloatAssociation association) {
     ordinalAssociations.put(ordinal, association.getValue());
   }
-  
-  @Override
-  public boolean setNextDoc(int docId) throws IOException {
-    ordinalAssociations.clear();
-    return super.setNextDoc(docId);
-  }
 
   /**
-   * Get the float association value for the given ordinal, or
-   * {@link Float#NaN} in case the ordinal has no association value.
+   * Returns the float association values of the categories that are associated
+   * with the given document, or {@code null} if the document has no
+   * associations.
+   * <p>
+   * <b>NOTE:</b> you are not expected to modify the returned map.
    */
-  public float getAssociation(int ordinal) {
-    if (ordinalAssociations.containsKey(ordinal)) {
-      return ordinalAssociations.get(ordinal);
-    }
-
-    return Float.NaN;
+  public IntToFloatMap getAssociations(int docID) throws IOException {
+    ordinalAssociations.clear();
+    return setNextDoc(docID) ? ordinalAssociations : null;
   }
-
+  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/associations/IntAssociationsPayloadIterator.java Fri Jan 11 14:39:45 2013
@@ -31,12 +31,6 @@ public class IntAssociationsPayloadItera
 
   private final IntToIntMap ordinalAssociations = new IntToIntMap();
 
-  /**
-   * The long-special-value returned for ordinals which have no associated int
-   * value. It is not in the int range of values making it a valid mark.
-   */
-  public final static long NO_ASSOCIATION = Integer.MAX_VALUE + 1;
-
   public IntAssociationsPayloadIterator(IndexReader reader, String field, CategoryIntAssociation association) 
       throws IOException {
     super(reader, field, association);
@@ -47,22 +41,16 @@ public class IntAssociationsPayloadItera
     ordinalAssociations.put(ordinal, association.getValue());
   }
   
-  @Override
-  public boolean setNextDoc(int docId) throws IOException {
-    ordinalAssociations.clear();
-    return super.setNextDoc(docId);
-  }
-
   /**
-   * Get the integer association value for the given ordinal, or
-   * {@link #NO_ASSOCIATION} in case the ordinal has no association value.
+   * Returns the integer association values of the categories that are
+   * associated with the given document, or {@code null} if the document has no
+   * associations.
+   * <p>
+   * <b>NOTE:</b> you are not expected to modify the returned map.
    */
-  public long getAssociation(int ordinal) {
-    if (ordinalAssociations.containsKey(ordinal)) {
-      return ordinalAssociations.get(ordinal);
-    }
-
-    return NO_ASSOCIATION;
+  public IntToIntMap getAssociations(int docID) throws IOException {
+    ordinalAssociations.clear();
+    return setNextDoc(docID) ? ordinalAssociations : null;
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/CategoryListBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/CategoryListBuilder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/CategoryListBuilder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/CategoryListBuilder.java Fri Jan 11 14:39:45 2013
@@ -1,19 +1,11 @@
 package org.apache.lucene.facet.index;
 
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map.Entry;
+import java.util.Map;
 
-import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
-import org.apache.lucene.facet.index.params.CategoryListParams;
-import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.taxonomy.CategoryPath;
-import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
-import org.apache.lucene.facet.util.PartitionsUtils;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
-import org.apache.lucene.util.UnsafeByteArrayOutputStream;
-import org.apache.lucene.util.encoding.IntEncoder;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -33,149 +25,14 @@ import org.apache.lucene.util.encoding.I
  */
 
 /**
- * Builds a category list by encoding the category ordinals into one or more
- * {@link BytesRef}. Each {@link BytesRef} corresponds to a set of ordinals that
- * belong to the same partition. When partitions are not enabled (i.e.
- * {@link FacetIndexingParams#getPartitionSize()} returns
- * {@link Integer#MAX_VALUE}), only one {@link BytesRef} is returned by this
- * class.
+ * Builds a category list data by encoding the appropriate information for every
+ * category and ordinal given to {@link #build(IntsRef, Iterable)}.
+ * 
+ * @lucene.experimental
  */
-public class CategoryListBuilder {
- 
-  /** Specializes encoding ordinals when partitions are enabled/disabled. */
-  private static abstract class OrdinalsEncoder {
-    OrdinalsEncoder() {}
-    public abstract void encode(int ordinal);
-    public abstract HashMap<String,BytesRef> finish();
-  }
+public interface CategoryListBuilder {
   
-  private static final class NoPartitionsOrdinalsEncoder extends OrdinalsEncoder {
-    
-    private final IntEncoder encoder;
-    private final UnsafeByteArrayOutputStream ubaos;
-    private final String name;
-    
-    NoPartitionsOrdinalsEncoder(CategoryListParams categoryListParams) {
-      name = categoryListParams.getTerm().text();
-      encoder = categoryListParams.createEncoder();
-      ubaos = new UnsafeByteArrayOutputStream();
-      encoder.reInit(ubaos);
-    }
-    
-    @Override
-    public void encode(int ordinal) {
-      try {
-        encoder.encode(ordinal);
-      } catch (IOException e) {
-        // shouldn't happen as we're writing to byte[]
-        throw new RuntimeException("unexpected exception", e);
-      }
-    }
-    
-    @Override
-    public HashMap<String,BytesRef> finish() {
-      try {
-        encoder.close();
-      } catch (IOException e) {
-        // shouldn't happen as we're writing to byte[]
-        throw new RuntimeException("unexpected exception", e);
-      }
-      HashMap<String,BytesRef> result = new HashMap<String,BytesRef>();
-      result.put(name, new BytesRef(ubaos.toByteArray(), ubaos.getStartPos(), ubaos.length()));
-      return result;
-    }
-    
-  }
-  
-  private static final class PerPartitionOrdinalsEncoder extends OrdinalsEncoder {
-
-    private final FacetIndexingParams indexingParams;
-    private final CategoryListParams categoryListParams;
-    private final int partitionSize;
-    private final HashMap<String,IntEncoder> partitionEncoder = new HashMap<String,IntEncoder>();
-    private final HashMap<String,UnsafeByteArrayOutputStream> partitionBytes = new HashMap<String,UnsafeByteArrayOutputStream>();
-
-    PerPartitionOrdinalsEncoder(FacetIndexingParams indexingParams, CategoryListParams categoryListParams) {
-      this.indexingParams = indexingParams;
-      this.categoryListParams = categoryListParams;
-      this.partitionSize = indexingParams.getPartitionSize();
-    }
-
-    @Override
-    public void encode(int ordinal) {
-      final String name = PartitionsUtils.partitionNameByOrdinal(indexingParams, categoryListParams, ordinal);
-      IntEncoder encoder = partitionEncoder.get(name);
-      if (encoder == null) {
-        encoder = categoryListParams.createEncoder();
-        final UnsafeByteArrayOutputStream ubaos = new UnsafeByteArrayOutputStream();        
-        encoder.reInit(ubaos);
-        partitionEncoder.put(name, encoder);
-        partitionBytes.put(name, ubaos);
-      }
-      try {
-        encoder.encode(ordinal % partitionSize);
-      } catch (IOException e) {
-        // shouldn't happen as we're writing to byte[]
-        throw new RuntimeException("unexpected exception", e);
-      }
-    }
-    
-    @Override
-    public HashMap<String,BytesRef> finish() {
-      // finish encoding
-      IOUtils.closeWhileHandlingException(partitionEncoder.values());
-      
-      HashMap<String,BytesRef> bytes = new HashMap<String,BytesRef>();
-      for (Entry<String,UnsafeByteArrayOutputStream> e : partitionBytes.entrySet()) {
-        UnsafeByteArrayOutputStream ubaos = e.getValue();
-        bytes.put(e.getKey(), new BytesRef(ubaos.toByteArray(), ubaos.getStartPos(), ubaos.length()));
-      }
-      return bytes;
-    }
-    
-  }
-  
-  private final TaxonomyWriter taxoWriter;
-  private final OrdinalsEncoder ordinalsEncoder;
-  private final OrdinalPolicy ordinalPolicy;
-  
-  public CategoryListBuilder(CategoryListParams categoryListParams, FacetIndexingParams indexingParams, 
-      TaxonomyWriter taxoWriter) {
-    this.taxoWriter = taxoWriter;
-    this.ordinalPolicy = indexingParams.getOrdinalPolicy();
-    if (indexingParams.getPartitionSize() == Integer.MAX_VALUE) {
-      ordinalsEncoder = new NoPartitionsOrdinalsEncoder(categoryListParams);
-    } else {
-      ordinalsEncoder = new PerPartitionOrdinalsEncoder(indexingParams, categoryListParams);
-    }
-  }
-
-  /**
-   * Encodes the given ordinal as well as any of its parent ordinals (per
-   * {@link OrdinalPolicy}).
-   */
-  public void handle(int ordinal, CategoryPath cp) throws IOException {
-    ordinalsEncoder.encode(ordinal);
-    
-    // add all parent ordinals, per OrdinalPolicy
-    int parent = taxoWriter.getParent(ordinal);
-    while (parent > 0) {
-      if (ordinalPolicy.shouldAdd(parent)) {
-        ordinalsEncoder.encode(parent);
-      }
-      parent = taxoWriter.getParent(parent);
-    }
-  }
-  
-  /**
-   * Returns the encoded ordinals data. Every returned {@link BytesRef}
-   * corresponds to a single partition (as defined by
-   * {@link FacetIndexingParams#getPartitionSize()}) and the key denotes the
-   * partition ID. When no partitions are defined, the returned map includes
-   * only one value.
-   */
-  public HashMap<String,BytesRef> finish() {
-    return ordinalsEncoder.finish();
-  }
+  /** Returns the encoded ordinals data. */
+  public Map<String,BytesRef> build(IntsRef ordinals, Iterable<CategoryPath> categories) throws IOException;
   
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/FacetFields.java Fri Jan 11 14:39:45 2013
@@ -2,6 +2,7 @@ package org.apache.lucene.facet.index;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -21,6 +22,7 @@ import org.apache.lucene.facet.taxonomy.
 import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -69,7 +71,7 @@ public class FacetFields {
       return true;
     }
     
-    void setCategoriesData(HashMap<String,BytesRef> categoriesData) {
+    void setCategoriesData(Map<String,BytesRef> categoriesData) {
       this.categoriesData = categoriesData.entrySet().iterator();
     }
     
@@ -132,6 +134,9 @@ public class FacetFields {
    */
   protected Map<CategoryListParams,Iterable<CategoryPath>> createCategoryListMapping(
       Iterable<CategoryPath> categories) {
+    if (indexingParams.getAllCategoryListParams().size() == 1) {
+      return Collections.singletonMap(indexingParams.getCategoryListParams(null), categories);
+    }
     HashMap<CategoryListParams,Iterable<CategoryPath>> categoryLists = 
         new HashMap<CategoryListParams,Iterable<CategoryPath>>();
     for (CategoryPath cp : categories) {
@@ -147,10 +152,15 @@ public class FacetFields {
     return categoryLists;
   }
   
-  /** Returns a {@link CategoryListBuilder} for encoding the given categories. */
-  protected CategoryListBuilder getCategoryListBuilder(CategoryListParams categoryListParams, 
-      Iterable<CategoryPath> categories /* needed for AssociationsFacetFields */) {
-    return new CategoryListBuilder(categoryListParams, indexingParams, taxonomyWriter);
+  /**
+   * Returns the category list data, as a mapping from key to {@link BytesRef}
+   * which includes the encoded data. Every ordinal in {@code ordinals}
+   * corrspond to a {@link CategoryPath} returned from {@code categories}.
+   */
+  protected Map<String,BytesRef> getCategoryListData(CategoryListParams categoryListParams, 
+      IntsRef ordinals, Iterable<CategoryPath> categories /* needed for AssociationsFacetFields */) 
+      throws IOException {
+    return new CountingListBuilder(categoryListParams, indexingParams, taxonomyWriter).build(ordinals, categories);
   }
   
   /**
@@ -185,17 +195,25 @@ public class FacetFields {
 
     // for each CLP we add a different field for drill-down terms as well as for
     // counting list data.
+    IntsRef ordinals = new IntsRef(32); // should be enough for most common applications
     for (Entry<CategoryListParams, Iterable<CategoryPath>> e : categoryLists.entrySet()) {
       final CategoryListParams clp = e.getKey();
       final String field = clp.getTerm().field();
 
-      // add the counting list data
-      CategoryListBuilder categoriesPayloadBuilder = getCategoryListBuilder(clp, e.getValue());
+      // build category list data
+      ordinals.length = 0; // reset
+      int maxNumOrds = 0;
       for (CategoryPath cp : e.getValue()) {
         int ordinal = taxonomyWriter.addCategory(cp);
-        categoriesPayloadBuilder.handle(ordinal , cp);
+        maxNumOrds += cp.length; // ordinal and potentially all parents
+        if (ordinals.ints.length < maxNumOrds) {
+          ordinals.grow(maxNumOrds);
+        }
+        ordinals.ints[ordinals.length++] = ordinal;
       }
-      HashMap<String,BytesRef> categoriesData = categoriesPayloadBuilder.finish();
+      Map<String,BytesRef> categoriesData = getCategoryListData(clp, ordinals, e.getValue());
+      
+      // add the counting list data
       CountingListStream ts = new CountingListStream();
       ts.setCategoriesData(categoriesData);
       doc.add(new Field(field, ts, COUNTING_LIST_PAYLOAD_TYPE));

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/OrdinalMappingAtomicReader.java Fri Jan 11 14:39:45 2013
@@ -17,10 +17,7 @@ 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;
 
@@ -36,6 +33,7 @@ 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;
 
@@ -187,7 +185,7 @@ public class OrdinalMappingAtomicReader 
   private class OrdinalMappingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
     private final IntEncoder encoder;
     private final IntDecoder decoder;
-    private final ByteArrayOutputStream os = new ByteArrayOutputStream();
+    private final IntsRef ordinals = new IntsRef(32);
     private final BytesRef payloadOut = new BytesRef();
 
     public OrdinalMappingDocsAndPositionsEnum(DocsAndPositionsEnum in, CategoryListParams params) {
@@ -202,21 +200,14 @@ public class OrdinalMappingAtomicReader 
       if (payload == null) {
         return payload;
       } 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(payload, 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;
+        
+        encoder.encode(ordinals, payloadOut);
         return payloadOut;
       }
     }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/CategoryListParams.java Fri Jan 11 14:39:45 2013
@@ -7,7 +7,7 @@ import org.apache.lucene.index.IndexRead
 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.PayloadCategoryListIteraor;
 import org.apache.lucene.facet.search.TotalFacetCounts;
 import org.apache.lucene.facet.util.PartitionsUtils;
 import org.apache.lucene.util.encoding.DGapIntEncoder;
@@ -142,7 +142,7 @@ public class CategoryListParams implemen
       int partition) throws IOException {
     String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
     Term payloadTerm = new Term(term.field(), categoryListTermStr);
-    return new PayloadIntDecodingIterator(reader, payloadTerm,
+    return new PayloadCategoryListIteraor(reader, payloadTerm,
         createEncoder().createMatchingDecoder());
   }
   

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/FacetIndexingParams.java Fri Jan 11 14:39:45 2013
@@ -83,18 +83,9 @@ public class FacetIndexingParams {
   }
 
   /**
-   * The name of the category list to put this category in, or {@code 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 {@code null} is returned for a category, it means that this category
-   * should not appear in any category list, and thus weights for it cannot be
-   * aggregated. This category can still be used for drill-down, even though the
-   * its weight is unknown.
+   * 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 PerDimensionIndexingParams
    */

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/index/params/PerDimensionIndexingParams.java Fri Jan 11 14:39:45 2013
@@ -78,7 +78,9 @@ public class PerDimensionIndexingParams 
 
   /**
    * Returns the {@link CategoryListParams} for the corresponding dimension
-   * which is returned by {@code category.getComponent(0)}.
+   * 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) {

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/CategoryListIterator.java Fri Jan 11 14:39:45 2013
@@ -2,6 +2,8 @@ package org.apache.lucene.facet.search;
 
 import java.io.IOException;
 
+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,20 +22,10 @@ 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.
  * <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
  */
@@ -41,29 +33,20 @@ 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.
+   * {@link #getOrdinals(int, IntsRef)}, and its return value indicates whether there are
+   * any relevant documents for this iterator.
    */
   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.
+   * 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 category ordinals
+   * associated with it, {@link IntsRef#length} is set to zero.
    */
-  public boolean skipTo(int docId) 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.
-   */
-  public long nextCategory() throws IOException;
-
+  public void getOrdinals(int docID, IntsRef ints) throws IOException;
+  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/PayloadIterator.java Fri Jan 11 14:39:45 2013
@@ -34,9 +34,9 @@ import org.apache.lucene.util.BytesRef;
  * A utility class for iterating through a posting list of a given term and
  * retrieving the payload of the first position in every document. For
  * efficiency, this class does not check if documents passed to
- * {@link #setdoc(int)} are deleted, since it is usually used to iterate on
+ * {@link #getPayload(int)} are deleted, since it is usually used to iterate on
  * payloads of documents that matched a query. If you need to skip over deleted
- * documents, you should do so before calling {@link #setdoc(int)}.
+ * documents, you should do so before calling {@link #getPayload(int)}.
  * 
  * @lucene.experimental
  */
@@ -84,8 +84,8 @@ public class PayloadIterator {
   
   /**
    * Initialize the iterator. Should be done before the first call to
-   * {@link #setdoc(int)}. Returns {@code false} if no category list is found,
-   * or the category list has no documents.
+   * {@link #getPayload(int)}. Returns {@code false} if no category list is
+   * found, or the category list has no documents.
    */
   public boolean init() throws IOException {
     nextSegment();
@@ -93,30 +93,29 @@ public class PayloadIterator {
   }
 
   /**
-   * Skip forward to document docId. Return true if this document exists and
-   * has any payload.
-   * <P>
-   * Users should call this method with increasing docIds, and implementations
-   * can assume that this is the case.
+   * Returns the {@link BytesRef payload} of the given document, or {@code null}
+   * if the document does not exist, there are no more documents in the posting
+   * list, or the document exists but has not payload. You should call
+   * {@link #init()} before the first call to this method.
    */
-  public boolean setdoc(int docId) throws IOException {
+  public BytesRef getPayload(int docID) throws IOException {
     if (!hasMore) {
-      return false;
+      return null;
     }
 
     // re-basing docId->localDocID is done fewer times than currentDoc->globalDoc
-    int localDocID = docId - curDocBase;
+    int localDocID = docID - curDocBase;
     
     if (curDocID > localDocID) {
       // document does not exist
-      return false;
+      return null;
     }
     
     if (curDocID < localDocID) {
       // look for the document either in that segment, or others
       while (hasMore && (curDocID = currentDPE.advance(localDocID)) == DocIdSetIterator.NO_MORE_DOCS) {
         nextSegment(); // also updates curDocID
-        localDocID = docId - curDocBase;
+        localDocID = docID - curDocBase;
         // nextSegment advances to nextDoc, so check if we still need to advance
         if (curDocID >= localDocID) {
           break;
@@ -127,7 +126,7 @@ public class PayloadIterator {
       // 1. we iterated over all segments (hasMore=false)
       // 2. current segment advanced to a doc, either requested or higher
       if (!hasMore || curDocID != localDocID) {
-        return false;
+        return null;
       }
     }
 
@@ -135,12 +134,7 @@ public class PayloadIterator {
     assert currentDPE.freq() == 1 : "expecting freq=1 (got " + currentDPE.freq() + ") term=" + term + " doc=" + (curDocID + curDocBase);
     int pos = currentDPE.nextPosition();
     assert pos != -1 : "no positions for term=" + term + " doc=" + (curDocID + curDocBase);
-    data = currentDPE.getPayload();
-    return data != null;
+    return currentDPE.getPayload();
   }
   
-  public BytesRef getPayload() {
-    return data;
-  }
-
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/StandardFacetsAccumulator.java Fri Jan 11 14:39:45 2013
@@ -10,6 +10,7 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.IntsRef;
 
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.facet.search.params.FacetSearchParams;
@@ -231,9 +232,9 @@ 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()) {
@@ -244,14 +245,11 @@ public class StandardFacetsAccumulator e
       ScoredDocIDsIterator iterator = docids.iterator();
       while (iterator.next()) {
         int docID = iterator.getDocID();
-        if (!categoryList.skipTo(docID)) {
+        categoryList.getOrdinals(docID, ordinals);
+        if (ordinals.length == 0) {
           continue;
         }
-        categorator.setNextDoc(docID, iterator.getScore());
-        long ordinal;
-        while ((ordinal = categoryList.nextCategory()) <= Integer.MAX_VALUE) {
-          categorator.aggregate((int) ordinal);
-        }
+        categorator.aggregate(docID, iterator.getScore(), ordinals);
       }
     }
   }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/Aggregator.java Fri Jan 11 14:39:45 2013
@@ -2,6 +2,8 @@ package org.apache.lucene.facet.search.a
 
 import java.io.IOException;
 
+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
@@ -36,16 +38,9 @@ import java.io.IOException;
 public interface Aggregator {
 
   /**
-   * Specify the document (and its score in the search) that the following
-   * {@link #aggregate(int)} calls will pertain to.
+   * Aggregate the ordinals of the given document ID (and its score). The given
+   * ordinals offset is always zero.
    */
-  void setNextDoc(int docid, float score) throws IOException;
-
-  /**
-   * Collect (and do whatever an implementation deems appropriate) the
-   * category given by its ordinal. This category belongs to a document
-   * given earlier by {@link #setNextDoc(int, float)}.
-   */
-  void aggregate(int ordinal);
-
+  public void aggregate(int docID, float score, IntsRef ordinals) throws IOException;
+  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ComplementCountingAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ComplementCountingAggregator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ComplementCountingAggregator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ComplementCountingAggregator.java Fri Jan 11 14:39:45 2013
@@ -1,5 +1,9 @@
 package org.apache.lucene.facet.search.aggregator;
 
+import java.io.IOException;
+
+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
@@ -29,9 +33,12 @@ public class ComplementCountingAggregato
   }
 
   @Override
-  public void aggregate(int ordinal) {
-    assert counterArray[ordinal]!=0:"complement aggregation: count is about to become negative for ordinal "+ordinal;
-    --counterArray[ordinal];
+  public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
+    for (int i = 0; i < ordinals.length; i++) {
+      int ord = ordinals.ints[i];
+      assert counterArray[ord] != 0 : "complement aggregation: count is about to become negative for ordinal " + ord;
+      --counterArray[ord];
+    }
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/CountingAggregator.java Fri Jan 11 14:39:45 2013
@@ -1,5 +1,9 @@
 package org.apache.lucene.facet.search.aggregator;
 
+import java.io.IOException;
+
+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
@@ -27,21 +31,17 @@ package org.apache.lucene.facet.search.a
 public class CountingAggregator implements Aggregator {
 
   protected int[] counterArray;
-
-  @Override
-  public void aggregate(int ordinal) {
-    ++counterArray[ordinal];
-  }
-
-  @Override
-  public void setNextDoc(int docid, float score) {
-    // There's nothing for us to do here since we only increment the count by 1
-    // in this aggregator.
-  }
-
+  
   public CountingAggregator(int[] counterArray) {
     this.counterArray = counterArray;
   }
+  
+  @Override
+  public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
+    for (int i = 0; i < ordinals.length; i++) {
+      counterArray[ordinals.ints[i]]++;
+    }
+  }
 
   @Override
   public boolean equals(Object obj) {
@@ -54,8 +54,7 @@ public class CountingAggregator implemen
 
   @Override
   public int hashCode() {
-    int hashCode = counterArray == null ? 0 : counterArray.hashCode();
-
-    return hashCode;
+    return counterArray == null ? 0 : counterArray.hashCode();
   }
+  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/ScoringAggregator.java Fri Jan 11 14:39:45 2013
@@ -1,5 +1,9 @@
 package org.apache.lucene.facet.search.aggregator;
 
+import java.io.IOException;
+
+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
@@ -26,7 +30,6 @@ package org.apache.lucene.facet.search.a
 public class ScoringAggregator implements Aggregator {
 
   private final float[] scoreArray;
-  private float score;
   private final int hashCode;
   
   public ScoringAggregator(float[] counterArray) {
@@ -35,10 +38,12 @@ public class ScoringAggregator implement
   }
 
   @Override
-  public void aggregate(int ordinal) {
-    scoreArray[ordinal] += score;
+  public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
+    for (int i = 0; i < ordinals.length; i++) {
+      scoreArray[ordinals.ints[i]] += score;
+    }
   }
-
+  
   @Override
   public boolean equals(Object obj) {
     if (obj == null || obj.getClass() != this.getClass()) {
@@ -53,8 +58,4 @@ public class ScoringAggregator implement
     return hashCode;
   }
 
-  @Override
-  public void setNextDoc(int docid, float score) {
-    this.score = score;
-  }
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java Fri Jan 11 14:39:45 2013
@@ -7,6 +7,8 @@ import org.apache.lucene.facet.associati
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.collections.IntToFloatMap;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -48,13 +50,18 @@ public class AssociationFloatSumAggregat
   }
 
   @Override
-  public void aggregate(int ordinal) {
-    float association = associations.getAssociation(ordinal);
-    if (!Float.isNaN(association)) {
-      sumArray[ordinal] += association;
+  public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
+    IntToFloatMap values = associations.getAssociations(docID);
+    if (values != null) {
+      for (int i = 0; i < ordinals.length; i++) {
+        int ord = ordinals.ints[i];
+        if (values.containsKey(ord)) {
+          sumArray[ord] += values.get(ord);
+        }
+      }
     }
   }
-
+  
   @Override
   public boolean equals(Object obj) {
     if (obj == null || obj.getClass() != this.getClass()) {
@@ -69,9 +76,4 @@ public class AssociationFloatSumAggregat
     return field.hashCode();
   }
 
-  @Override
-  public void setNextDoc(int docid, float score) throws IOException {
-    associations.setNextDoc(docid);
-  }
-  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java Fri Jan 11 14:39:45 2013
@@ -7,6 +7,8 @@ import org.apache.lucene.facet.associati
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.search.aggregator.Aggregator;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.collections.IntToIntMap;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -48,13 +50,18 @@ public class AssociationIntSumAggregator
   }
 
   @Override
-  public void aggregate(int ordinal) {
-    long association = associations.getAssociation(ordinal);
-    if (association != IntAssociationsPayloadIterator.NO_ASSOCIATION) {
-      sumArray[ordinal] += association;
+  public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
+    IntToIntMap values = associations.getAssociations(docID);
+    if (values != null) {
+      for (int i = 0; i < ordinals.length; i++) {
+        int ord = ordinals.ints[i];
+        if (values.containsKey(ord)) {
+          sumArray[ord] += values.get(ord);
+        }
+      }
     }
   }
-
+  
   @Override
   public boolean equals(Object obj) {
     if (obj == null || obj.getClass() != this.getClass()) {
@@ -69,9 +76,4 @@ public class AssociationIntSumAggregator
     return field.hashCode();
   }
 
-  @Override
-  public void setNextDoc(int docid, float score) throws IOException {
-    associations.setNextDoc(docid);
-  }
-  
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java Fri Jan 11 14:39:45 2013
@@ -2,13 +2,12 @@ package org.apache.lucene.facet.search.c
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
-
 import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
 import org.apache.lucene.facet.search.CategoryListIterator;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
-import org.apache.lucene.util.collections.IntArray;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -56,33 +55,26 @@ public class CategoryListData {
   protected CategoryListData() {
   }
   
-  /**
-   * Compute category list data for caching for faster iteration.
-   */
+  /** Compute category list data for caching for faster iteration. */
   CategoryListData(IndexReader reader, TaxonomyReader taxo, 
       FacetIndexingParams iparams, CategoryListParams clp) throws IOException {
   
     final int maxDoc = reader.maxDoc();
     int[][][]dpf  = new int[maxDoc][][];
     int numPartitions = (int)Math.ceil(taxo.getSize()/(double)iparams.getPartitionSize());
-    IntArray docCategories = new IntArray(); 
-    for (int part=0; part<numPartitions; part++) {
+    IntsRef ordinals = new IntsRef(32);
+    for (int part = 0; part < numPartitions; part++) {
       CategoryListIterator cli = clp.createCategoryListIterator(reader, part);
       if (cli.init()) {
-        for (int doc=0; doc<maxDoc; doc++) {
-          if (cli.skipTo(doc)) {
-            docCategories.clear(false);
-            if (dpf[doc]==null) {
+        for (int doc = 0; doc < maxDoc; doc++) {
+          cli.getOrdinals(doc, ordinals);
+          if (ordinals.length > 0) {
+            if (dpf[doc] == null) {
               dpf[doc] = new int[numPartitions][];
             }
-            long category;
-            while ((category = cli.nextCategory()) <= Integer.MAX_VALUE) {
-              docCategories.addToArray((int)category);
-            }
-            final int size = docCategories.size();
-            dpf[doc][part] = new int[size];
-            for (int i=0; i<size; i++) {
-              dpf[doc][part][i] = docCategories.get(i);
+            dpf[doc][part] = new int[ordinals.length];
+            for (int i = 0; i < ordinals.length; i++) {
+              dpf[doc][part][i] = ordinals.ints[i];
             }
           }
         }
@@ -98,14 +90,11 @@ public class CategoryListData {
     return new RAMCategoryListIterator(partition, docPartitionCategories);
   }
 
-  /**
-   * Internal: category list iterator over uncompressed category info in RAM
-   */
+  /** Internal: category list iterator over uncompressed category info in RAM */
   private static class RAMCategoryListIterator implements CategoryListIterator {
+    
     private final int part;
     private final int[][][] dpc;
-    private int currDoc = -1;
-    private int nextCategoryIndex = -1;  
     
     RAMCategoryListIterator(int part, int[][][] docPartitionCategories) {
       this.part = part;
@@ -114,25 +103,22 @@ public class CategoryListData {
 
     @Override
     public boolean init() throws IOException {
-      return dpc!=null && dpc.length>part;
-    }
-
-    @Override
-    public long nextCategory() throws IOException {
-      if (nextCategoryIndex >= dpc[currDoc][part].length) {
-        return 1L+Integer.MAX_VALUE;
-      }
-      return dpc[currDoc][part][nextCategoryIndex++]; 
+      return dpc != null && dpc.length > part;
     }
 
     @Override
-    public boolean skipTo(int docId) throws IOException {
-      final boolean res = dpc.length>docId && dpc[docId]!=null && dpc[docId][part]!=null;
-      if (res) {
-        currDoc = docId;
-        nextCategoryIndex = 0;
+    public void getOrdinals(int docID, IntsRef ints) throws IOException {
+      ints.length = 0;
+      if (dpc.length > docID && dpc[docID] != null && dpc[docID][part] != null) {
+        if (ints.ints.length < dpc[docID][part].length) {
+          ints.grow(dpc[docID][part].length);
+        }
+        ints.length = 0;
+        for (int i = 0; i < dpc[docID][part].length; i++) {
+          ints.ints[ints.length++] = dpc[docID][part][i];
+        }
       }
-      return res;
     }
   }
+  
 }
\ No newline at end of file

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java Fri Jan 11 14:39:45 2013
@@ -48,8 +48,7 @@ public class CountFacetRequest extends F
 
   @Override
   public Aggregator createAggregator(boolean useComplements,
-                                      FacetArrays arrays, IndexReader reader,
-                                      TaxonomyReader taxonomy) {
+      FacetArrays arrays, IndexReader reader, TaxonomyReader taxonomy) {
     // we rely on that, if needed, result is cleared by arrays!
     int[] a = arrays.getIntArray();
     if (useComplements) {

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java Fri Jan 11 14:39:45 2013
@@ -5,6 +5,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.facet.search.CategoryListIterator;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -33,16 +34,13 @@ public class MultiCategoryListIterator i
 
   private final CategoryListIterator[] iterators;
   private final List<CategoryListIterator> validIterators;
-  private final List<CategoryListIterator> perDocValidIterators;
 
   /** Receives the iterators to iterate on */
   public MultiCategoryListIterator(CategoryListIterator... iterators) {
     this.iterators = iterators;
     this.validIterators = new ArrayList<CategoryListIterator>();
-    this.perDocValidIterators = new ArrayList<CategoryListIterator>();
   }
 
-  /** Fails if all given iterators fail to init */
   @Override
   public boolean init() throws IOException {
     for (CategoryListIterator cli : iterators) {
@@ -52,35 +50,17 @@ public class MultiCategoryListIterator i
     }
     return !validIterators.isEmpty();
   }
-
-  /**
-   * Return a value larger than {@link Integer#MAX_VALUE} only if all
-   * iterators are exhausted
-   */
-  @Override
-  public long nextCategory() throws IOException {
-    while (!perDocValidIterators.isEmpty()) {
-      long value = perDocValidIterators.get(0).nextCategory();
-      if (value <= Integer.MAX_VALUE) {
-        return value;
-      }
-      perDocValidIterators.remove(0);
-    }
-    return 0x100000000L;
-  }
-
-  /**
-   * Fails only if skipTo on all the provided iterators returned {@code false}
-   */
+  
   @Override
-  public boolean skipTo(int docId) throws IOException {
-    perDocValidIterators.clear();
+  public void getOrdinals(int docID, IntsRef ints) throws IOException {
+    IntsRef tmp = new IntsRef(ints.length);
     for (CategoryListIterator cli : validIterators) {
-      if (cli.skipTo(docId)) {
-        perDocValidIterators.add(cli);
+      cli.getOrdinals(docID, tmp);
+      if (ints.ints.length < ints.length + tmp.length) {
+        ints.grow(ints.length + tmp.length);
       }
+      ints.length += tmp.length;
     }
-    return !perDocValidIterators.isEmpty();
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.OutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,38 +27,31 @@ import java.io.OutputStream;
  * read more on the two implementations {@link FourFlagsIntEncoder} and
  * {@link EightFlagsIntEncoder}.
  * <p>
- * Extensions of this class need to implement {@link #encode(int)} in order to
- * build the proper indicator (flags). When enough values were accumulated
- * (typically the batch size), extensions can call {@link #encodeChunk()} to
- * flush the indicator and the rest of the values.
+ * Extensions of this class need to implement {@link #encode(IntsRef, BytesRef)}
+ * in order to build the proper indicator (flags). When enough values were
+ * accumulated (typically the batch size), extensions can call
+ * {@link #encodeChunk(BytesRef)} to flush the indicator and the rest of the
+ * values.
  * <p>
  * <b>NOTE:</b> flags encoders do not accept values &le; 0 (zero) in their
- * {@link #encode(int)}. For performance reasons they do not check that
- * condition, however if such value is passed the result stream may be corrupt
- * or an exception will be thrown. Also, these encoders perform the best when
- * there are many consecutive small values (depends on the encoder
+ * {@link #encode(IntsRef, BytesRef)}. For performance reasons they do not check
+ * that condition, however if such value is passed the result stream may be
+ * corrupt or an exception will be thrown. Also, these encoders perform the best
+ * when there are many consecutive small values (depends on the encoder
  * implementation). If that is not the case, the encoder will occupy 1 more byte
  * for every <i>batch</i> number of integers, over whatever
  * {@link VInt8IntEncoder} would have occupied. Therefore make sure to check
  * whether your data fits into the conditions of the specific encoder.
  * <p>
  * For the reasons mentioned above, these encoders are usually chained with
- * {@link UniqueValuesIntEncoder} and {@link DGapIntEncoder} in the following
- * manner: <code><pre class="prettyprint">
- * IntEncoder fourFlags = 
- *         new SortingEncoderFilter(new UniqueValuesIntEncoder(new DGapIntEncoder(new FlagsIntEncoderImpl())));
- * </pre></code>
+ * {@link UniqueValuesIntEncoder} and {@link DGapIntEncoder}.
  * 
  * @lucene.experimental
  */
 public abstract class ChunksIntEncoder extends IntEncoder {
 
   /** Holds the values which must be encoded, outside the indicator. */
-  protected final int[] encodeQueue;
-  protected int encodeQueueSize = 0;
-
-  /** Encoder used to encode values outside the indicator. */
-  protected final IntEncoder encoder = new VInt8IntEncoder();
+  protected final IntsRef encodeQueue;
 
   /** Represents bits flag byte. */
   protected int indicator = 0;
@@ -67,39 +60,33 @@ public abstract class ChunksIntEncoder e
   protected byte ordinal = 0;
 
   protected ChunksIntEncoder(int chunkSize) {
-    encodeQueue = new int[chunkSize];
+    encodeQueue = new IntsRef(chunkSize);
   }
 
   /**
    * Encodes the values of the current chunk. First it writes the indicator, and
    * then it encodes the values outside the indicator.
    */
-  protected void encodeChunk() throws IOException {
-    out.write(indicator);
-    for (int i = 0; i < encodeQueueSize; ++i) {
-      encoder.encode(encodeQueue[i]);
+  protected void encodeChunk(BytesRef buf) {
+    // ensure there's enough room in the buffer
+    int maxBytesRequired = buf.length + 1 + encodeQueue.length * 4; /* indicator + at most 4 bytes per positive VInt */
+    if (buf.bytes.length < maxBytesRequired) {
+      buf.grow(maxBytesRequired);
     }
-    encodeQueueSize = 0;
-    ordinal = 0;
-    indicator = 0;
-  }
-
-  @Override
-  public void close() throws IOException {
-    if (ordinal != 0) {
-      encodeChunk();
+    
+    buf.bytes[buf.length++] = ((byte) indicator);
+    for (int i = 0; i < encodeQueue.length; i++) {
+      VInt8.encode(encodeQueue.ints[i], buf);
     }
-    encoder.close();
-    super.close();
+    
+    reset();
   }
 
   @Override
-  public void reInit(OutputStream out) {
-    encoder.reInit(out);
-    super.reInit(out);
+  protected void reset() {
     ordinal = 0;
     indicator = 0;
-    encodeQueueSize = 0;
+    encodeQueue.length = 0;
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.InputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,10 +21,8 @@ import java.io.InputStream;
  */
 
 /**
- * An {@link IntDecoder} which wraps another {@link IntDecoder} and reverts the
- * d-gap that was encoded by {@link DGapIntEncoder}. The wrapped decoder
- * performs the actual decoding, while this class simply adds the decoded value
- * to the previous value.
+ * An {@link IntDecoder} which wraps another decoder and reverts the d-gap that
+ * was encoded by {@link DGapIntEncoder}.
  * 
  * @lucene.experimental
  */
@@ -32,26 +30,23 @@ public class DGapIntDecoder extends IntD
 
   private final IntDecoder decoder;
 
-  private int prev = 0;
-
   public DGapIntDecoder(IntDecoder decoder) {
     this.decoder = decoder;
   }
 
   @Override
-  public long decode() throws IOException {
-    long decode = decoder.decode();
-    if (decode == EOS) {
-      return EOS;
-    }
-
-    return prev += decode;
+  protected void reset() {
+    decoder.reset();
   }
-
+  
   @Override
-  public void reInit(InputStream in) {
-    decoder.reInit(in);
-    prev = 0;
+  protected void doDecode(BytesRef buf, IntsRef values, int upto) {
+    decoder.doDecode(buf, values, upto);
+    int prev = 0;
+    for (int i = 0; i < values.length; i++) {
+      values.ints[i] += prev;
+      prev = values.ints[i];
+    }
   }
 
   @Override

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.OutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -27,7 +27,7 @@ import java.io.OutputStream;
  * space) if the values are 'close' to each other.
  * <p>
  * <b>NOTE:</b> this encoder assumes the values are given to
- * {@link #encode(int)} in an ascending sorted manner, which ensures only
+ * {@link #encode(IntsRef, BytesRef)} in an ascending sorted manner, which ensures only
  * positive values are encoded and thus yields better performance. If you are
  * not sure whether the values are sorted or not, it is possible to chain this
  * encoder with {@link SortingIntEncoder} to ensure the values will be
@@ -37,17 +37,20 @@ import java.io.OutputStream;
  */
 public class DGapIntEncoder extends IntEncoderFilter {
 
-  private int prev = 0;
-
   /** Initializes with the given encoder. */
   public DGapIntEncoder(IntEncoder encoder) {
     super(encoder);
   }
 
   @Override
-  public void encode(int value) throws IOException {
-    encoder.encode(value - prev);
-    prev = value;
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    int prev = 0;
+    for (int i = values.offset; i < upto; i++) {
+      int tmp = values.ints[i];
+      values.ints[i] -= prev;
+      prev = tmp;
+    }
+    encoder.doEncode(values, buf, upto);
   }
 
   @Override
@@ -56,12 +59,6 @@ public class DGapIntEncoder extends IntE
   }
   
   @Override
-  public void reInit(OutputStream out) {
-    super.reInit(out);
-    prev = 0;
-  }
-
-  @Override
   public String toString() {
     return "DGap (" + encoder.toString() + ")";
   }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.InputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,20 +21,17 @@ import java.io.InputStream;
  */
 
 /**
- * Decodes data which was encoded by {@link EightFlagsIntEncoder}. Scans
- * the <code>indicator</code>, one flag (1-bits) at a time, and decodes extra
- * data using {@link VInt8IntDecoder}.
+ * Decodes values encoded with {@link EightFlagsIntEncoder}.
  * 
- * @see EightFlagsIntEncoder
  * @lucene.experimental
  */
 public class EightFlagsIntDecoder extends IntDecoder {
 
-  /**
+  /*
    * Holds all combinations of <i>indicator</i> for fast decoding (saves time
    * on real-time bit manipulation)
    */
-  private static final byte[][] decodeTable = new byte[256][8];
+  private static final byte[][] DECODE_TABLE = new byte[256][8];
 
   /** Generating all combinations of <i>indicator</i> into separate flags. */
   static {
@@ -42,45 +39,36 @@ public class EightFlagsIntDecoder extend
       --i;
       for (int j = 8; j != 0;) {
         --j;
-        decodeTable[i][j] = (byte) ((i >>> j) & 0x1);
+        DECODE_TABLE[i][j] = (byte) ((i >>> j) & 0x1);
       }
     }
   }
 
-  private final IntDecoder decoder = new VInt8IntDecoder();
-
-  /** The indicator for decoding a chunk of 8 integers. */
-  private int indicator;
-
-  /** Used as an ordinal of 0 - 7, as the decoder decodes chunks of 8 integers. */
-  private int ordinal = 0;
-
   @Override
-  public long decode() throws IOException { 
-    // If we've decoded 8 integers, read the next indicator.
-    if ((ordinal & 0x7) == 0) {
-      indicator = in.read();
-      if (indicator < 0) {
-        return EOS;
+  protected void doDecode(BytesRef buf, IntsRef values, int upto) {
+    while (buf.offset < upto) {
+      // read indicator
+      int indicator = buf.bytes[buf.offset++] & 0xFF;
+      int ordinal = 0;
+
+      int capacityNeeded = values.length + 8;
+      if (values.ints.length < capacityNeeded) {
+        values.grow(capacityNeeded);
       }
-      ordinal = 0;
-    }
 
-    if (decodeTable[indicator][ordinal++] == 0) {
-      // decode the value from the stream.
-      long decode = decoder.decode();
-      return decode == EOS ? EOS : decode + 2;
+      // process indicator, until we read 8 values, or end-of-buffer
+      while (ordinal != 8) {
+        if (DECODE_TABLE[indicator][ordinal++] == 0) {
+          if (buf.offset == upto) { // end of buffer
+            return;
+          }
+          // decode the value from the stream.
+          values.ints[values.length++] = VInt8.decode(buf) + 2; 
+        } else {
+          values.ints[values.length++] = 1;
+        }
+      }
     }
-
-    return 1;
-  }
-
-  @Override
-  public void reInit(InputStream in) {
-    super.reInit(in);
-    decoder.reInit(in);
-    ordinal = 0;
-    indicator = 0;
   }
 
   @Override

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,6 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,14 +21,15 @@ import java.io.IOException;
  */
 
 /**
- * A {@link ChunksIntEncoder} which encodes data in chunks of 8. Every group starts with a single
- * byte (called indicator) which represents 8 - 1 bit flags, where the value:
+ * A {@link ChunksIntEncoder} which encodes data in chunks of 8. Every group
+ * starts with a single byte (called indicator) which represents 8 - 1 bit
+ * flags, where the value:
  * <ul>
  * <li>1 means the encoded value is '1'
  * <li>0 means the value is encoded using {@link VInt8IntEncoder}, and the
  * encoded bytes follow the indicator.<br>
- * Since value 0 is illegal, and 1 is encoded in the indicator, the actual
- * value that is encoded is <code>value-2</code>, which saves some more bits.
+ * Since value 0 is illegal, and 1 is encoded in the indicator, the actual value
+ * that is encoded is <code>value-2</code>, which saves some more bits.
  * </ul>
  * Encoding example:
  * <ul>
@@ -46,28 +48,36 @@ import java.io.IOException;
  */
 public class EightFlagsIntEncoder extends ChunksIntEncoder {
 
-  /**
+  /*
    * Holds all combinations of <i>indicator</i> flags for fast encoding (saves
    * time on bit manipulation at encode time)
    */
-  private static byte[] encodeTable = new byte[] { 0x1, 0x2, 0x4, 0x8, 0x10, 0x20, 0x40, (byte) 0x80 };
+  private static final byte[] ENCODE_TABLE = new byte[] { 0x1, 0x2, 0x4, 0x8, 0x10, 0x20, 0x40, (byte) 0x80 };
 
   public EightFlagsIntEncoder() {
     super(8);
   }
 
   @Override
-  public void encode(int data) throws IOException {
-    if (data == 1) {
-      indicator |= encodeTable[ordinal];
-    } else {
-      encodeQueue[encodeQueueSize++] = data - 2;
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    for (int i = values.offset; i < upto; i++) {
+      int value = values.ints[i];
+      if (value == 1) {
+        indicator |= ENCODE_TABLE[ordinal];
+      } else {
+        encodeQueue.ints[encodeQueue.length++] = value - 2;
+      }
+      ++ordinal;
+      
+      // encode the chunk and the indicator
+      if (ordinal == 8) {
+        encodeChunk(buf);
+      }
     }
-    ++ordinal;
-
-    // If 8 values were encoded thus far, 'flush' them including the indicator.
-    if ((ordinal & 0x7) == 0) {
-      encodeChunk();
+    
+    // encode remaining values
+    if (ordinal != 0) {
+      encodeChunk(buf);
     }
   }
 
@@ -78,7 +88,7 @@ public class EightFlagsIntEncoder extend
 
   @Override
   public String toString() {
-    return "EightFlags (" + encoder.toString() + ")";
+    return "EightFlags (VInt)";
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.InputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,11 +21,8 @@ import java.io.InputStream;
  */
 
 /**
- * Decodes data which was encoded by {@link FourFlagsIntEncoder}. Scans
- * the <code>indicator</code>, one flag (1-bits) at a time, and decodes extra
- * data using {@link VInt8IntDecoder}.
+ * Decodes values encoded with {@link FourFlagsIntEncoder}.
  * 
- * @see FourFlagsIntEncoder
  * @lucene.experimental
  */
 public class FourFlagsIntDecoder extends IntDecoder {
@@ -34,7 +31,7 @@ public class FourFlagsIntDecoder extends
    * Holds all combinations of <i>indicator</i> for fast decoding (saves time
    * on real-time bit manipulation)
    */
-  private final static byte[][] decodeTable = new byte[256][4];
+  private final static byte[][] DECODE_TABLE = new byte[256][4];
 
   /** Generating all combinations of <i>indicator</i> into separate flags. */
   static {
@@ -42,46 +39,36 @@ public class FourFlagsIntDecoder extends
       --i;
       for (int j = 4; j != 0;) {
         --j;
-        decodeTable[i][j] = (byte) ((i >>> (j << 1)) & 0x3);
+        DECODE_TABLE[i][j] = (byte) ((i >>> (j << 1)) & 0x3);
       }
     }
   }
 
-  private final IntDecoder decoder = new VInt8IntDecoder();
-
-  /** The indicator for decoding a chunk of 4 integers. */
-  private int indicator;
-
-  /** Used as an ordinal of 0 - 3, as the decoder decodes chunks of 4 integers. */
-  private int ordinal = 0;
-
   @Override
-  public long decode() throws IOException {
-    // If we've decoded 8 integers, read the next indicator.
-    if ((ordinal & 0x3) == 0) {
-      indicator = in.read();
-      if (indicator < 0) {
-        return EOS;
+  protected void doDecode(BytesRef buf, IntsRef values, int upto) {
+    while (buf.offset < upto) {
+      // read indicator
+      int indicator = buf.bytes[buf.offset++] & 0xFF;
+      int ordinal = 0;
+      
+      int capacityNeeded = values.length + 4;
+      if (values.ints.length < capacityNeeded) {
+        values.grow(capacityNeeded);
+      }
+      
+      while (ordinal != 4) {
+        byte decodeVal = DECODE_TABLE[indicator][ordinal++];
+        if (decodeVal == 0) {
+          if (buf.offset == upto) { // end of buffer
+            return;
+          }
+          // decode the value from the stream.
+          values.ints[values.length++] = VInt8.decode(buf) + 4;
+        } else {
+          values.ints[values.length++] = decodeVal;
+        }
       }
-      ordinal = 0;
-    }
-
-    byte decodeVal = decodeTable[indicator][ordinal++];
-    if (decodeVal == 0) {
-      // decode the value from the stream.
-      long decode = decoder.decode();
-      return decode == EOS ? EOS : decode + 4;
     }
-
-    return decodeVal;
-  }
-
-  @Override
-  public void reInit(InputStream in) {
-    super.reInit(in);
-    decoder.reInit(in);
-    ordinal = 0;
-    indicator = 0;
   }
 
   @Override

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,6 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -48,11 +49,11 @@ import java.io.IOException;
  */
 public class FourFlagsIntEncoder extends ChunksIntEncoder {
 
-  /**
+  /*
    * Holds all combinations of <i>indicator</i> flags for fast encoding (saves
    * time on bit manipulation @ encode time)
    */
-  private static byte[][] encodeTable = new byte[][] {
+  private static final byte[][] ENCODE_TABLE = new byte[][] {
     new byte[] { 0x00, 0x00, 0x00, 0x00 },
     new byte[] { 0x01, 0x04, 0x10, 0x40 },
     new byte[] { 0x02, 0x08, 0x20, (byte) 0x80 },
@@ -63,26 +64,26 @@ public class FourFlagsIntEncoder extends
     super(4);
   }
 
-  /**
-   * Small values (<=3) are stored in the <code>indicator</code> while larger
-   * values are saved for later encoding in the {@link #encodeQueue}. Since
-   * Vint8 will only encode values larger or equal to 4, the values saves for
-   * encoded are transformed to (value - 4).<br>
-   * When a chunk is ready (got 4 values), the {@link #encodeChunk()}
-   * takes control.
-   */
   @Override
-  public void encode(int data) throws IOException {
-    if (data <= 3) {
-      indicator |= encodeTable[data][ordinal];
-    } else {
-      encodeQueue[encodeQueueSize++] = data - 4;
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    for (int i = values.offset; i < upto; i++) {
+      int value = values.ints[i];
+      if (value <= 3) {
+        indicator |= ENCODE_TABLE[value][ordinal];
+      } else {
+        encodeQueue.ints[encodeQueue.length++] = value - 4;
+      }
+      ++ordinal;
+      
+      // encode the chunk and the indicator
+      if (ordinal == 4) {
+        encodeChunk(buf);
+      }
     }
-    ++ordinal;
-
-    // If 4 values were encoded thus far, 'flush' them including the indicator.
-    if ((ordinal & 0x3) == 0) {
-      encodeChunk();
+    
+    // encode remaining values
+    if (ordinal != 0) {
+      encodeChunk(buf);
     }
   }
 
@@ -93,7 +94,7 @@ public class FourFlagsIntEncoder extends
 
   @Override
   public String toString() {
-    return "FourFlags (" + encoder.toString() + ")";
+    return "FourFlags (VInt)";
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.InputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,33 +21,50 @@ import java.io.InputStream;
  */
 
 /**
- * Decodes integers from a set {@link InputStream}. For re-usability, the
- * decoder's input stream can be set by ({@link #reInit(InputStream)}).
- * By design, Decoders are NOT thread-safe.
+ * Decodes integers from a set {@link BytesRef}.
  * 
  * @lucene.experimental
  */
 public abstract class IntDecoder {
   
-  /** A special long value which is used to indicate end-of-stream has reached. */
-  public static final long EOS = 0x100000000L;
-
-  /** Input stream from which the encoded bytes are read */
-  protected InputStream in;
-
-  /** Sets the input stream from which the encoded data is read. */
-  public void reInit(InputStream in) {
-    this.in = in;
+  /**
+   * Performs the actual decoding. Values should be read from
+   * {@link BytesRef#offset} up to {@code upto}. Also, {@code values} offset and
+   * length are set to 0 and the encoder is expected to update
+   * {@link IntsRef#length}, but not {@link IntsRef#offset}.
+   * 
+   * <p>
+   * <b>NOTE:</b> it is ok to use the buffer's offset as the current position in
+   * the buffer (and modify it), it will be reset by
+   * {@link #decode(BytesRef, IntsRef)}.
+   */
+  protected abstract void doDecode(BytesRef buf, IntsRef values, int upto);
+  
+  /**
+   * Called before {@link #doDecode(BytesRef, IntsRef, int)} so that decoders
+   * can reset their state.
+   */
+  protected void reset() {
+    // do nothing by default
   }
   
   /**
-   * Decodes data received from the input stream, and returns one decoded
-   * integer. If end of stream is reached, {@link #EOS} is returned.
-   * 
-   * @return one decoded integer as long or {@link #EOS} if end-of-stream
-   *         reached.
-   * @throws IOException if an I/O error occurs
+   * Decodes the values from the buffer into the given {@link IntsRef}. Note
+   * that {@code values.offset} and {@code values.length} are set to 0.
    */
-  public abstract long decode() throws IOException;
+  public final void decode(BytesRef buf, IntsRef values) {
+    values.offset = values.length = 0; // must do that because we cannot grow() them otherwise
+    
+    // some decoders may use the buffer's offset as a position index, so save
+    // current offset.
+    int bufOffset = buf.offset;
+    
+    reset();
+    doDecode(buf, values, buf.offset + buf.length);
+    assert values.offset == 0 : "offset should not have been modified by the decoder.";
+    
+    // fix offset
+    buf.offset = bufOffset;
+  }
 
 }