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 [16/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/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/CompactLabelToOrdinal.java Fri Jan 18 18:30:54 2013
@@ -29,8 +29,6 @@ import java.util.Iterator;
 
 import org.apache.lucene.facet.taxonomy.CategoryPath;
 
-// TODO: maybe this could use an FST instead...
-
 /**
  * This is a very efficient LabelToOrdinal implementation that uses a
  * CharBlockArray to store all labels and a configurable number of HashArrays to
@@ -59,8 +57,8 @@ public class CompactLabelToOrdinal exten
 
   public static final float DefaultLoadFactor = 0.15f;
 
-  static final char TerminatorChar = 0xffff;
-  private static final int Collision = -5;
+  static final char TERMINATOR_CHAR = 0xffff;
+  private static final int COLLISION = -5;
 
   private HashArray[] hashArrays;
   private CollisionMap collisionMap;
@@ -103,9 +101,7 @@ public class CompactLabelToOrdinal exten
 
   private void init() {
     labelRepository = new CharBlockArray();
-    try {
-      new CategoryPath().serializeAppendTo(labelRepository);
-    } catch (IOException e) { }  //can't happen 
+    CategoryPathUtils.serialize(CategoryPath.EMPTY, labelRepository);
 
     int c = this.capacity;
     for (int i = 0; i < this.hashArrays.length; i++) {
@@ -116,7 +112,7 @@ public class CompactLabelToOrdinal exten
 
   @Override
   public void addLabel(CategoryPath label, int ordinal) {
-    if (this.collisionMap.size() > this.threshold) {
+    if (collisionMap.size() > threshold) {
       grow();
     }
 
@@ -127,43 +123,22 @@ public class CompactLabelToOrdinal exten
       }
     }
 
-    int prevVal = this.collisionMap.addLabel(label, hash, ordinal);
-    if (prevVal != ordinal) {
-      throw new IllegalArgumentException("Label already exists: " +
-          label.toString('/') + " prev ordinal " + prevVal);
-    }
-  }
-
-  @Override
-  public void addLabel(CategoryPath label, int prefixLen, int ordinal) {
-    if (this.collisionMap.size() > this.threshold) {
-      grow();
-    }
-
-    int hash = CompactLabelToOrdinal.stringHashCode(label, prefixLen);
-    for (int i = 0; i < this.hashArrays.length; i++) {
-      if (addLabel(this.hashArrays[i], label, prefixLen, hash, ordinal)) {
-        return;
-      }
-    }
-
-    int prevVal = this.collisionMap.addLabel(label, prefixLen, hash, ordinal);
+    int prevVal = collisionMap.addLabel(label, hash, ordinal);
     if (prevVal != ordinal) {
-      throw new IllegalArgumentException("Label already exists: " +
-          label.toString('/', prefixLen) + " prev ordinal " + prevVal);
+      throw new IllegalArgumentException("Label already exists: " + label.toString('/') + " prev ordinal " + prevVal);
     }
   }
 
   @Override
   public int getOrdinal(CategoryPath label) {
     if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
     int hash = CompactLabelToOrdinal.stringHashCode(label);
     for (int i = 0; i < this.hashArrays.length; i++) {
       int ord = getOrdinal(this.hashArrays[i], label, hash);
-      if (ord != Collision) {
+      if (ord != COLLISION) {
         return ord;
       }
     }
@@ -171,23 +146,6 @@ public class CompactLabelToOrdinal exten
     return this.collisionMap.get(label, hash);
   }
 
-  @Override
-  public int getOrdinal(CategoryPath label, int prefixLen) {
-    if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    int hash = CompactLabelToOrdinal.stringHashCode(label, prefixLen);
-    for (int i = 0; i < this.hashArrays.length; i++) {
-      int ord = getOrdinal(this.hashArrays[i], label, prefixLen, hash);
-      if (ord != Collision) {
-        return ord;
-      }
-    }
-
-    return this.collisionMap.get(label, prefixLen, hash);
-  }
-
   private void grow() {
     HashArray temp = this.hashArrays[this.hashArrays.length - 1];
 
@@ -241,39 +199,13 @@ public class CompactLabelToOrdinal exten
     }
   }
 
-  private boolean addLabel(HashArray a, CategoryPath label, int hash,
-                            int ordinal) {
+  private boolean addLabel(HashArray a, CategoryPath label, int hash, int ordinal) {
     int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
     int offset = a.offsets[index];
 
     if (offset == 0) {
       a.offsets[index] = this.labelRepository.length();
-      try {
-        label.serializeAppendTo(this.labelRepository);
-      } catch (IOException e) {
-        // can't happen - LabelRepository.append() never throws an
-        // exception
-      }
-      a.cids[index] = ordinal;
-      return true;
-    }
-
-    return false;
-  }
-
-  private boolean addLabel(HashArray a, CategoryPath label, int prefixLen,
-                            int hash, int ordinal) {
-    int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
-    int offset = a.offsets[index];
-
-    if (offset == 0) {
-      a.offsets[index] = this.labelRepository.length();
-      try {
-        label.serializeAppendTo(prefixLen, this.labelRepository);
-      } catch (IOException e) {
-        // can't happen - LabelRepository.append() never throws an
-        // exception
-      }
+      CategoryPathUtils.serialize(label, labelRepository);
       a.cids[index] = ordinal;
       return true;
     }
@@ -313,43 +245,23 @@ public class CompactLabelToOrdinal exten
 
   private int getOrdinal(HashArray a, CategoryPath label, int hash) {
     if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
-    int offset = a.offsets[index];
-    if (offset == 0) {
-      return LabelToOrdinal.InvalidOrdinal;
-    }
-
-    if (label.equalsToSerialized(labelRepository, offset)) {
-      return a.cids[index];
-    }
-
-    return Collision;
-  }
-
-  private int getOrdinal(HashArray a, CategoryPath label, int prefixLen, int hash) {
-    if (label == null) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
-    int index = CompactLabelToOrdinal.indexFor(hash, a.offsets.length);
+    int index = indexFor(hash, a.offsets.length);
     int offset = a.offsets[index];
     if (offset == 0) {
-      return LabelToOrdinal.InvalidOrdinal;
+      return LabelToOrdinal.INVALID_ORDINAL;
     }
 
-    if (label.equalsToSerialized(prefixLen, labelRepository, offset)) {
+    if (CategoryPathUtils.equalsToSerialized(label, labelRepository, offset)) {
       return a.cids[index];
     }
 
-    return Collision;
+    return COLLISION;
   }
 
-  /**
-   * Returns index for hash code h.
-   */
+  /** Returns index for hash code h. */
   static int indexFor(int h, int length) {
     return h & (length - 1);
   }
@@ -378,22 +290,10 @@ public class CompactLabelToOrdinal exten
 
   }
 
-  static int stringHashCode(CategoryPath label, int prefixLen) {
-    int hash = label.hashCode(prefixLen);
-
-    hash = hash ^ ((hash >>> 20) ^ (hash >>> 12));
-    hash = hash ^ (hash >>> 7) ^ (hash >>> 4);
-
-    return hash;
-
-  }
-
   static int stringHashCode(CharBlockArray labelRepository, int offset) {
-    int hash = CategoryPath.hashCodeOfSerialized(labelRepository, offset);
-
+    int hash = CategoryPathUtils.hashCodeOfSerialized(labelRepository, offset);
     hash = hash ^ ((hash >>> 20) ^ (hash >>> 12));
     hash = hash ^ (hash >>> 7) ^ (hash >>> 4);
-
     return hash;
   }
 
@@ -495,25 +395,16 @@ public class CompactLabelToOrdinal exten
       // that array offsets will work).  Since the initial file is machine 
       // generated, I think this should be OK.
       while (offset < l2o.labelRepository.length()) {
-        // First component is numcomponents, so we initialize the hash
-        // to this
-        int ncomponents = l2o.labelRepository.charAt(offset++);
-        int hash = ncomponents;
-        // If ncomponents is 0, then we are done?
-        if (ncomponents != 0) {
-
-          // usedchars is always the last member of the 'ends' array
-          // in serialization. Rather than rebuild the entire array,
-          // assign usedchars to the last value we read in. This will
-          // be slightly more memory efficient.
-          int usedchars = 0;
-          for (int i = 0; i < ncomponents; i++) {
-            usedchars = l2o.labelRepository.charAt(offset++);
-            hash = hash * 31 + usedchars;
-          }
-          // Hash the usedchars for this label
-          for (int i = 0; i < usedchars; i++) {
-            hash = hash * 31 + l2o.labelRepository.charAt(offset++);
+        // identical code to CategoryPath.hashFromSerialized. since we need to
+        // advance offset, we cannot call the method directly. perhaps if we
+        // could pass a mutable Integer or something...
+        int length = (short) l2o.labelRepository.charAt(offset++);
+        int hash = length;
+        if (length != 0) {
+          for (int i = 0; i < length; i++) {
+            int len = (short) l2o.labelRepository.charAt(offset++);
+            hash = hash * 31 + l2o.labelRepository.subSequence(offset, offset + len).hashCode();
+            offset += len;
           }
         }
         // Now that we've hashed the components of the label, do the

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/cl2o/LabelToOrdinal.java Fri Jan 18 18:30:54 2013
@@ -27,7 +27,7 @@ import org.apache.lucene.facet.taxonomy.
 public abstract class LabelToOrdinal {
 
   protected int counter;
-  public static final int InvalidOrdinal = -2;
+  public static final int INVALID_ORDINAL = -2;
 
   /**
    * return the maximal Ordinal assigned so far
@@ -52,22 +52,9 @@ public abstract class LabelToOrdinal {
   public abstract void addLabel(CategoryPath label, int ordinal);
 
   /**
-   * Adds a new label if its not yet in the table.
-   * Throws an {@link IllegalArgumentException} if the same label with
-   * a different ordinal was previoulsy added to this table.
-   */
-  public abstract void addLabel(CategoryPath label, int prefixLen, int ordinal);
-
-  /**
    * @return the ordinal assigned to the given label, 
-   * or {@link #InvalidOrdinal} if the label cannot be found in this table.
+   * or {@link #INVALID_ORDINAL} if the label cannot be found in this table.
    */
   public abstract int getOrdinal(CategoryPath label);
 
-  /**
-   * @return the ordinal assigned to the given label, 
-   * or {@link #InvalidOrdinal} if the label cannot be found in this table.
-   */
-  public abstract int getOrdinal(CategoryPath label, int prefixLen);
-
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/LruTaxonomyWriterCache.java Fri Jan 18 18:30:54 2013
@@ -87,23 +87,6 @@ public class LruTaxonomyWriterCache impl
   }
 
   @Override
-  public synchronized int get(CategoryPath categoryPath, int length) {
-    if (length<0 || length>categoryPath.length()) {
-      length = categoryPath.length();
-    }
-    // TODO (Facet): unfortunately, we make a copy here! we can avoid part of
-    // the copy by creating a wrapper object (but this still creates a new
-    // object). A better implementation of the cache would not use Java's
-    // hash table, but rather some other hash table we can control, and
-    // pass the length parameter into it...
-    Integer res = cache.get(new CategoryPath(categoryPath, length));
-    if (res==null) {
-      return -1;
-    }
-    return res.intValue();
-  }
-
-  @Override
   public synchronized boolean put(CategoryPath categoryPath, int ordinal) {
     boolean ret = cache.put(categoryPath, new Integer(ordinal));
     // If the cache is full, we need to clear one or more old entries
@@ -119,20 +102,4 @@ public class LruTaxonomyWriterCache impl
     return ret;
   }
 
-  @Override
-  public synchronized boolean put(CategoryPath categoryPath, int prefixLen, int ordinal) {
-    boolean ret = cache.put(categoryPath, prefixLen, new Integer(ordinal));
-    // If the cache is full, we need to clear one or more old entries
-    // from the cache. However, if we delete from the cache a recent
-    // addition that isn't yet in our reader, for this entry to be
-    // visible to us we need to make sure that the changes have been
-    // committed and we reopen the reader. Because this is a slow
-    // operation, we don't delete entries one-by-one but rather in bulk
-    // (put() removes the 2/3rd oldest entries).
-    if (ret) { 
-      cache.makeRoomLRU(); 
-    }
-    return ret;
-  }
-
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameHashIntCacheLRU.java Fri Jan 18 18:30:54 2013
@@ -41,6 +41,7 @@ public class NameHashIntCacheLRU extends
 
   @Override
   Object key(CategoryPath name, int prefixLen) {
-    return new Long(name.longHashCode(prefixLen));
+    return new Long(name.subpath(prefixLen).longHashCode());
   }
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/taxonomy/writercache/lru/NameIntCacheLRU.java Fri Jan 18 18:30:54 2013
@@ -68,23 +68,13 @@ class NameIntCacheLRU {
     return res;
   }
 
-  /**
-   * Subclasses can override this to provide caching by e.g. hash of the string.
-   */
+  /** Subclasses can override this to provide caching by e.g. hash of the string. */
   Object key(CategoryPath name) {
-    // Note that a copy constructor (cloning) here is necessary, because a
-    // CategoryPath object is mutable, so we cannot save a reference to an
-    // existing CategoryPath. Subclasses which override this method can
-    // avoid this cloning by, e.g., hashing the name.
-    return new CategoryPath(name);
+    return name;
   }
 
   Object key(CategoryPath name, int prefixLen) {
-    // Note that a copy constructor (cloning) here is necessary, because a
-    // CategoryPath object is mutable, so we cannot save a reference to an
-    // existing CategoryPath. Subclasses which override this method can
-    // avoid this cloning by, e.g., hashing the name.
-    return new CategoryPath(name, prefixLen);
+    return name.subpath(prefixLen);
   }
 
   /**

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java Fri Jan 18 18:30:54 2013
@@ -5,6 +5,8 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.lucene.facet.search.CategoryListIterator;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -33,51 +35,34 @@ 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 */
-  public boolean init() throws IOException {
+  @Override
+  public boolean setNextReader(AtomicReaderContext context) throws IOException {
+    validIterators.clear();
     for (CategoryListIterator cli : iterators) {
-      if (cli.init()) {
+      if (cli.setNextReader(context)) {
         validIterators.add(cli);
       }
     }
     return !validIterators.isEmpty();
   }
-
-  /**
-   * Return a value larger than {@link Integer#MAX_VALUE} only if all
-   * iterators are exhausted
-   */
-  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}
-   */
-  public boolean skipTo(int docId) throws IOException {
-    perDocValidIterators.clear();
+  
+  @Override
+  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/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/PartitionsUtils.java Fri Jan 18 18:30:54 2013
@@ -1,8 +1,6 @@
 package org.apache.lucene.facet.util;
 
-import org.apache.lucene.facet.index.params.CategoryListParams;
 import org.apache.lucene.facet.index.params.FacetIndexingParams;
-import org.apache.lucene.facet.search.params.FacetSearchParams;
 import org.apache.lucene.facet.taxonomy.TaxonomyReader;
 
 /*
@@ -29,25 +27,9 @@ import org.apache.lucene.facet.taxonomy.
  */
 public final class PartitionsUtils {
 
-  /**
-   * Get the offset for a given partition.  That is, what is the minimum number an
-   * ordinal could be for a particular partition. 
-   */
-  public final static int partitionOffset (  FacetIndexingParams iParams, 
-                        int partitionNumber, 
-                        final TaxonomyReader taxonomyReader) {
-    return partitionNumber * partitionSize(iParams, taxonomyReader);
-  }
-
-  /**
-   * @see #partitionOffset(FacetIndexingParams, int, TaxonomyReader)
-   */
-  public final static int partitionOffset (  FacetSearchParams sParams, 
-                        int partitionNumber, 
-                        final TaxonomyReader taxonomyReader) {
-    return partitionOffset(sParams.getFacetIndexingParams(), partitionNumber, taxonomyReader);
-  }
-
+  /** The prefix that is added to the name of the partition. */
+  public static final String PART_NAME_PREFIX = "$part";
+  
   /**
    * Get the partition size in this parameter, or return the size of the taxonomy, which
    * is smaller.  (Guarantees usage of as little memory as possible at search time).
@@ -57,13 +39,6 @@ public final class PartitionsUtils {
   }
 
   /**
-   * @see #partitionSize(FacetIndexingParams, TaxonomyReader)
-   */
-  public final static int partitionSize(FacetSearchParams sParams, final TaxonomyReader taxonomyReader) {
-    return partitionSize(sParams.getFacetIndexingParams(), taxonomyReader);
-  }
-
-  /**
    * Partition number of an ordinal.
    * <p>
    * This allows to locate the partition containing a certain (facet) ordinal.
@@ -74,31 +49,23 @@ public final class PartitionsUtils {
   }
 
   /**
-   * @see #partitionNumber(FacetIndexingParams, int)
-   */
-  public final static int partitionNumber(FacetSearchParams sParams, int ordinal) {
-    return partitionNumber(sParams.getFacetIndexingParams(), ordinal);
-  }
-
-  /**
    * Partition name by category ordinal
    */
-  public final static String partitionNameByOrdinal(  FacetIndexingParams iParams, 
-                            CategoryListParams clParams, 
-                            int ordinal) {
-    int partition = partitionNumber(iParams, ordinal); 
-    return partitionName(clParams, partition);
+  public final static String partitionNameByOrdinal(FacetIndexingParams iParams, int ordinal) {
+    int partition = partitionNumber(iParams, ordinal);
+    return partitionName(partition);
   }
 
-  /** 
-   * Partition name by its number
-   */
-  public final static String partitionName(CategoryListParams clParams, int partition) {
-    String term = clParams.getTerm().text();
+  /** Partition name by its number */
+  public final static String partitionName(int partition) {
+    // TODO would be good if this method isn't called when partitions are not enabled.
+    // perhaps through some specialization code.
     if (partition == 0) {
-      return term; // for backwards compatibility we do not add a partition number in this case
+      // since regular faceted search code goes through this method too,
+      // return the same value for partition 0 and when there are no partitions
+      return "";
     }
-    return term + partition;
+    return PART_NAME_PREFIX + Integer.toString(partition);
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ResultSortUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ResultSortUtils.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ResultSortUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ResultSortUtils.java Fri Jan 18 18:30:54 2013
@@ -144,14 +144,17 @@ public class ResultSortUtils {
       this.accending = accending;
     }
 
+    @Override
     public FacetResultNode insertWithOverflow(FacetResultNode node) {
       resultNodes.add(node);
       return null;
     }
 
+    @Override
     public FacetResultNode pop() {
       if (!isReady) {
         Collections.sort(resultNodes, new Comparator<FacetResultNode>() {
+          @Override
           public int compare(FacetResultNode o1, FacetResultNode o2) {
             int value = Double.compare(o1.getValue(), o2
                 .getValue());
@@ -170,10 +173,12 @@ public class ResultSortUtils {
       return resultNodes.remove(0);
     }
 
+    @Override
     public int size() {
       return resultNodes.size();
     }
 
+    @Override
     public FacetResultNode top() {
       if (resultNodes.size() > 0) {
         return resultNodes.get(0);
@@ -182,11 +187,13 @@ public class ResultSortUtils {
       return null;
     }
 
+    @Override
     public FacetResultNode add(FacetResultNode frn) {
       resultNodes.add(frn);
       return null;
     }
 
+    @Override
     public void clear() {
       resultNodes.clear();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/facet/util/ScoredDocIdsUtils.java Fri Jan 18 18:30:54 2013
@@ -3,17 +3,18 @@ package org.apache.lucene.facet.util;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.facet.search.ScoredDocIDs;
+import org.apache.lucene.facet.search.ScoredDocIDsIterator;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.OpenBitSetDISI;
 
-import org.apache.lucene.facet.search.ScoredDocIDs;
-import org.apache.lucene.facet.search.ScoredDocIDsIterator;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -49,48 +50,57 @@ public class ScoredDocIdsUtils {
    * @param reader holding the number of documents & information about deletions.
    */
   public final static ScoredDocIDs getComplementSet(final ScoredDocIDs docids, final IndexReader reader)
-  throws IOException {
+      throws IOException {
     final int maxDoc = reader.maxDoc();
 
     DocIdSet docIdSet = docids.getDocIDs();
-    final OpenBitSet complement;
-    if (docIdSet instanceof OpenBitSet) {
+    final FixedBitSet complement;
+    if (docIdSet instanceof FixedBitSet) {
       // That is the most common case, if ScoredDocIdsCollector was used.
-      complement = ((OpenBitSet) docIdSet).clone();
+      complement = ((FixedBitSet) docIdSet).clone();
     } else {
-      complement = new OpenBitSetDISI(docIdSet.iterator(), maxDoc);
+      complement = new FixedBitSet(maxDoc);
+      DocIdSetIterator iter = docIdSet.iterator();
+      int doc;
+      while ((doc = iter.nextDoc()) < maxDoc) {
+        complement.set(doc);
+      }
     }
-
     complement.flip(0, maxDoc);
-
-    // Remove all Deletions from the complement set
     clearDeleted(reader, complement);
 
     return createScoredDocIds(complement, maxDoc);
   }
-
-  /**
-   * Clear all deleted documents from a given open-bit-set according to a given reader 
-   */
-  private static void clearDeleted(final IndexReader reader, 
-      final OpenBitSet set) throws IOException {
-
+  
+  /** Clear all deleted documents from a given open-bit-set according to a given reader */
+  private static void clearDeleted(final IndexReader reader, final FixedBitSet set) throws IOException {
+    
     // If there are no deleted docs
     if (!reader.hasDeletions()) {
       return; // return immediately
     }
     
-    Bits bits = MultiFields.getLiveDocs(reader);
-
     DocIdSetIterator it = set.iterator();
-    int doc = DocIdSetIterator.NO_MORE_DOCS;
-    while ((doc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      if (!bits.get(doc)) {
-        set.fastClear(doc);
+    int doc = it.nextDoc(); 
+    for (AtomicReaderContext context : reader.leaves()) {
+      AtomicReader r = context.reader();
+      final int maxDoc = r.maxDoc() + context.docBase;
+      if (doc >= maxDoc) { // skip this segment
+        continue;
+      }
+      if (!r.hasDeletions()) { // skip all docs that belong to this reader as it has no deletions
+        while ((doc = it.nextDoc()) < maxDoc) {}
+        continue;
       }
+      Bits liveDocs = r.getLiveDocs();
+      do {
+        if (!liveDocs.get(doc - context.docBase)) {
+          set.clear(doc);
+        }
+      } while ((doc = it.nextDoc()) < maxDoc);
     }
   }
-
+  
   /**
    * Create a subset of an existing ScoredDocIDs object.
    * 
@@ -118,6 +128,7 @@ public class ScoredDocIdsUtils {
 
     return new ScoredDocIDs() {
 
+      @Override
       public DocIdSet getDocIDs() {
         return new DocIdSet() {
 
@@ -155,19 +166,24 @@ public class ScoredDocIdsUtils {
         };
       }
 
+      @Override
       public ScoredDocIDsIterator iterator() {
         return new ScoredDocIDsIterator() {
 
           int next = -1;
 
+          @Override
           public boolean next() { return ++next < size; }
 
+          @Override
           public float getScore() { return scores[next]; }
 
+          @Override
           public int getDocID() { return docids[next]; }
         };
       }
 
+      @Override
       public int size() { return size; }
 
     };
@@ -191,11 +207,14 @@ public class ScoredDocIdsUtils {
   public static final ScoredDocIDs createScoredDocIds(final DocIdSet docIdSet, final int maxDoc) {
     return new ScoredDocIDs() {
       private int size = -1;
+      @Override
       public DocIdSet getDocIDs() { return docIdSet; }
 
+      @Override
       public ScoredDocIDsIterator iterator() throws IOException {
         final DocIdSetIterator docIterator = docIdSet.iterator();
         return new ScoredDocIDsIterator() {
+          @Override
           public boolean next() {
             try {
               return docIterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
@@ -204,12 +223,15 @@ public class ScoredDocIdsUtils {
             }
           }
 
+          @Override
           public float getScore() { return DEFAULT_SCORE; }
 
+          @Override
           public int getDocID() { return docIterator.docID(); }
         };
       }
 
+      @Override
       public int size() {
         // lazy size computation
         if (size < 0) {
@@ -238,10 +260,12 @@ public class ScoredDocIdsUtils {
       this.maxDoc = reader.maxDoc();
     }
 
+    @Override
     public int size() {  
       return maxDoc;
     }
 
+    @Override
     public DocIdSet getDocIDs() {
       return new DocIdSet() {
 
@@ -260,8 +284,7 @@ public class ScoredDocIdsUtils {
               if (target <= next) {
                 target = next + 1;
               }
-              return next = target >= maxDoc ? NO_MORE_DOCS
-                  : target;
+              return next = target >= maxDoc ? NO_MORE_DOCS : target;
             }
 
             @Override
@@ -279,10 +302,12 @@ public class ScoredDocIdsUtils {
       };
     }
 
+    @Override
     public ScoredDocIDsIterator iterator() {
       try {
         final DocIdSetIterator iter = getDocIDs().iterator();
         return new ScoredDocIDsIterator() {
+          @Override
           public boolean next() {
             try {
               return iter.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
@@ -292,10 +317,12 @@ public class ScoredDocIdsUtils {
             }
           }
 
+          @Override
           public float getScore() {
             return DEFAULT_SCORE;
           }
 
+          @Override
           public int getDocID() {
             return iter.docID();
           }
@@ -324,10 +351,12 @@ public class ScoredDocIdsUtils {
       this.reader = reader;
     }
 
+    @Override
     public int size() {
       return reader.numDocs();
     }
 
+    @Override
     public DocIdSet getDocIDs() {
       return new DocIdSet() {
 
@@ -369,10 +398,12 @@ public class ScoredDocIdsUtils {
       };
     }
 
+    @Override
     public ScoredDocIDsIterator iterator() {
       try {
         final DocIdSetIterator iter = getDocIDs().iterator();
         return new ScoredDocIDsIterator() {
+          @Override
           public boolean next() {
             try {
               return iter.nextDoc() != DocIdSetIterator.NO_MORE_DOCS;
@@ -382,10 +413,12 @@ public class ScoredDocIdsUtils {
             }
           }
 
+          @Override
           public float getScore() {
             return DEFAULT_SCORE;
           }
 
+          @Override
           public int getDocID() {
             return iter.docID();
           }
@@ -396,4 +429,5 @@ public class ScoredDocIdsUtils {
       }
     }
   }
+  
 }
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ArrayHashMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ArrayHashMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ArrayHashMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ArrayHashMap.java Fri Jan 18 18:30:54 2013
@@ -65,10 +65,12 @@ public class ArrayHashMap<K,V> implement
       }
     }
 
+    @Override
     public boolean hasNext() {
       return index != 0;
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -86,6 +88,7 @@ public class ArrayHashMap<K,V> implement
       return lastIndex;
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public void remove() {
       ArrayHashMap.this.remove((K) keys[lastIndex]);
@@ -99,15 +102,18 @@ public class ArrayHashMap<K,V> implement
 
     KeyIterator() { }
 
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public K next() {
       return (K) keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -119,15 +125,18 @@ public class ArrayHashMap<K,V> implement
 
     ValueIterator() { }
 
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public V next() {
       return (V) values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -378,6 +387,7 @@ public class ArrayHashMap<K,V> implement
   }
 
   /** Returns an iterator on the mapped objects. */
+  @Override
   public Iterator<V> iterator() {
     return new ValueIterator();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/FloatToObjectMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/FloatToObjectMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/FloatToObjectMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/FloatToObjectMap.java Fri Jan 18 18:30:54 2013
@@ -73,10 +73,12 @@ public class FloatToObjectMap<T> impleme
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -94,6 +96,7 @@ public class FloatToObjectMap<T> impleme
       return lastIndex;
     }
 
+    @Override
     public void remove() {
       FloatToObjectMap.this.remove(keys[lastIndex]);
     }
@@ -108,14 +111,17 @@ public class FloatToObjectMap<T> impleme
 
     KeyIterator() { }
 
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public float next() {
       return keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -130,15 +136,18 @@ public class FloatToObjectMap<T> impleme
 
     ValueIterator() { }
 
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public T next() {
       return (T) values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -445,6 +454,7 @@ public class FloatToObjectMap<T> impleme
   /**
    * Returns a new iterator for the mapped objects.
    */
+  @Override
   public Iterator<T> iterator() {
     return new ValueIterator();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntHashSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntHashSet.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntHashSet.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntHashSet.java Fri Jan 18 18:30:54 2013
@@ -64,10 +64,12 @@ public class IntHashSet {
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -85,6 +87,7 @@ public class IntHashSet {
       return lastIndex;
     }
 
+    @Override
     public void remove() {
       IntHashSet.this.remove(keys[lastIndex]);
     }
@@ -99,14 +102,17 @@ public class IntHashSet {
 
     KeyIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public int next() {
       return keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToDoubleMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToDoubleMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToDoubleMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToDoubleMap.java Fri Jan 18 18:30:54 2013
@@ -72,10 +72,12 @@ public class IntToDoubleMap {
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -93,6 +95,7 @@ public class IntToDoubleMap {
       return lastIndex;
     }
 
+    @Override
     public void remove() {
       IntToDoubleMap.this.remove(keys[lastIndex]);
     }
@@ -107,14 +110,17 @@ public class IntToDoubleMap {
 
     KeyIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public int next() {
       return keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -129,14 +135,17 @@ public class IntToDoubleMap {
 
     ValueIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public double next() {
       return values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToIntMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToIntMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToIntMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToIntMap.java Fri Jan 18 18:30:54 2013
@@ -71,10 +71,12 @@ public class IntToIntMap {
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -92,6 +94,7 @@ public class IntToIntMap {
       return lastIndex;
     }
 
+    @Override
     public void remove() {
       IntToIntMap.this.remove(keys[lastIndex]);
     }
@@ -106,14 +109,17 @@ public class IntToIntMap {
 
     KeyIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public int next() {
       return keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -127,14 +133,17 @@ public class IntToIntMap {
 
     ValueIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public int next() {
       return values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToObjectMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToObjectMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToObjectMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/IntToObjectMap.java Fri Jan 18 18:30:54 2013
@@ -72,10 +72,12 @@ public class IntToObjectMap<T> implement
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -93,6 +95,7 @@ public class IntToObjectMap<T> implement
       return lastIndex;
     }
 
+    @Override
     public void remove() {
       IntToObjectMap.this.remove(keys[lastIndex]);
     }
@@ -107,14 +110,17 @@ public class IntToObjectMap<T> implement
 
     KeyIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public int next() {
       return keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -129,15 +135,18 @@ public class IntToObjectMap<T> implement
 
     ValueIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public T next() {
       return (T) values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -445,6 +454,7 @@ public class IntToObjectMap<T> implement
   /**
    * Returns a new iterator for the mapped objects.
    */
+  @Override
   public Iterator<T> iterator() {
     return new ValueIterator();
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToFloatMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToFloatMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToFloatMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToFloatMap.java Fri Jan 18 18:30:54 2013
@@ -72,10 +72,12 @@ public class ObjectToFloatMap<K> {
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -93,6 +95,7 @@ public class ObjectToFloatMap<K> {
       return lastIndex;
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public void remove() {
       ObjectToFloatMap.this.remove((K) keys[lastIndex]);
@@ -108,15 +111,18 @@ public class ObjectToFloatMap<K> {
 
     KeyIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public K next() {
       return (K) keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -131,14 +137,17 @@ public class ObjectToFloatMap<K> {
 
     ValueIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public float next() {
       return values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToIntMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToIntMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToIntMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/collections/ObjectToIntMap.java Fri Jan 18 18:30:54 2013
@@ -72,10 +72,12 @@ public class ObjectToIntMap<K> {
       }
     }
 
+    @Override
     public boolean hasNext() {
       return (index != 0);
     }
 
+    @Override
     public int next() {
       // Save the last index visited
       lastIndex = index;
@@ -93,6 +95,7 @@ public class ObjectToIntMap<K> {
       return lastIndex;
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public void remove() {
       ObjectToIntMap.this.remove((K) keys[lastIndex]);
@@ -108,15 +111,18 @@ public class ObjectToIntMap<K> {
 
     KeyIterator() { }
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public K next() {
       return (K) keys[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }
@@ -131,14 +137,17 @@ public class ObjectToIntMap<K> {
 
     ValueIterator() {}
     
+    @Override
     public boolean hasNext() {
       return iterator.hasNext();
     }
 
+    @Override
     public int next() {
       return values[iterator.next()];
     }
 
+    @Override
     public void remove() {
       iterator.remove();
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java Fri Jan 18 18:30:54 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,79 +27,89 @@ 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;
-
+  
   /** Counts the current ordinal of the encoded value. */
   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++) {
+      // it is better if the encoding is inlined like so, and not e.g.
+      // in a utility method
+      int value = encodeQueue.ints[i];
+      if ((value & ~0x7F) == 0) {
+        buf.bytes[buf.length] = (byte) value;
+        buf.length++;
+      } else if ((value & ~0x3FFF) == 0) {
+        buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+        buf.bytes[buf.length + 1] = (byte) (value & 0x7F);
+        buf.length += 2;
+      } else if ((value & ~0x1FFFFF) == 0) {
+        buf.bytes[buf.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
+        buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+        buf.bytes[buf.length + 2] = (byte) (value & 0x7F);
+        buf.length += 3;
+      } else if ((value & ~0xFFFFFFF) == 0) {
+        buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
+        buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
+        buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+        buf.bytes[buf.length + 3] = (byte) (value & 0x7F);
+        buf.length += 4;
+      } else {
+        buf.bytes[buf.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
+        buf.bytes[buf.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
+        buf.bytes[buf.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
+        buf.bytes[buf.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+        buf.bytes[buf.length + 4] = (byte) (value & 0x7F);
+        buf.length += 5;
+      }
     }
-    encoder.close();
-    super.close();
-  }
-
-  @Override
-  public void reInit(OutputStream out) {
-    encoder.reInit(out);
-    super.reInit(out);
+    
     ordinal = 0;
     indicator = 0;
-    encodeQueueSize = 0;
+    encodeQueue.length = 0;
   }
-
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java Fri Jan 18 18:30:54 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,42 +21,32 @@ 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
  */
-public class DGapIntDecoder extends IntDecoder {
+public final class DGapIntDecoder extends IntDecoder {
 
   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;
+  public void decode(BytesRef buf, IntsRef values) {
+    decoder.decode(buf, values);
+    int prev = 0;
+    for (int i = 0; i < values.length; i++) {
+      values.ints[i] += prev;
+      prev = values.ints[i];
     }
-
-    return prev += decode;
-  }
-
-  @Override
-  public void reInit(InputStream in) {
-    decoder.reInit(in);
-    prev = 0;
   }
 
   @Override
   public String toString() {
-    return "DGap (" + decoder.toString() + ")";
+    return "DGap(" + decoder.toString() + ")";
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java Fri Jan 18 18:30:54 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
@@ -35,9 +35,7 @@ import java.io.OutputStream;
  * 
  * @lucene.experimental
  */
-public class DGapIntEncoder extends IntEncoderFilter {
-
-  private int prev = 0;
+public final class DGapIntEncoder extends IntEncoderFilter {
 
   /** Initializes with the given encoder. */
   public DGapIntEncoder(IntEncoder encoder) {
@@ -45,9 +43,15 @@ public class DGapIntEncoder extends IntE
   }
 
   @Override
-  public void encode(int value) throws IOException {
-    encoder.encode(value - prev);
-    prev = value;
+  public void encode(IntsRef values, BytesRef buf) {
+    int prev = 0;
+    int upto = values.offset + values.length;
+    for (int i = values.offset; i < upto; i++) {
+      int tmp = values.ints[i];
+      values.ints[i] -= prev;
+      prev = tmp;
+    }
+    encoder.encode(values, buf);
   }
 
   @Override
@@ -56,14 +60,8 @@ public class DGapIntEncoder extends IntE
   }
   
   @Override
-  public void reInit(OutputStream out) {
-    super.reInit(out);
-    prev = 0;
-  }
-
-  @Override
   public String toString() {
-    return "DGap (" + encoder.toString() + ")";
+    return "DGap(" + encoder.toString() + ")";
   }
   
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java Fri Jan 18 18:30:54 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,50 +39,54 @@ 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;
+  public void decode(BytesRef buf, IntsRef values) {
+    values.offset = values.length = 0;
+    int upto = buf.offset + buf.length;
+    int offset = buf.offset;
+    while (offset < upto) {
+      // read indicator
+      int indicator = buf.bytes[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 (offset == upto) { // end of buffer
+            return;
+          }
+          // it is better if the decoding is inlined like so, and not e.g.
+          // in a utility method
+          int value = 0;
+          while (true) {
+            byte b = buf.bytes[offset++];
+            if (b >= 0) {
+              values.ints[values.length++] = ((value << 7) | b) + 2;
+              break;
+            } else {
+              value = (value << 7) | (b & 0x7F);
+            }
+          }
+        } 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
   public String toString() {
-    return "EightFlags (VInt8)";
+    return "EightFlags(VInt8)";
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java Fri Jan 18 18:30:54 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,38 @@ 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;
+  public void encode(IntsRef values, BytesRef buf) {
+    buf.offset = buf.length = 0;
+    int upto = values.offset + values.length;
+    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 +90,7 @@ public class EightFlagsIntEncoder extend
 
   @Override
   public String toString() {
-    return "EightFlags (" + encoder.toString() + ")";
+    return "EightFlags(VInt)";
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java Fri Jan 18 18:30:54 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,51 +39,54 @@ 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;
+  public void decode(BytesRef buf, IntsRef values) {
+    values.offset = values.length = 0;
+    int upto = buf.offset + buf.length;
+    int offset = buf.offset;
+    while (offset < upto) {
+      // read indicator
+      int indicator = buf.bytes[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 (offset == upto) { // end of buffer
+            return;
+          }
+          // it is better if the decoding is inlined like so, and not e.g.
+          // in a utility method
+          int value = 0;
+          while (true) {
+            byte b = buf.bytes[offset++];
+            if (b >= 0) {
+              values.ints[values.length++] = ((value << 7) | b) + 4;
+              break;
+            } else {
+              value = (value << 7) | (b & 0x7F);
+            }
+          }
+        } 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
   public String toString() {
-    return "FourFlags (VInt8)";
+    return "FourFlags(VInt8)";
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java Fri Jan 18 18:30:54 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,28 @@ 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;
+  public void encode(IntsRef values, BytesRef buf) {
+    buf.offset = buf.length = 0;
+    int upto = values.offset + values.length;
+    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 +96,7 @@ public class FourFlagsIntEncoder extends
 
   @Override
   public String toString() {
-    return "FourFlags (" + encoder.toString() + ")";
+    return "FourFlags(VInt)";
   }
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java Fri Jan 18 18:30:54 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,16 @@ 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;
-  }
-  
   /**
-   * 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 abstract void decode(BytesRef buf, IntsRef values);
 
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java Fri Jan 18 18:30:54 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
@@ -21,93 +21,26 @@ import java.io.OutputStream;
  */
 
 /**
- * Encodes integers to a set {@link OutputStream}. Extending classes need to
- * override {@link #encode(int)} to encode the value using their encoding
- * algorithm. The default implementation of {@link #close()} closes the set
- * {@link OutputStream}.
- * <p>
- * The default {@link #IntEncoder() constructor} is provided for convenience
- * only. One must call {@link #reInit(OutputStream)} before calling
- * {@link #encode(int)} or {@link #close()}.
- * <p>
- * For convenience, each encoder implements {@link #createMatchingDecoder()} for
- * easy access to the matching decoder.
- * <p>
- * <b>NOTE:</b> some implementations may buffer the encoded values in memory
- * (such as {@link IntEncoderFilter} implementations) and encoding will happen
- * only upon calling {@link #close()}. Therefore it is important to always call
- * {@link #close()} on the encoder at hand.
- * <p>
- * <b>NOTE:</b> encoders are usually not thread safe, unless specifically
- * documented otherwise by an implementation.
+ * Encodes integers to a set {@link BytesRef}. For convenience, each encoder
+ * implements {@link #createMatchingDecoder()} for easy access to the matching
+ * decoder.
  * 
  * @lucene.experimental
  */
 public abstract class IntEncoder {
 
-  protected OutputStream out = null;
+  public IntEncoder() {}
 
   /**
-   * Default constructor, provided here for robustness: if in the future a
-   * constructor with parameters will be added, this might break custom
-   * implementations of this class which call this implicit constructor. So we
-   * make it explicit to avoid any such issue in the future.
+   * Encodes the values to the given buffer. Note that the buffer's offset and
+   * length are set to 0.
    */
-  public IntEncoder() {
-  }
+  public abstract void encode(IntsRef values, BytesRef buf);
 
   /**
-   * Instructs the encoder to finish the encoding process. This method closes
-   * the output stream which was specified by {@link #reInit(OutputStream)
-   * reInit}. An implementation may do here additional cleanup required to
-   * complete the encoding, such as flushing internal buffers, etc.<br>
-   * Once this method was called, no further calls to {@link #encode(int)
-   * encode} should be made before first calling {@link #reInit(OutputStream)
-   * reInit}.
-   * <p>
-   * <b>NOTE:</b> overriding classes should make sure they either call
-   * <code>super.close()</code> or close the output stream themselves.
-   */
-  public void close() throws IOException {
-    if (out != null) {
-      out.close();
-    }
-  }
-
-  /**
-   * Encodes an integer to the output stream given in
-   * {@link #reInit(OutputStream) reInit}
-   */
-  public abstract void encode(int value) throws IOException;
-
-  /**
-   * Returns an {@link IntDecoder} which matches this encoder. Every encoder
-   * must return an {@link IntDecoder} and <code>null</code> is not a valid
-   * value. If an encoder is just a filter, it should at least return its
-   * wrapped encoder's matching decoder.
-   * <p>
-   * <b>NOTE:</b> this method should create a new instance of the matching
-   * decoder and leave the instance sharing to the caller. Returning the same
-   * instance over and over is risky because encoders and decoders are not
-   * thread safe.
+   * Returns an {@link IntDecoder} which can decode the values that were encoded
+   * with this encoder.
    */
   public abstract IntDecoder createMatchingDecoder();
   
-  /**
-   * Reinitializes the encoder with the give {@link OutputStream}. For
-   * re-usability it can be changed without the need to reconstruct a new
-   * object.
-   * <p>
-   * <b>NOTE:</b> after calling {@link #close()}, one <u><i>must</i></u> call
-   * this method even if the output stream itself hasn't changed. An example
-   * case is that the output stream wraps a byte[], and the output stream itself
-   * is reset, but its instance hasn't changed. Some implementations of
-   * {@link IntEncoder} may write some metadata about themselves to the output
-   * stream, and therefore it is imperative that one calls this method before
-   * encoding any data.
-   */
-  public void reInit(OutputStream out) {
-    this.out = out;
-  }
-
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java Fri Jan 18 18:30:54 2013
@@ -1,7 +1,5 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.OutputStream;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,17 +19,7 @@ import java.io.OutputStream;
  */
 
 /**
- * An abstract implementation of {@link IntEncoder} which is served as a filter
- * on the values to encode. An encoder filter wraps another {@link IntEncoder}
- * which does the actual encoding. This allows for chaining filters and
- * encoders, such as: <code><pre class="prettyprint">
- * new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEnoder()));
- * {@link UniqueValuesIntEncoder} followed by {@link DGapIntEncoder}
-  </pre></code>
- * <p>
- * The default implementation implements {@link #close()} by closing the wrapped
- * encoder and {@link #reInit(OutputStream)} by re-initializing the wrapped
- * encoder.
+ * An abstract implementation of {@link IntEncoder} which wraps another encoder.
  * 
  * @lucene.experimental
  */
@@ -43,16 +31,4 @@ public abstract class IntEncoderFilter e
     this.encoder = encoder;
   }
 
-  @Override
-  public void close() throws IOException {
-    // There is no need to call super.close(), since we don't pass the output
-    // stream to super.
-    encoder.close();
-  }
-
-  @Override
-  public void reInit(OutputStream out) {
-    encoder.reInit(out);
-  }
-
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java Fri Jan 18 18:30:54 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,59 +21,66 @@ import java.io.InputStream;
  */
 
 /**
- * Decodes data which was encoded by {@link NOnesIntEncoder}. Uses a
- * {@link FourFlagsIntDecoder} to perform the actual encoding and translates the
- * values back as described in {@link NOnesIntEncoder}.
+ * Decodes values encoded encoded with {@link NOnesIntEncoder}.
  * 
- * @see NOnesIntEncoder
  * @lucene.experimental
  */
 public class NOnesIntDecoder extends FourFlagsIntDecoder {
 
-  /** Number of consecutive '1's to generate upon decoding a '2'. */
-  private int n;
-
-  private int onesCounter;
-
+  // Number of consecutive '1's to generate upon decoding a '2'
+  private final int n;
+  private final IntsRef internalBuffer;
+  
   /**
    * Constructs a decoder with a given N (Number of consecutive '1's which are
    * translated into a single target value '2'.
    */
   public NOnesIntDecoder(int n) {
     this.n = n;
+    // initial size (room for 100 integers)
+    internalBuffer = new IntsRef(100);
   }
 
   @Override
-  public long decode() throws IOException {
-    // If we read '2', we should return n '1's.
-    if (onesCounter > 0) {
-      --onesCounter;
-      return 1;
-    }
-
-    long decode = super.decode();
-    if (decode == 1) {
-      return 1;
+  public void decode(BytesRef buf, IntsRef values) {
+    values.offset = values.length = 0;
+    internalBuffer.length = 0;
+    super.decode(buf, internalBuffer);
+    if (values.ints.length < internalBuffer.length) {
+      // need space for internalBuffer.length to internalBuffer.length*N,
+      // grow mildly at first
+      values.grow(internalBuffer.length * n/2);
     }
-    if (decode == 2) {
-      onesCounter = n - 1;
-      return 1;
+    
+    for (int i = 0; i < internalBuffer.length; i++) {
+      int decode = internalBuffer.ints[i];
+      if (decode == 1) {
+        if (values.length == values.ints.length) {
+          values.grow(values.length + 10); // grow by few items, however not too many
+        }
+        // 1 is 1
+        values.ints[values.length++] = 1;
+      } else if (decode == 2) {
+        if (values.length + n >= values.ints.length) {
+          values.grow(values.length + n); // grow by few items, however not too many
+        }
+        // '2' means N 1's
+        for (int j = 0; j < n; j++) {
+          values.ints[values.length++] = 1;
+        }
+      } else {
+        if (values.length == values.ints.length) {
+          values.grow(values.length + 10); // grow by few items, however not too many
+        }
+        // any other value is val-1
+        values.ints[values.length++] = decode - 1;
+      }
     }
-    if (decode == 3) {
-      return 2;
-    }
-    return decode == EOS ? EOS : decode - 1;
-  }
-
-  @Override
-  public void reInit(InputStream in) {
-    super.reInit(in);
-    onesCounter = 0;
   }
 
   @Override
   public String toString() {
-    return "NOnes (" + n + ") (" + super.toString() + ")";
+    return "NOnes(" + n + ") (" + super.toString() + ")";
   }
 
 }