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

svn commit: r1432028 [2/3] - in /lucene/dev/trunk/lucene: ./ facet/ facet/src/examples/org/apache/lucene/facet/example/association/ facet/src/examples/org/apache/lucene/facet/example/simple/ facet/src/java/org/apache/lucene/facet/associations/ facet/sr...

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationFloatSumAggregator.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/aggregator/associations/AssociationIntSumAggregator.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/cache/CategoryListData.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/params/CountFacetRequest.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/util/MultiCategoryListIterator.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/ChunksIntEncoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntDecoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/DGapIntEncoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntDecoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/EightFlagsIntEncoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntDecoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/FourFlagsIntEncoder.java Fri Jan 11 13:09:06 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntDecoder.java Fri Jan 11 13:09:06 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;
+  }
 
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java Fri Jan 11 13:09:06 2013
@@ -1,8 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.Closeable;
-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
@@ -22,94 +21,47 @@ 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 implements Closeable {
+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.
+   * Performs the actual encoding. Values should be read from
+   * {@link IntsRef#offset} up to {@code upto}. Also, it is guaranteed that
+   * {@code buf's} offset and length are set to 0 and the encoder is expected to
+   * update {@link BytesRef#length}, but not {@link BytesRef#offset}.
    */
-  public IntEncoder() {
-  }
-
+  protected abstract void doEncode(IntsRef values, BytesRef buf, int upto);
+  
   /**
-   * 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.
+   * Called before {@link #doEncode(IntsRef, BytesRef, int)} so that encoders
+   * can reset their state.
    */
-  @Override
-  public void close() throws IOException {
-    if (out != null) {
-      out.close();
-    }
+  protected void reset() {
+    // do nothing by default
   }
 
   /**
-   * Encodes an integer to the output stream given in
-   * {@link #reInit(OutputStream) reInit}
+   * Encodes the values to the given buffer. Note that the buffer's offset and
+   * length are set to 0.
    */
-  public abstract void encode(int value) throws IOException;
+  public final void encode(IntsRef values, BytesRef buf) {
+    buf.offset = buf.length = 0;
+    reset();
+    doEncode(values, buf, values.offset + values.length);
+    assert buf.offset == 0 : "offset should not have been modified by the encoder.";
+  }
 
   /**
-   * 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/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java Fri Jan 11 13:09:06 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
  */
@@ -44,15 +32,8 @@ public abstract class IntEncoderFilter e
   }
 
   @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);
+  public void reset() {
+    encoder.reset();
   }
 
 }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java Fri Jan 11 13:09:06 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,54 +21,65 @@ 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;
-    }
-    if (decode == 2) {
-      onesCounter = n - 1;
-      return 1;
-    }
-    if (decode == 3) {
-      return 2;
-    }
-    return decode == EOS ? EOS : decode - 1;
+  protected void reset() {
+    internalBuffer.length = 0;
+    super.reset();
   }
-
+  
   @Override
-  public void reInit(InputStream in) {
-    super.reInit(in);
-    onesCounter = 0;
+  protected void doDecode(BytesRef buf, IntsRef values, int upto) {
+    super.doDecode(buf, internalBuffer, upto);
+    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);
+    }
+    
+    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;
+      }
+    }
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java Fri Jan 11 13:09:06 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
@@ -50,11 +50,10 @@ import java.io.OutputStream;
  */
 public class NOnesIntEncoder extends FourFlagsIntEncoder {
 
+  private final IntsRef internalBuffer;
+  
   /** Number of consecutive '1's to be translated into single target value '2'. */
-  private int n;
-
-  /** Counts the number of consecutive ones seen. */
-  private int onesCounter = 0;
+  private final int n;
 
   /**
    * Constructs an encoder with a given value of N (N: Number of consecutive
@@ -62,38 +61,48 @@ public class NOnesIntEncoder extends Fou
    */
   public NOnesIntEncoder(int n) {
     this.n = n;
+    internalBuffer = new IntsRef(n);
   }
 
   @Override
-  public void close() throws IOException {
-    // We might have ones in our buffer, encode them as neccesary.
-    while (onesCounter-- > 0) {
-      super.encode(1);
-    }
-
-    super.close();
+  protected void reset() {
+    internalBuffer.length = 0;
+    super.reset();
   }
-
+  
   @Override
-  public void encode(int value) throws IOException {
-    if (value == 1) {
-      // Increment the number of consecutive ones seen so far
-      if (++onesCounter == n) {
-        super.encode(2);
-        onesCounter = 0;
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    // make sure the internal buffer is large enough
+    if (values.length > internalBuffer.ints.length) {
+      internalBuffer.grow(values.length);
+    }
+    
+    int onesCounter = 0;
+    for (int i = values.offset; i < upto; i++) {
+      int value = values.ints[i];
+      if (value == 1) {
+        // every N 1's should be encoded as '2'
+        if (++onesCounter == n) {
+          internalBuffer.ints[internalBuffer.length++] = 2;
+          onesCounter = 0;
+        }
+      } else {
+        // there might have been 1's that we need to encode
+        while (onesCounter > 0) {
+          --onesCounter;
+          internalBuffer.ints[internalBuffer.length++] = 1;
+        }
+        
+        // encode value as value+1
+        internalBuffer.ints[internalBuffer.length++] = value + 1;
       }
-      return;
     }
-
-    // If it's not one - there might have been ones we had to encode prior to
-    // this value
+    // there might have been 1's that we need to encode
     while (onesCounter > 0) {
       --onesCounter;
-      super.encode(1);
+      internalBuffer.ints[internalBuffer.length++] = 1;
     }
-
-    // encode value + 1 --> the translation.
-    super.encode(value + 1);
+    super.doEncode(internalBuffer, buf, internalBuffer.length);
   }
 
   @Override
@@ -102,12 +111,6 @@ public class NOnesIntEncoder extends Fou
   }
 
   @Override
-  public void reInit(OutputStream out) {
-    super.reInit(out);
-    onesCounter = 0;
-  }
-
-  @Override
   public String toString() {
     return "NOnes (" + n + ") (" + super.toString() + ")";
   }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java Fri Jan 11 13:09:06 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.StreamCorruptedException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,41 +21,24 @@ import java.io.StreamCorruptedException;
  */
 
 /**
- * A simple stream decoder which can decode values encoded with
- * {@link SimpleIntEncoder}.
+ * Decodes values encoded with {@link SimpleIntEncoder}.
  * 
  * @lucene.experimental
  */
 public class SimpleIntDecoder extends IntDecoder {
 
-  /**
-   * reusable buffer - allocated only once as this is not a thread-safe object
-   */
-  private byte[] buffer = new byte[4];
-
   @Override
-  public long decode() throws IOException {
-
-    // we need exactly 4 bytes to decode an int in this decoder impl, otherwise, throw an exception
-    int offset = 0;
-    while (offset < 4) {
-      int nRead = in.read(buffer, offset, 4 - offset);
-      if (nRead == -1) {
-        if (offset > 0) {
-          throw new StreamCorruptedException(
-              "Need 4 bytes for decoding an int, got only " + offset);
-        }
-        return EOS;
+  protected void doDecode(BytesRef buf, IntsRef values, int upto) {
+    while (buf.offset < upto) {
+      if (values.length == values.ints.length) {
+        values.grow(values.length + 10); // grow by few items, however not too many
       }
-      offset += nRead;
+      values.ints[values.length++] = 
+          ((buf.bytes[buf.offset++] & 0xFF) << 24) | 
+          ((buf.bytes[buf.offset++] & 0xFF) << 16) | 
+          ((buf.bytes[buf.offset++] & 0xFF) <<  8) | 
+          (buf.bytes[buf.offset++] & 0xFF);
     }
-
-    int v = buffer[3] & 0xff;
-    v |= (buffer[2] << 8) & 0xff00;
-    v |= (buffer[1] << 16) & 0xff0000;
-    v |= (buffer[0] << 24) & 0xff000000;
-
-    return v;
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java Fri Jan 11 13:09:06 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
@@ -26,22 +27,21 @@ import java.io.IOException;
  */
 public class SimpleIntEncoder extends IntEncoder {
 
-  /**
-   * This method makes sure the value wasn't previously encoded by checking
-   * against the Set. If the value wasn't encoded, it's added to the Set, and
-   * encoded with {#link Vint8#encode}
-   * 
-   * @param value
-   *            an integer to be encoded
-   * @throws IOException
-   *             possibly thrown by the OutputStream
-   */
   @Override
-  public void encode(int value) throws IOException {
-    out.write(value >>> 24);
-    out.write((value >> 16) & 0xFF);
-    out.write((value >> 8) & 0xFF);
-    out.write(value & 0xFF);
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    // ensure there's enough room in the buffer
+    int bytesNeeded = values.length * 4;
+    if (buf.bytes.length < bytesNeeded) {
+      buf.grow(bytesNeeded);
+    }
+    
+    for (int i = values.offset; i < upto; i++) {
+      int value = values.ints[i];
+      buf.bytes[buf.length++] = (byte) (value >>> 24);
+      buf.bytes[buf.length++] = (byte) ((value >> 16) & 0xFF);
+      buf.bytes[buf.length++] = (byte) ((value >> 8) & 0xFF);
+      buf.bytes[buf.length++] = (byte) (value & 0xFF);
+    }
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java Fri Jan 11 13:09:06 2013
@@ -1,9 +1,10 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.OutputStream;
 import java.util.Arrays;
 
+import org.apache.lucene.util.BytesRef;
+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
@@ -23,47 +24,21 @@ import java.util.Arrays;
 
 /**
  * An {@link IntEncoderFilter} which sorts the values to encode in ascending
- * order before encoding them. Encoding therefore happens upon calling
- * {@link #close()}. Since this encoder is usually chained with another encoder
- * that relies on sorted values, it does not offer a default constructor.
+ * order before encoding them.
  * 
  * @lucene.experimental
  */
 public class SortingIntEncoder extends IntEncoderFilter {
 
-  private float grow = 2.0f;
-  private int index = 0;
-  private int[] set = new int[1024];
-
   /** Initializes with the given encoder. */
   public SortingIntEncoder(IntEncoder encoder) {
     super(encoder);
   }
 
   @Override
-  public void close() throws IOException {
-    if (index == 0) {
-      return;
-    }
-
-    Arrays.sort(set, 0, index);
-    for (int i = 0; i < index; i++) {
-      encoder.encode(set[i]);
-    }
-    encoder.close();
-    index = 0;
-
-    super.close();
-  }
-
-  @Override
-  public void encode(int value) throws IOException {
-    if (index == set.length) {
-      int[] newSet = new int[(int) (set.length * grow)];
-      System.arraycopy(set, 0, newSet, 0, set.length);
-      set = newSet;
-    }
-    set[index++] = value;
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    Arrays.sort(values.ints, values.offset, upto);
+    encoder.doEncode(values, buf, upto);
   }
 
   @Override
@@ -72,12 +47,6 @@ public class SortingIntEncoder extends I
   }
   
   @Override
-  public void reInit(OutputStream out) {
-    super.reInit(out);
-    index = 0;
-  }
-
-  @Override
   public String toString() {
     return "Sorting (" + encoder.toString() + ")";
   }

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java Fri Jan 11 13:09:06 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
@@ -22,7 +22,7 @@ import java.io.OutputStream;
 
 /**
  * An {@link IntEncoderFilter} which ensures only unique values are encoded. The
- * implementation assumes the values given to {@link #encode(int)} are sorted.
+ * implementation assumes the values given to {@link #encode(IntsRef, BytesRef)} are sorted.
  * If this is not the case, you can chain this encoder with
  * {@link SortingIntEncoder}.
  * 
@@ -30,26 +30,23 @@ import java.io.OutputStream;
  */
 public final class UniqueValuesIntEncoder extends IntEncoderFilter {
 
-  /**
-   * Denotes an illegal value which we can use to init 'prev' to. Since all
-   * encoded values are integers, this value is init to MAX_INT+1 and is of type
-   * long. Therefore we are guaranteed not to get this value in encode.
-   */
-  private static final long ILLEGAL_VALUE = Integer.MAX_VALUE + 1;
-
-  private long prev = ILLEGAL_VALUE;
-  
   /** Constructs a new instance with the given encoder. */
   public UniqueValuesIntEncoder(IntEncoder encoder) {
     super(encoder);
   }
 
   @Override
-  public void encode(int value) throws IOException {
-    if (prev != value) {
-      encoder.encode(value);
-      prev = value;
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    int prev = values.ints[values.offset];
+    int idx = values.offset + 1;
+    for (int i = idx; i < upto; i++) {
+      if (values.ints[i] != prev) {
+        values.ints[idx++] = values.ints[i];
+        prev = values.ints[i];
+      }
     }
+    values.length = idx - values.offset;
+    encoder.doEncode(values, buf, idx);
   }
 
   @Override
@@ -58,12 +55,6 @@ public final class UniqueValuesIntEncode
   }
   
   @Override
-  public void reInit(OutputStream out) {
-    super.reInit(out);
-    prev = ILLEGAL_VALUE;
-  }
-
-  @Override
   public String toString() {
     return "Unique (" + encoder.toString() + ")";
   }

Copied: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8.java (from r1429680, lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/Vint8.java)
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8.java?p2=lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8.java&p1=lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/Vint8.java&r1=1429680&r2=1432028&rev=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/Vint8.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8.java Fri Jan 11 13:09:06 2013
@@ -1,9 +1,6 @@
-package org.apache.lucene.util;
+package org.apache.lucene.util.encoding;
 
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
+import org.apache.lucene.util.BytesRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -23,206 +20,118 @@ import java.io.OutputStream;
  */
 
 /**
- * Variable-length encoding of 32-bit integers, into 8-bit bytes. A number is encoded as follows:
+ * Variable-length encoding of 32-bit integers, into 8-bit bytes. A number is
+ * encoded as follows:
  * <ul>
- * <li>If it is less than 127 and non-negative (i.e., if the number uses only 7 bits), it is encoded as 
- *  as single byte: 0bbbbbbb.
- * <li>If its highest nonzero bit is greater than bit 6 (0x40), it is represented as a series of
- * bytes, each byte's
- * 7 LSB containing bits from the original value, with the MSB set for all but the last
- * byte. The first encoded byte contains the highest nonzero bits from the
- * original; the second byte contains the next 7 MSB; and so on, with the last byte
- * containing the 7 LSB of the original.
+ * <li>If it is less than 127 and non-negative (i.e., if the number uses only 7
+ * bits), it is encoded as as single byte: 0bbbbbbb.
+ * <li>If its highest nonzero bit is greater than bit 6 (0x40), it is
+ * represented as a series of bytes, each byte's 7 LSB containing bits from the
+ * original value, with the MSB set for all but the last byte. The first encoded
+ * byte contains the highest nonzero bits from the original; the second byte
+ * contains the next 7 MSB; and so on, with the last byte containing the 7 LSB
+ * of the original.
  * </ul>
- * Examples: 
+ * Examples:
  * <ol>
- * <li>n = 117 = 1110101: This has fewer than 8 significant bits, and so is encoded as
- *   01110101 = 0x75.
- * <li>n = 100000 = (binary) 11000011010100000. This has 17 significant bits, and so needs 
- *   three Vint8 bytes. Left-zero-pad it to a multiple of 7 bits, then split it into chunks of 7 
- *   and add an MSB, 0 for the last byte, 1 for the others: 1|0000110 1|0001101 0|0100000
- *   = 0x86 0x8D 0x20.
- * </ol>   
- * This encoder/decoder will correctly handle any 32-bit integer, but for negative numbers,
- * and positive numbers with more than 28 significant bits, encoding requires 5 bytes; this
- * is not an efficient encoding scheme for large
- * positive numbers or any negative number.
- * <p>
- * <b>Compatibility:</b><br>
- * This class has been used in products that have shipped to customers, and is needed to
- * decode legacy data. Do not modify this class in ways that will break compatibility.
+ * <li>n = 117 = 1110101: This has fewer than 8 significant bits, and so is
+ * encoded as 01110101 = 0x75.
+ * <li>n = 100000 = (binary) 11000011010100000. This has 17 significant bits,
+ * and so needs three Vint8 bytes. Left-zero-pad it to a multiple of 7 bits,
+ * then split it into chunks of 7 and add an MSB, 0 for the last byte, 1 for the
+ * others: 1|0000110 1|0001101 0|0100000 = 0x86 0x8D 0x20.
+ * </ol>
+ * {@link #encode(int, BytesRef)} and {@link #decode(BytesRef)} will correctly
+ * handle any 32-bit integer, but for negative numbers, and positive numbers
+ * with more than 28 significant bits, encoding requires 5 bytes; this is not an
+ * efficient encoding scheme for large positive numbers or any negative number.
  * 
  * @lucene.experimental
  */
-public class Vint8 {
+public class VInt8 {
 
-  /**
-   * Because Java lacks call-by-reference, this class boxes the decoding position, which
-   * is initially set by the caller, and returned after decoding, incremented by the number
-   * of bytes processed.
-   */
-  public static class Position {
-    /**
-     * Creates a position value set to zero.
-     */
-    public Position() {
-      // The initial position is zero by default.
-    }
-    /**
-     * Creates a position set to {@code initialPosition}.
-     * @param initialPosition The starting decoding position in the source buffer.
-     */
-    public Position(int initialPosition) {
-      this.pos = initialPosition;
-    }
-    /**
-     * The value passed by reference.
-     */
-    public int pos;
-  }
-
-  /**
-   * Returns the number of bytes needed to encode {@code number}.
-   * @param number The number whose encoded length is needed.
-   * @return The number of bytes needed to encode {@code number}.
-   */
-  public static int bytesNeeded(int number) {
-    if ((number & ~0x7F) == 0) {
-      return 1;
-    } else if ((number & ~0x3FFF) == 0) {
-      return 2;
-    } else if ((number & ~0x1FFFFF) == 0) {
-      return 3;
-    } else if ((number & ~0xFFFFFFF) == 0) {
-      return 4;
-    } else {
-      return 5;
-    }
-  }
-
-  /**
-   * The maximum number of bytes needed to encode a number using {@code Vint8}.
-   */
+  /** The maximum number of bytes needed to encode an integer. */
   public static final int MAXIMUM_BYTES_NEEDED = 5;
-
+  
   /**
-   * Encodes {@code number} to {@code out}.
-   * @param number The value to be written in encoded form, to {@code out}.
-   * @param out The output stream receiving the encoded bytes.
-   * @exception IOException If there is a problem writing to {@code out}.
-   */
-  public static void encode(int number, OutputStream out) throws IOException {
-    if ((number & ~0x7F) == 0) {
-      out.write(number);
-    } else if ((number & ~0x3FFF) == 0) {
-      out.write(0x80 | (number >> 7));
-      out.write(0x7F & number);
-    } else if ((number & ~0x1FFFFF) == 0) {
-      out.write(0x80 | (number >> 14));
-      out.write(0x80 | (number >> 7));
-      out.write(0x7F & number);
-    } else if ((number & ~0xFFFFFFF) == 0) {
-      out.write(0x80 | (number >> 21));
-      out.write(0x80 | (number >> 14));
-      out.write(0x80 | (number >> 7));
-      out.write(0x7F & number);
-    } else {
-      out.write(0x80 | (number >> 28));
-      out.write(0x80 | (number >> 21));
-      out.write(0x80 | (number >> 14));
-      out.write(0x80 | (number >> 7));
-      out.write(0x7F & number);
-    }
-  }
-
-  /** 
-   * Encodes {@code number} into {@code dest}, starting at offset {@code start} from
-   * the beginning of the array. This method assumes {@code dest} is large enough to
-   * hold the required number of bytes.
-   * @param number The number to be encoded.
-   * @param dest The destination array.
-   * @param start The starting offset in the array.
-   * @return The number of bytes used in the array.
-   */
-  public static int encode(int number, byte[] dest, int start) {
-    if ((number & ~0x7F) == 0) {
-      dest[start] = (byte) number;
-      return 1;
-    } else if ((number & ~0x3FFF) == 0) {
-      dest[start] = (byte) (0x80 | ((number & 0x3F80) >> 7));
-      dest[start + 1] = (byte) (number & 0x7F);
-      return 2;
-    } else if ((number & ~0x1FFFFF) == 0) {
-      dest[start] = (byte) (0x80 | ((number & 0x1FC000) >> 14));
-      dest[start + 1] = (byte) (0x80 | ((number & 0x3F80) >> 7));
-      dest[start + 2] = (byte) (number & 0x7F);
-      return 3;
-    } else if ((number & ~0xFFFFFFF) == 0) {
-      dest[start] = (byte) (0x80 | ((number & 0xFE00000) >> 21));
-      dest[start + 1] = (byte) (0x80 | ((number & 0x1FC000) >> 14));
-      dest[start + 2] = (byte) (0x80 | ((number & 0x3F80) >> 7));
-      dest[start + 3] = (byte) (number & 0x7F);
-      return 4;
-    } else {
-      dest[start] = (byte) (0x80 | ((number & 0xF0000000) >> 28));
-      dest[start + 1] = (byte) (0x80 | ((number & 0xFE00000) >> 21));
-      dest[start + 2] = (byte) (0x80 | ((number & 0x1FC000) >> 14));
-      dest[start + 3] = (byte) (0x80 | ((number & 0x3F80) >> 7));
-      dest[start + 4] = (byte) (number & 0x7F);
-      return 5;
-    }
-  }
-
-  /** 
-   * Decodes a 32-bit integer from {@code bytes}, beginning at offset {@code pos.pos}.
-   * The decoded value is returned, and {@code pos.pos} is incremented by the number of
-   * bytes processed.
-   * @param bytes The byte array containing an encoded value.
-   * @param pos On entry, the starting position in the array; on return, one greater
-   * than the position of the last byte decoded in the call.
-   * @return The decoded value.
+   * Decodes an int from the given bytes, starting at {@link BytesRef#offset}.
+   * Returns the decoded bytes and updates {@link BytesRef#offset}.
    */
-  public static int decode(byte[] bytes, Position pos) {
+  public static int decode(BytesRef bytes) {
+    /*
+    This is the original code of this method, but a Hotspot bug
+    corrupted the for-loop of DataInput.readVInt() (see LUCENE-2975)
+    so the loop was unwounded here too, to be on the safe side
     int value = 0;
     while (true) {
-      byte first = bytes[pos.pos];
-      ++pos.pos;
+      byte first = bytes.bytes[bytes.offset++];
       value |= first & 0x7F;
       if ((first & 0x80) == 0) {
         return value;
       }
       value <<= 7;
     }
+    */
+
+    // byte 1
+    byte b = bytes.bytes[bytes.offset++];
+    if (b >= 0) return b;
+    
+    // byte 2
+    int value = b & 0x7F;
+    b = bytes.bytes[bytes.offset++];
+    value = (value << 7) | b & 0x7F;
+    if (b >= 0) return value;
+    
+    // byte 3
+    b = bytes.bytes[bytes.offset++];
+    value = (value << 7) | b & 0x7F;
+    if (b >= 0) return value;
+    
+    // byte 4
+    b = bytes.bytes[bytes.offset++];
+    value = (value << 7) | b & 0x7F;
+    if (b >= 0) return value;
+    
+    // byte 5
+    b = bytes.bytes[bytes.offset++];
+    return (value << 7) | b & 0x7F;
   }
 
   /**
-   * Decodes a 32-bit integer from bytes read from {@code in}. Bytes are read,
-   * one at a time, from {@code in}, and it is assumed they represent a 32-bit
-   * integer encoded using this class's encoding scheme. The decoded value is
-   * returned.
-   * @param in The input stream containing the encoded bytes.
-   * @return The decoded value.
-   * @exception EOFException If the stream ends before a value has been decoded.
-   */
-  public static int decode(InputStream in) throws IOException {
-    int value = 0;
-    while (true) {
-      int first = in.read();
-      if (first < 0) {
-        throw new EOFException();
-      }
-      value |= first & 0x7F;
-      if ((first & 0x80) == 0) {
-        return value;
-      }
-      value <<= 7;
+   * Encodes the given number into bytes, starting at {@link BytesRef#length}.
+   * Assumes that the array is large enough.
+   */
+  public static void encode(int value, BytesRef bytes) {
+    if ((value & ~0x7F) == 0) {
+      bytes.bytes[bytes.length] = (byte) value;
+      bytes.length++;
+    } else if ((value & ~0x3FFF) == 0) {
+      bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+      bytes.bytes[bytes.length + 1] = (byte) (value & 0x7F);
+      bytes.length += 2;
+    } else if ((value & ~0x1FFFFF) == 0) {
+      bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
+      bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+      bytes.bytes[bytes.length + 2] = (byte) (value & 0x7F);
+      bytes.length += 3;
+    } else if ((value & ~0xFFFFFFF) == 0) {
+      bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
+      bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
+      bytes.bytes[bytes.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+      bytes.bytes[bytes.length + 3] = (byte) (value & 0x7F);
+      bytes.length += 4;
+    } else {
+      bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
+      bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
+      bytes.bytes[bytes.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
+      bytes.bytes[bytes.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
+      bytes.bytes[bytes.length + 4] = (byte) (value & 0x7F);
+      bytes.length += 5;
     }
   }
 
-  /**
-   * The default ctor is made private because all methods of this class are static.
-   */
-  private Vint8() {
+  private VInt8() {
     // Just making it impossible to instantiate.
   }
 

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java Fri Jan 11 13:09:06 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,33 +21,19 @@ import java.io.IOException;
  */
 
 /**
- * An {@link IntDecoder} which can decode values encoded by
- * {@link VInt8IntEncoder}.
+ * Decodes values encoded by {@link VInt8IntEncoder}.
  * 
  * @lucene.experimental
  */
 public class VInt8IntDecoder extends IntDecoder {
 
-  private boolean legalEOS = true;
-
   @Override
-  public long decode() throws IOException {
-    int value = 0;
-    while (true) {
-      int first = in.read();
-      if (first < 0) {
-        if (!legalEOS) {
-          throw new IOException("Unexpected End-Of-Stream");
-        }
-        return EOS;
-      }
-      value |= first & 0x7F;
-      if ((first & 0x80) == 0) {
-        legalEOS = true;
-        return value;
+  protected void doDecode(BytesRef buf, IntsRef values, int upto) {
+    while (buf.offset < upto) {
+      if (values.length == values.ints.length) {
+        values.grow(values.length + 10); // grow by few items, however not too many
       }
-      legalEOS = false;
-      value <<= 7;
+      values.ints[values.length++] = VInt8.decode(buf);
     }
   }
 

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java Fri Jan 11 13:09:06 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
@@ -49,27 +50,14 @@ import java.io.IOException;
 public class VInt8IntEncoder extends IntEncoder {
 
   @Override
-  public void encode(int value) throws IOException {
-    if ((value & ~0x7F) == 0) {
-      out.write(value);
-    } else if ((value & ~0x3FFF) == 0) {
-      out.write(0x80 | (value >> 7));
-      out.write(0x7F & value);
-    } else if ((value & ~0x1FFFFF) == 0) {
-      out.write(0x80 | (value >> 14));
-      out.write(0x80 | (value >> 7));
-      out.write(0x7F & value);
-    } else if ((value & ~0xFFFFFFF) == 0) {
-      out.write(0x80 | (value >> 21));
-      out.write(0x80 | (value >> 14));
-      out.write(0x80 | (value >> 7));
-      out.write(0x7F & value);
-    } else {
-      out.write(0x80 | (value >> 28));
-      out.write(0x80 | (value >> 21));
-      out.write(0x80 | (value >> 14));
-      out.write(0x80 | (value >> 7));
-      out.write(0x7F & value);
+  protected void doEncode(IntsRef values, BytesRef buf, int upto) {
+    int maxBytesNeeded = 5 * values.length; // at most 5 bytes per VInt
+    if (buf.bytes.length < maxBytesNeeded) {
+      buf.grow(maxBytesNeeded);
+    }
+    
+    for (int i = values.offset; i < upto; i++) {
+      VInt8.encode(values.ints[i], buf);
     }
   }
 

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html Fri Jan 11 13:09:06 2013
@@ -25,49 +25,8 @@ mechanisms to create new ones. The super
 encoders there is a matching {@link
 org.apache.lucene.util.encoding.IntDecoder} implementation (not all
 encoders need a decoder).
-<p>An encoder encodes the integers that are passed to {@link
-org.apache.lucene.util.encoding.IntEncoder#encode(int) encode} into a
-set output stream (see {@link
-org.apache.lucene.util.encoding.IntEncoder#reInit(OutputStream)
-reInit}). One should always call {@link
-org.apache.lucene.util.encoding.IntEncoder#close() close} when all
-integers have been encoded, to ensure proper finish by the encoder. Some
-encoders buffer values in-memory and encode in batches in order to
-optimize the encoding, and not closing them may result in loss of
-information or corrupt stream.
-<p>A proper and typical usage of an encoder looks like this:
-<blockquote><pre class="prettyprint"><code>
-int[] data = &lt;the values to encode&gt;
-IntEncoder encoder = new VInt8IntEncoder();
-OutputStream out = new ByteArrayOutputStream();
-encoder.reInit(out);
-for (int val : data) {
-  encoder.encode(val);
-}
-encoder.close();
-
-// Print the bytes in binary
-byte[] bytes = out.toByteArray();
-for (byte b : bytes) {
-  System.out.println(Integer.toBinaryString(b));
-}
-</code></pre></blockquote>
-Each encoder also implements {@link
-org.apache.lucene.util.encoding.IntEncoder#createMatchingDecoder()
-createMatchingDecoder} which returns the matching decoder for this encoder.
-As mentioned above, not all encoders have a matching decoder (like some
-encoder filters which are explained next), however every encoder should
-return a decoder following a call to that method. To complete the
-example above, one can easily iterate over the decoded values like this:
-<blockquote><pre class="prettyprint"><code>
-IntDecoder d = e.createMatchingDecoder();
-d.reInit(new ByteArrayInputStream(bytes));
-long val;
-while ((val = d.decode()) != IntDecoder.EOS) {
-  System.out.println(val);
-}
-</code></pre></blockquote>
-<p>Some encoders don't perform any encoding at all, or do not include an
+<p>
+Some encoders don't perform any encoding at all, or do not include an
 encoding logic. Those are called {@link
 org.apache.lucene.util.encoding.IntEncoderFilter}s. A filter is an
 encoder which delegates the encoding task to a given encoder, however
@@ -76,91 +35,6 @@ example is {@link org.apache.lucene.util
 which encodes the gaps between values rather than the values themselves.
 Another example is {@link
 org.apache.lucene.util.encoding.SortingIntEncoder} which sorts all the
-values in ascending order before they are sent for encoding. This
-encoder aggregates the values in its {@link
-org.apache.lucene.util.encoding.IntEncoder#encode(int) encode} implementation 
-and decoding only happens upon calling {@link
-org.apache.lucene.util.encoding.IntEncoder#close() close}.
-<h4>Extending IntEncoder</h4>
-Extending {@link org.apache.lucene.util.encoding.IntEncoder} is a very
-easy task. One only needs to implement {@link
-org.apache.lucene.util.encoding.IntEncoder#encode(int) encode} and
-{@link org.apache.lucene.util.encoding.IntEncoder#createMatchingDecoder()
-createMatchingDecoder} as the base implementation takes care of
-re-initializing the output stream and closing it. The following example
-illustrates how can one write an encoder (and a matching decoder) which
-'tags' the stream with type/ID of the encoder. Such tagging is important
-in scenarios where an application uses different encoders for different
-streams, and wants to manage some sort of mapping between an encoder ID
-to an IntEncoder/Decoder implementation, so a proper decoder will be
-initialized on the fly:
-<blockquote><pre class="prettyprint"><code>
-public class TaggingIntEncoder extends IntEncoderFilter {
-  
-  public TaggingIntEncoder(IntEncoder encoder) {
-    super(encoder);
-  }
-  
-  &#64;Override
-  public void encode(int value) throws IOException {
-    encoder.encode(value);
-  }
-
-  &#64;Override
-  public IntDecoder createMatchingDecoder() {
-    return new TaggingIntDecoder();
-  }
-	
-  &#64;Override
-  public void reInit(OutputStream out) {
-    super.reInit(os);
-    // Assumes the application has a static EncodersMap class which is able to 
-    // return a unique ID for a given encoder.
-    int encoderID = EncodersMap.getID(encoder);
-    this.out.write(encoderID);
-  }
-
-  &#64;Override
-  public String toString() {
-    return "Tagging (" + encoder.toString() + ")";
-  }
-
-}
-</code></pre></blockquote>
-And the matching decoder:
-<blockquote><pre class="prettyprint"><code>
-public class TaggingIntDecoder extends IntDecoder {
-  
-  // Will be initialized upon calling reInit.
-  private IntDecoder decoder;
-  
-  &#64;Override
-  public void reInit(InputStream in) {
-    super.reInit(in);
-    
-    // Read the ID of the encoder that tagged this stream.
-    int encoderID = in.read();
-    
-    // Assumes EncodersMap can return the proper IntEncoder given the ID.
-    decoder = EncodersMap.getEncoder(encoderID).createMatchingDecoder();
-  }
-	
-  &#64;Override
-  public long decode() throws IOException {
-    return decoder.decode();
-  }
-
-  &#64;Override
-  public String toString() {
-    return "Tagging (" + decoder == null ? "none" : decoder.toString() + ")";
-  }
-
-}
-</code></pre></blockquote>
-The example implements <code>TaggingIntEncoder</code> as a filter over another 
-encoder. Even though it does not do any filtering on the actual values, it feels 
-right to present it as a filter. Anyway, this is just an example code and one 
-can choose to implement it however it makes sense to the application. For 
-simplicity, error checking was omitted from the sample code.
+values in ascending order before they are sent for encoding.
 </body>
 </html>
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java?rev=1432028&r1=1432027&r2=1432028&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java (original)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java Fri Jan 11 13:09:06 2013
@@ -4,7 +4,7 @@ import org.junit.Test;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.facet.example.ExampleResult;
-import org.apache.lucene.facet.example.association.AssociationMain;
+import org.apache.lucene.facet.example.association.CategoryAssociationsMain;
 import org.apache.lucene.facet.search.results.FacetResultNode;
 
 /*
@@ -35,8 +35,8 @@ public class TestAssociationExample exte
 
   @Test
   public void testAssociationExamples() throws Exception {
-    assertExampleResult(new AssociationMain().runSumIntAssociationSample(), EXPECTED_INT_SUM_RESULTS);
-    assertExampleResult(new AssociationMain().runSumFloatAssociationSample(), EXPECTED_FLOAT_SUM_RESULTS);
+    assertExampleResult(new CategoryAssociationsMain().runSumIntAssociationSample(), EXPECTED_INT_SUM_RESULTS);
+    assertExampleResult(new CategoryAssociationsMain().runSumFloatAssociationSample(), EXPECTED_FLOAT_SUM_RESULTS);
   }
 
   private void assertExampleResult(ExampleResult res, double[] expectedResults) {