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

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

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoder.java Fri Jan 11 14:39:45 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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/IntEncoderFilter.java Fri Jan 11 14:39:45 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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.InputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -21,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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/NOnesIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.OutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SimpleIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,6 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/SortingIntEncoder.java Fri Jan 11 14:39:45 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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/UniqueValuesIntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,7 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
-import java.io.OutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -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() + ")";
   }

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntDecoder.java Fri Jan 11 14:39:45 2013
@@ -1,6 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/VInt8IntEncoder.java Fri Jan 11 14:39:45 2013
@@ -1,6 +1,7 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -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/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/java/org/apache/lucene/util/encoding/package.html Fri Jan 11 14:39:45 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/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/example/TestAssociationExample.java Fri Jan 11 14:39:45 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) {

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/CategoryListIteratorTest.java Fri Jan 11 14:39:45 2013
@@ -19,8 +19,8 @@ import org.apache.lucene.index.RandomInd
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.UnsafeByteArrayOutputStream;
 import org.apache.lucene.util.encoding.DGapIntEncoder;
 import org.apache.lucene.util.encoding.IntEncoder;
 import org.apache.lucene.util.encoding.SortingIntEncoder;
@@ -49,17 +49,19 @@ public class CategoryListIteratorTest ex
 
   private static final class DataTokenStream extends TokenStream {
 
+    private final PayloadAttribute payload = addAttribute(PayloadAttribute.class);
+    private final BytesRef buf;
+    private final IntEncoder encoder;
+    private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
+    
     private int idx;
-    private PayloadAttribute payload = addAttribute(PayloadAttribute.class);
-    private byte[] buf = new byte[20];
-    UnsafeByteArrayOutputStream ubaos = new UnsafeByteArrayOutputStream(buf);
-    IntEncoder encoder;
     private boolean exhausted = false;
-    private CharTermAttribute term = addAttribute(CharTermAttribute.class);
 
     public DataTokenStream(String text, IntEncoder encoder) {
       this.encoder = encoder;
       term.setEmpty().append(text);
+      buf = new BytesRef();
+      payload.setPayload(buf);
     }
 
     public void setIdx(int idx) {
@@ -73,30 +75,26 @@ public class CategoryListIteratorTest ex
         return false;
       }
 
-      int[] values = data[idx];
-      ubaos.reInit(buf);
-      encoder.reInit(ubaos);
-      for (int val : values) {
-        encoder.encode(val);
-      }
-      encoder.close();
-      payload.setPayload(new BytesRef(buf, 0, ubaos.length()));
-
+      // must copy because encoders may change the buffer
+      encoder.encode(IntsRef.deepCopyOf(data[idx]), buf);
       exhausted = true;
       return true;
     }
 
   }
 
-  static final int[][] data = new int[][] {
-    new int[] { 1, 2 }, new int[] { 3, 4 }, new int[] { 1, 3 }, new int[] { 1, 2, 3, 4 },
+  static final IntsRef[] data = new IntsRef[] {
+    new IntsRef(new int[] { 1, 2 }, 0, 2), 
+    new IntsRef(new int[] { 3, 4 }, 0, 2),
+    new IntsRef(new int[] { 1, 3 }, 0, 2),
+    new IntsRef(new int[] { 1, 2, 3, 4 }, 0, 4)
   };
 
   @Test
-  public void testPayloadIntDecodingIterator() throws Exception {
+  public void testPayloadCategoryListIteraor() throws Exception {
     Directory dir = newDirectory();
-    DataTokenStream dts = new DataTokenStream("1",new SortingIntEncoder(
-        new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))));
+    final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
+    DataTokenStream dts = new DataTokenStream("1",encoder);
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig(TEST_VERSION_CURRENT, 
         new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
     for (int i = 0; i < data.length; i++) {
@@ -108,21 +106,21 @@ public class CategoryListIteratorTest ex
     IndexReader reader = writer.getReader();
     writer.close();
 
-    CategoryListIterator cli = new PayloadIntDecodingIterator(reader, new Term(
-        "f","1"), dts.encoder.createMatchingDecoder());
+    IntsRef ordinals = new IntsRef();
+    CategoryListIterator cli = new PayloadCategoryListIteraor(reader, new Term("f","1"), encoder.createMatchingDecoder());
     cli.init();
     int totalCategories = 0;
     for (int i = 0; i < data.length; i++) {
       Set<Integer> values = new HashSet<Integer>();
       for (int j = 0; j < data[i].length; j++) {
-        values.add(data[i][j]);
+        values.add(data[i].ints[j]);
       }
-      cli.skipTo(i);
-      long cat;
-      while ((cat = cli.nextCategory()) < Integer.MAX_VALUE) {
-        assertTrue("expected category not found: " + cat, values.contains((int) cat));
-        totalCategories ++;
+      cli.getOrdinals(i, ordinals);
+      assertTrue("no ordinals for document " + i, ordinals.length > 0);
+      for (int j = 0; j < ordinals.length; j++) {
+        assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
       }
+      totalCategories += ordinals.length;
     }
     assertEquals("Missing categories!",10,totalCategories);
     reader.close();
@@ -135,8 +133,8 @@ public class CategoryListIteratorTest ex
   @Test
   public void testPayloadIteratorWithInvalidDoc() throws Exception {
     Directory dir = newDirectory();
-    DataTokenStream dts = new DataTokenStream("1",new SortingIntEncoder(
-        new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))));
+    final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
+    DataTokenStream dts = new DataTokenStream("1", encoder);
     // this test requires that no payloads ever be randomly present!
     final Analyzer noPayloadsAnalyzer = new Analyzer() {
       @Override
@@ -162,30 +160,27 @@ public class CategoryListIteratorTest ex
     IndexReader reader = writer.getReader();
     writer.close();
 
-    CategoryListIterator cli = new PayloadIntDecodingIterator(reader, new Term(
-        "f","1"), dts.encoder.createMatchingDecoder());
+    IntsRef ordinals = new IntsRef();
+    CategoryListIterator cli = new PayloadCategoryListIteraor(reader, new Term("f","1"), encoder.createMatchingDecoder());
     assertTrue("Failed to initialize payload iterator", cli.init());
-    int totalCats = 0;
+    int totalCategories = 0;
     for (int i = 0; i < data.length; i++) {
-      // doc no. i
       Set<Integer> values = new HashSet<Integer>();
       for (int j = 0; j < data[i].length; j++) {
-        values.add(data[i][j]);
+        values.add(data[i].ints[j]);
       }
-      boolean hasDoc = cli.skipTo(i);
-      if (hasDoc) {
-        assertTrue("Document " + i + " must not have a payload!", i == 0);
-        long cat;
-        while ((cat = cli.nextCategory()) < Integer.MAX_VALUE) {
-          assertTrue("expected category not found: " + cat, values.contains((int) cat));
-          ++totalCats;
+      cli.getOrdinals(i, ordinals);
+      if (i == 0) {
+        assertTrue("document 0 must have a payload", ordinals.length > 0);
+        for (int j = 0; j < ordinals.length; j++) {
+          assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
         }
+        totalCategories += ordinals.length;
       } else {
-        assertFalse("Document " + i + " must have a payload!", i == 0);
+        assertTrue("only document 0 should have a payload", ordinals.length == 0);
       }
-
     }
-    assertEquals("Wrong number of total categories!", 2, totalCats);
+    assertEquals("Wrong number of total categories!", 2, totalCategories);
 
     reader.close();
     dir.close();

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/DrillDownTest.java Fri Jan 11 14:39:45 2013
@@ -90,7 +90,9 @@ public class DrillDownTest extends Lucen
         paths.add(new CategoryPath("b"));
       }
       FacetFields facetFields = new FacetFields(taxoWriter);
-      facetFields.addFields(doc, paths);
+      if (paths.size() > 0) {
+        facetFields.addFields(doc, paths);
+      }
       writer.addDocument(doc);
     }
     

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/facet/search/TestCategoryListCache.java Fri Jan 11 14:39:45 2013
@@ -6,6 +6,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.util.IntsRef;
 
 import org.junit.After;
 import org.junit.Before;
@@ -118,18 +119,17 @@ public class TestCategoryListCache exten
       @Override
       public CategoryListIterator iterator(int partition)  throws IOException {
         final CategoryListIterator it = cld.iterator(partition);
-        return new CategoryListIterator() {              
+        return new CategoryListIterator() {
           @Override
-          public boolean skipTo(int docId) throws IOException {
-            return it.skipTo(docId);
-          }
-          @Override
-          public long nextCategory() throws IOException {
-            long res = it.nextCategory();
-            if (res>Integer.MAX_VALUE) {
-              return res;
+          public void getOrdinals(int docID, IntsRef ints) throws IOException {
+            it.getOrdinals(docID, ints);
+            for (int i = 0; i < ints.length; i++) {
+              if (ints.ints[i] > 1) {
+                ints.ints[i]--;
+              } else {
+                ints.ints[i]++;
+              }
             }
-            return res>1 ? res-1 : res+1;
           }
           @Override
           public boolean init() throws IOException {

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingSpeed.java Fri Jan 11 14:39:45 2013
@@ -1,21 +1,12 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.text.NumberFormat;
 import java.util.Arrays;
 import java.util.Locale;
 
-import org.apache.lucene.util.encoding.DGapIntEncoder;
-import org.apache.lucene.util.encoding.EightFlagsIntEncoder;
-import org.apache.lucene.util.encoding.FourFlagsIntEncoder;
-import org.apache.lucene.util.encoding.IntDecoder;
-import org.apache.lucene.util.encoding.IntEncoder;
-import org.apache.lucene.util.encoding.NOnesIntEncoder;
-import org.apache.lucene.util.encoding.SortingIntEncoder;
-import org.apache.lucene.util.encoding.UniqueValuesIntEncoder;
-import org.apache.lucene.util.encoding.VInt8IntEncoder;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -40,8 +31,8 @@ public class EncodingSpeed {
   private static int[] data9910 = null;
   private static int[] data501871 = null;
   private static int[] data10k = null;
-  private static String resultsFormat = "%-20s %10s %20d %26s %20d %26s";
-  private static String headerFormat = "%-20s %10s %20s %26s %20s %26s";
+  private static String resultsFormat = "%-60s %10s %20d %26s %20d %26s";
+  private static String headerFormat = "%-60s %10s %20s %26s %20s %26s";
   private static int integers = 100000000;
 
   private static NumberFormat nf;
@@ -53,8 +44,14 @@ public class EncodingSpeed {
     testFacetIDs(data501871, 501871);
   }
 
-  private static void testFacetIDs(int[] facetIDs, int docID)
-      throws IOException {
+  private static IntsRef newIntsRef(int[] data) {
+    IntsRef res = new IntsRef(data.length);
+    System.arraycopy(data, 0, res.ints, 0, data.length);
+    res.length = data.length;
+    return res;
+  }
+  
+  private static void testFacetIDs(int[] facetIDs, int docID) throws IOException {
     int loopFactor = integers / facetIDs.length;
     System.out
         .println("\nEstimating ~"
@@ -88,68 +85,53 @@ public class EncodingSpeed {
     System.out.println();
   }
 
-  private static void encoderTest(IntEncoder encoder, int[] data,
-      int loopFactor) throws IOException {
+  private static void encoderTest(IntEncoder encoder, int[] values, int loopFactor) throws IOException {
 
-    long startTime, endTime;
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    BytesRef bytes = new BytesRef(values.length); // at least one byte per value
 
     // -- Looping 100 times as a warm up --------------------------
     for (int i = 100; i != 0; --i) {
-      baos.reset();
-      encoder.reInit(baos);
-      for (int value : data) {
-        encoder.encode(value);
-      }
-      encoder.close();
+      IntsRef data = newIntsRef(values);
+      encoder.encode(data, bytes);
     }
     // -----------------------------------------------------------
 
-    startTime = System.currentTimeMillis();
+    long encodeTime = 0;
     for (int factor = loopFactor; factor > 0; --factor) {
-      baos.reset();
-      encoder.reInit(baos);
-      for (int value : data) {
-        encoder.encode(value);
-      }
-      encoder.close();
+      IntsRef data = newIntsRef(values);
+      long start = System.currentTimeMillis();
+      encoder.encode(data, bytes);
+      encodeTime += System.currentTimeMillis() - start;
     }
-    endTime = System.currentTimeMillis();
-
-    long encodeTime = endTime - startTime;
 
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
+    IntsRef decoded = new IntsRef(values.length);
+    int encodedSize = bytes.length;
     IntDecoder decoder = encoder.createMatchingDecoder();
-    decoder.reInit(bais);
     
     // -- Looping 100 times as a warm up --------------------------
     for (int i = 100; i != 0; --i) {
-      bais.mark(baos.size());
-      while (decoder.decode() != IntDecoder.EOS) {
-      }
-      bais.reset();
-      decoder.reInit(bais);
+      decoder.decode(bytes, decoded);
     }
     // -----------------------------------------------------------
 
-    decoder.reInit(bais);
-    startTime = System.currentTimeMillis();
+    long decodeTime = 0;
     for (int i = loopFactor; i > 0; --i) {
-      bais.mark(baos.size());
-      while (decoder.decode() != IntDecoder.EOS) {
-      }
-      bais.reset();
-      decoder.reInit(bais);
+      long start = System.currentTimeMillis();
+      decoder.decode(bytes, decoded);
+      decodeTime += System.currentTimeMillis() - start;
+    }
+    
+    if (decoded.length != values.length) {
+      throw new RuntimeException("wrong num values. expected=" + values.length + " actual=" + decoded.length + 
+          " decoder=" + decoder);
     }
 
-    endTime = System.currentTimeMillis();
-    long decodeTime = endTime - startTime;
-
-    System.out.println(String.format(Locale.ROOT, resultsFormat, encoder, nf.format(baos
-        .size()
-        * 8.0 / data.length), encodeTime, nf.format(encodeTime
-        * 1000000.0 / (loopFactor * data.length)), decodeTime, nf
-        .format(decodeTime * 1000000.0 / (loopFactor * data.length))));
+    System.out.println(String.format(Locale.ROOT, resultsFormat, encoder, 
+        nf.format(encodedSize * 8.0 / values.length), 
+        encodeTime, 
+        nf.format(encodeTime * 1000000.0 / (loopFactor * values.length)), 
+        decodeTime, 
+        nf.format(decodeTime * 1000000.0 / (loopFactor * values.length))));
   }
 
   static {

Modified: lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java (original)
+++ lucene/dev/branches/lucene4547/lucene/facet/src/test/org/apache/lucene/util/encoding/EncodingTest.java Fri Jan 11 14:39:45 2013
@@ -1,14 +1,13 @@
 package org.apache.lucene.util.encoding;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.util.HashSet;
-import java.util.TreeSet;
-
-import org.junit.Test;
+import java.util.Arrays;
 
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.LuceneTestCase;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -29,386 +28,124 @@ import org.apache.lucene.util.LuceneTest
 
 public class EncodingTest extends LuceneTestCase {
 
-  static int[] data = null;
-
-  private static TreeSet<Long> dataSet = new TreeSet<Long>();
-  static {
-    setData();
+  private static IntsRef uniqueSortedData, data;
+  
+  @BeforeClass
+  public static void beforeClassEncodingTest() throws Exception {
+    int capacity = atLeast(10000);
+    data = new IntsRef(capacity);
+    for (int i = 0; i < 10; i++) {
+      data.ints[i] = i + 1; // small values
+    }
+    for (int i = 10; i < data.ints.length; i++) {
+      data.ints[i] = random().nextInt(Integer.MAX_VALUE - 1) + 1; // some encoders don't allow 0
+    }
+    data.length = data.ints.length;
+    
+    uniqueSortedData = IntsRef.deepCopyOf(data);
+    Arrays.sort(uniqueSortedData.ints);
+    uniqueSortedData.length = 0;
+    int prev = -1;
+    for (int i = 0; i < uniqueSortedData.ints.length; i++) {
+      if (uniqueSortedData.ints[i] != prev) {
+        uniqueSortedData.ints[uniqueSortedData.length++] = uniqueSortedData.ints[i];
+        prev = uniqueSortedData.ints[i];
+      }
+    }
+  }
+  
+  private static void encoderTest(IntEncoder encoder, IntsRef data, IntsRef expected) throws IOException {
+    // ensure toString is implemented
+    String toString = encoder.toString();
+    assertFalse(toString.startsWith(encoder.getClass().getName() + "@"));
+    IntDecoder decoder = encoder.createMatchingDecoder();
+    toString = decoder.toString();
+    assertFalse(toString.startsWith(decoder.getClass().getName() + "@"));
+    
+    BytesRef bytes = new BytesRef(100); // some initial capacity - encoders should grow the byte[]
+    IntsRef values = new IntsRef(100); // some initial capacity - decoders should grow the int[]
+    encoding(encoder, data, bytes);
+    decoding(bytes, values, encoder.createMatchingDecoder());
+    assertTrue(expected.intsEquals(values));
+  }
+
+  private static void encoding(IntEncoder encoder, IntsRef data, BytesRef bytes) throws IOException {
+    final IntsRef values;
+    if (random().nextBoolean()) { // randomly set the offset
+      values = new IntsRef(data.length + 1);
+      System.arraycopy(data.ints, 0, values.ints, 1, data.length);
+      values.offset = 1; // ints start at index 1
+      values.length = data.length;
+    } else {
+      // need to copy the array because it may be modified by encoders (e.g. sorting)
+      values = IntsRef.deepCopyOf(data);
+    }
+    encoder.encode(values, bytes);
+  }
+
+  private static void decoding(BytesRef bytes, IntsRef values, IntDecoder decoder) throws IOException {
+    int offset = 0;
+    if (random().nextBoolean()) { // randomly set the offset and length to other than 0,0
+      bytes.grow(bytes.length + 1); // ensure that we have enough capacity to shift values by 1
+      bytes.offset = 1; // bytes start at index 1 (must do that after grow)
+      System.arraycopy(bytes.bytes, 0, bytes.bytes, 1, bytes.length);
+      offset = 1;
+    }
+    decoder.decode(bytes, values);
+    assertEquals(offset, bytes.offset); // decoders should not mess with offsets
   }
 
   @Test
   public void testVInt8() throws Exception {
-    encoderTest(new VInt8IntEncoder());
+    encoderTest(new VInt8IntEncoder(), data, data);
     
     // cover negative numbers;
+    BytesRef bytes = new BytesRef(5);
     IntEncoder enc = new VInt8IntEncoder();
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    enc.reInit(baos);
-    enc.encode(-1);
+    IntsRef values = new IntsRef(1);
+    values.ints[values.length++] = -1;
+    enc.encode(values, bytes);
     
     IntDecoder dec = enc.createMatchingDecoder();
-    dec.reInit(new ByteArrayInputStream(baos.toByteArray()));
-    assertEquals(-1, dec.decode());
+    values.length = 0;
+    dec.decode(bytes, values);
+    assertEquals(1, values.length);
+    assertEquals(-1, values.ints[0]);
   }
   
   @Test
-  public void testSimpleInt() {
-    encoderTest(new SimpleIntEncoder());
+  public void testSimpleInt() throws Exception {
+    encoderTest(new SimpleIntEncoder(), data, data);
   }
   
   @Test
-  public void testSortingUniqueValues() {
-    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new VInt8IntEncoder())));
+  public void testSortingUniqueValues() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new VInt8IntEncoder())), data, uniqueSortedData);
   }
 
   @Test
-  public void testSortingUniqueDGap() {
-    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))));
+  public void testSortingUniqueDGap() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))), data, uniqueSortedData);
   }
 
   @Test
-  public void testSortingUniqueDGapEightFlags() {
-    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new EightFlagsIntEncoder()))));
+  public void testSortingUniqueDGapEightFlags() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new EightFlagsIntEncoder()))), data, uniqueSortedData);
   }
 
   @Test
-  public void testSortingUniqueDGapFourFlags() {
-    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new FourFlagsIntEncoder()))));
+  public void testSortingUniqueDGapFourFlags() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new FourFlagsIntEncoder()))), data, uniqueSortedData);
   }
 
   @Test
-  public void testSortingUniqueDGapNOnes4() {
-    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(4)))));
+  public void testSortingUniqueDGapNOnes4() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(4)))), data, uniqueSortedData);
   }
   
   @Test
-  public void testSortingUniqueDGapNOnes3() {
-    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(3)))));
-  }
-
-  private static void encoderTest(IntEncoder encoder) {
-
-    // ensure toString is implemented
-    String toString = encoder.toString();
-    assertFalse(toString.startsWith(encoder.getClass().getName() + "@"));
-    IntDecoder decoder = encoder.createMatchingDecoder();
-    toString = decoder.toString();
-    assertFalse(toString.startsWith(decoder.getClass().getName() + "@"));
-    
-    ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-    try {
-      encoding(encoder, baos);
-      decoding(baos, encoder.createMatchingDecoder());
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-    
-    baos.reset();
-
-    try {
-      encoding(encoder, baos);
-      decoding(baos, encoder.createMatchingDecoder());
-    } catch (Exception e) {
-      e.printStackTrace();
-      fail(e.getMessage());
-    }
-  }
-
-  private static void encoding(IntEncoder encoder, ByteArrayOutputStream baos) throws IOException {
-    encoder.reInit(baos);
-    for (int value : data) {
-      encoder.encode(value);
-    }
-    encoder.close();
-
-    baos.reset();
-    encoder.reInit(baos);
-    for (int value : data) {
-      encoder.encode(value);
-    }
-    encoder.close();
-  }
-
-  private static void decoding(ByteArrayOutputStream baos, IntDecoder decoder)
-      throws IOException {
-    ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
-    decoder.reInit(bais);
-    
-    HashSet<Long> set = new HashSet<Long>();
-    long value = 0;
-    while ((value = decoder.decode()) != IntDecoder.EOS) {
-      set.add(value);
-    }
-    assertEquals(dataSet.size(), set.size());
-    assertTrue(set.equals(dataSet));
-    
-    set.clear();
-    bais.reset();
-    decoder.reInit(bais);
-    value = 0;
-    while ((value = decoder.decode()) != IntDecoder.EOS) {
-      set.add(value);
-    }
-    assertEquals(dataSet.size(), set.size());
-    assertTrue(set.equals(dataSet));
-
-  }
-
-  private static void setData() {
-    data = new int[] { 2, 4, 86133, 11, 16505, 86134, 86135, 86136, 1290,
-        86137, 86138, 32473, 19346, 32474, 4922, 32475, 86139, 16914,
-        86140, 86141, 86142, 86143, 32478, 86144, 86145, 32480, 4884,
-        4887, 32481, 86146, 16572, 86147, 16295, 165, 86148, 3183,
-        21920, 21921, 21922, 555, 4006, 32484, 21925, 21926, 13775,
-        86149, 13777, 85833, 85834, 13779, 13773, 13780, 75266, 17674,
-        13784, 13785, 13786, 13787, 13788, 6258, 86150, 13790, 75267,
-        13793, 13794, 13795, 312, 4914, 4915, 6222, 86151, 4845, 4883,
-        4918, 4894, 4919, 86152, 4921, 6223, 6224, 6225, 6226, 67909,
-        6229, 18170, 6230, 5198, 25625, 6231, 6232, 6233, 1808, 6234,
-        6235, 6236, 41376, 6238, 6239, 67911, 6240, 86153, 6243, 6244,
-        83549, 6246, 6247, 6248, 6249, 782, 444, 6251, 6250, 19863,
-        28963, 310, 2234, 144, 2236, 2309, 69437, 2311, 2325, 2241,
-        69438, 69439, 2244, 2245, 2246, 23504, 2314, 69440, 36603,
-        2250, 2268, 2271, 2251, 2254, 2255, 2257, 2240, 36604, 84726,
-        36605, 84727, 2262, 2263, 18431, 38853, 2317, 2149, 2326, 2327,
-        2329, 3980, 2275, 2277, 2258, 84728, 2260, 84729, 84730, 13766,
-        36607, 2282, 2283, 84731, 2284, 2286, 2287, 2337, 7424, 2288,
-        2338, 3522, 2290, 84733, 32902, 371, 37708, 2096, 3065, 3066,
-        375, 377, 374, 378, 2100, 86154, 381, 382, 58795, 379, 383,
-        384, 385, 4449, 387, 388, 389, 390, 9052, 391, 18358, 2107,
-        394, 2111, 2108, 393, 2109, 395, 86155, 86156, 397, 2113, 398,
-        399, 400, 273, 274, 275, 40980, 276, 277, 31716, 279, 280,
-        31717, 281, 282, 1628, 1623, 1624, 1625, 2052, 1626, 725, 727,
-        728, 729, 730, 731, 1633, 733, 734, 735, 86157, 737, 738, 739,
-        1634, 3563, 3564, 3565, 1667, 12461, 76276, 3567, 5413, 77622,
-        5415, 5416, 5417, 5418, 107, 86158, 7784, 15363, 153, 3723,
-        2713, 7786, 3835, 7787, 86159, 7789, 7791, 7792, 7794, 86160,
-        7796, 86161, 6708, 7798, 7799, 7800, 7801, 7802, 7803, 1665,
-        43150, 15365, 1581, 5656, 43152, 80258, 7450, 39922, 86162,
-        51587, 9059, 4606, 396, 86163, 86164, 7250, 401, 403, 2860,
-        33281, 2964, 408, 9119, 409, 86165, 7669, 2861, 410, 413,
-        86166, 414, 415, 33282, 405, 33283, 7498, 2865, 7230, 33284,
-        2866, 86167, 2867, 47518, 2868, 86168, 2869, 2870, 4712, 7096,
-        28484, 6913, 6914, 6915, 6916, 37169, 37170, 7103, 28269, 6919,
-        86169, 45431, 6922, 7104, 6923, 7108, 6924, 6925, 6926, 6927,
-        6928, 86170, 86171, 86172, 6930, 6931, 6932, 6934, 6935, 6936,
-        451, 6937, 6938, 4756, 3554, 5309, 8145, 3586, 16417, 9767,
-        14126, 25854, 6580, 10174, 86173, 5519, 21309, 8561, 20938,
-        10386, 86174, 781, 2030, 16419, 30323, 16420, 16421, 16424,
-        86175, 86176, 86177, 28871, 86178, 28872, 63980, 6329, 49561,
-        4271, 38778, 86179, 86180, 20126, 16245, 193, 195, 196, 197,
-        56973, 199, 200, 201, 202, 203, 204, 56974, 56975, 205, 206,
-        4662, 207, 208, 209, 210, 211, 212, 47901, 641, 642, 643, 1380,
-        1079, 47902, 1381, 1081, 1082, 1083, 47903, 1382, 47904, 1087,
-        47905, 965, 966, 1298, 968, 1387, 1300, 50288, 971, 972, 973,
-        974, 23974, 22183, 1390, 23313, 1389, 1391, 902, 23029, 296,
-        1304, 1395, 1303, 1309, 1308, 50289, 1312, 50290, 50291, 1315,
-        1317, 9270, 19796, 3605, 1320, 1321, 44946, 1322, 1323, 50292,
-        967, 1587, 1326, 1331, 17482, 633, 29115, 53858, 29118, 29119,
-        62624, 44494, 6965, 6966, 6959, 6967, 71562, 6969, 23459,
-        23460, 17464, 4225, 23461, 23462, 23463, 5893, 23464, 17467,
-        17468, 23465, 12562, 1405, 1406, 1407, 960, 961, 962, 687, 963,
-        86181, 86182, 5997, 10812, 11976, 11977, 1850, 577, 13393,
-        10810, 13394, 65040, 86183, 3935, 3936, 3937, 710, 86184, 5785,
-        5786, 29949, 5787, 5788, 283, 284, 2687, 285, 286, 287, 2689,
-        288, 289, 8880, 290, 2690, 13899, 991, 292, 295, 42007, 35616,
-        63103, 298, 299, 3520, 297, 9024, 303, 301, 302, 300, 31345,
-        3719, 304, 305, 306, 307, 308, 368, 364, 85002, 9026, 63105,
-        367, 39596, 25835, 19746, 293, 294, 26505, 85003, 18377, 56785,
-        10122, 10123, 10124, 86185, 39863, 86186, 10125, 39865, 4066,
-        4067, 24257, 4068, 4070, 86187, 4073, 4074, 86188, 4076, 7538,
-        4077, 86189, 4078, 4079, 7540, 7541, 4084, 4085, 7542, 86190,
-        4086, 86191, 4087, 4088, 86192, 7545, 44874, 7821, 44875,
-        86193, 4286, 86194, 51470, 17609, 1408, 47486, 1411, 1412,
-        47487, 1413, 1414, 1417, 1415, 47488, 1416, 1418, 1420, 470,
-        1422, 1423, 1424, 5001, 5002, 47489, 1427, 1429, 1430, 31811,
-        1432, 1433, 47490, 1435, 3753, 1437, 1439, 1440, 47491, 1443,
-        47492, 1446, 5004, 5005, 1450, 47493, 353, 1452, 42145, 3103,
-        3402, 3104, 3105, 4780, 3106, 3107, 3108, 12157, 3111, 42146,
-        42147, 3114, 4782, 42148, 3116, 3117, 42149, 42150, 3407, 3121,
-        3122, 18154, 3126, 3127, 3128, 3410, 3130, 3411, 3412, 3415,
-        24241, 3417, 3418, 3449, 42151, 3421, 3422, 7587, 42152, 3424,
-        3427, 3428, 3448, 3430, 3432, 42153, 42154, 41648, 1991, 407,
-        57234, 411, 2862, 57235, 2863, 18368, 57236, 2874, 7350, 4115,
-        2876, 2877, 17975, 86195, 4116, 2881, 2882, 2883, 2886, 463,
-        870, 872, 873, 874, 875, 8783, 8784, 877, 1480, 1481, 459,
-        2778, 881, 8785, 2779, 8786, 8787, 8788, 886, 887, 8789, 889,
-        8790, 86196, 6920, 86197, 5080, 5081, 7395, 7396, 9395, 9396,
-        1528, 42737, 805, 86198, 1209, 13595, 4126, 9680, 34368, 9682,
-        86199, 86200, 174, 175, 176, 177, 178, 179, 180, 182, 183,
-        1477, 31138, 186, 172, 187, 188, 189, 190, 191, 458, 871,
-        31294, 31295, 27604, 31296, 31297, 882, 883, 884, 31298, 890,
-        1089, 1488, 1489, 1092, 1093, 1094, 1095, 1096, 1097, 1490,
-        1098, 1495, 1502, 1099, 1100, 1101, 1493, 2997, 12223, 1103,
-        2654, 1498, 1499, 1500, 80615, 80616, 80617, 33359, 86201,
-        9294, 1501, 86202, 1506, 1507, 23454, 38802, 38803, 1014,
-        86203, 5583, 5584, 651, 74717, 5586, 5587, 5588, 5589, 74720,
-        5590, 38808, 33527, 78330, 10930, 5119, 10931, 1000, 10928,
-        10932, 10933, 10934, 10935, 5863, 10936, 86204, 10938, 10939,
-        86205, 192, 194, 38754, 38755, 198, 38756, 38757, 38758, 2842,
-        640, 22780, 22781, 1080, 86206, 86207, 1084, 1086, 1088, 63916,
-        9412, 970, 9413, 9414, 9415, 9416, 9417, 1310, 7168, 7169,
-        1318, 9418, 1324, 39159, 1804, 1557, 24850, 41499, 1560, 41500,
-        1562, 1563, 1565, 1927, 1928, 1566, 1569, 1570, 1571, 1572,
-        1573, 1574, 1575, 1576, 2674, 2677, 2678, 2679, 2946, 2682,
-        2676, 2683, 2947, 1156, 1157, 1158, 1467, 1160, 1468, 1469,
-        1161, 1162, 1163, 4369, 1165, 1166, 1167, 12923, 2917, 1169,
-        1170, 1171, 1172, 1173, 1174, 1175, 1176, 1177, 18153, 8359,
-        1178, 1164, 1191, 1180, 12924, 86208, 86209, 54817, 66962,
-        2476, 86210, 86211, 41820, 41821, 41822, 41824, 1130, 1131,
-        1132, 32692, 1134, 34848, 1136, 1133, 1137, 1138, 1139, 1140,
-        1141, 1143, 1144, 1145, 34849, 2639, 34850, 1146, 1147, 1148,
-        34851, 1150, 1151, 1152, 1153, 1154, 1155, 1678, 1679, 1680,
-        1681, 40870, 2059, 1685, 1686, 32686, 14970, 1688, 1689, 86212,
-        1692, 1682, 1693, 1695, 1696, 1698, 12955, 8909, 41690, 1700,
-        41691, 86213, 30949, 41692, 1703, 1704, 1705, 41693, 14976,
-        1708, 2071, 1709, 1710, 1711, 1712, 1727, 86214, 86215, 86216,
-        1715, 86217, 1714, 1717, 1690, 41697, 86218, 1720, 86219, 2073,
-        41699, 1724, 2075, 1726, 1729, 1730, 1732, 2078, 2223, 1735,
-        1713, 41700, 1737, 14977, 1739, 1740, 1741, 2080, 1743, 1744,
-        1745, 1746, 1747, 1748, 1749, 1750, 1751, 41701, 1752, 1753,
-        1909, 86220, 2085, 1754, 19548, 86221, 19551, 5733, 3856, 5190,
-        4581, 25145, 86222, 86223, 4846, 86224, 4861, 86225, 86226,
-        86227, 25150, 86228, 86229, 13820, 2027, 4898, 4899, 4901,
-        2135, 4902, 4868, 4904, 86230, 4905, 25155, 4907, 86231, 4909,
-        4910, 4911, 4912, 86232, 6220, 81357, 86233, 2589, 73877,
-        29706, 6227, 6228, 86234, 6237, 86235, 6241, 6242, 1812, 13808,
-        13809, 70908, 2293, 2294, 86236, 2295, 2296, 2297, 22947,
-        16511, 2299, 2300, 2301, 13097, 73079, 86237, 13099, 50121,
-        86238, 86239, 13101, 86240, 2424, 4725, 4726, 4727, 4728, 4729,
-        4730, 86241, 26881, 10944, 4734, 4735, 4736, 26239, 26240,
-        71408, 86242, 57401, 71410, 26244, 5344, 26245, 86243, 4102,
-        71414, 11091, 6736, 86244, 6737, 6738, 38152, 6740, 6741, 6742,
-        6298, 6743, 6745, 6746, 20867, 6749, 20616, 86245, 9801, 65297,
-        20617, 65298, 20619, 5629, 65299, 20621, 20622, 8385, 20623,
-        20624, 5191, 20625, 20626, 442, 443, 445, 27837, 77681, 86246,
-        27839, 86247, 86248, 41435, 66511, 2478, 2479, 2480, 2481,
-        2482, 2483, 2484, 2485, 2486, 2487, 2488, 2489, 2490, 2494,
-        2493, 33025, 12084, 2542, 2497, 2499, 2501, 2503, 2504, 2505,
-        33026, 2506, 2507, 2508, 2509, 2511, 1787, 12080, 2513, 2514,
-        3988, 3176, 3989, 2518, 2521, 9285, 2522, 2524, 2525, 3990,
-        2527, 2528, 27499, 2529, 2530, 3991, 2532, 2534, 2535, 18038,
-        2536, 2538, 2495, 46077, 61493, 61494, 1006, 713, 4971, 4972,
-        4973, 4975, 4976, 650, 170, 7549, 7550, 7551, 7552, 7553,
-        86249, 7936, 956, 11169, 11170, 1249, 1244, 1245, 1247, 2544,
-        1250, 2545, 1252, 2547, 1253, 1254, 2549, 39636, 1259, 1257,
-        1258, 39637, 1260, 1261, 2551, 1262, 1263, 848, 86250, 86251,
-        854, 74596, 856, 1957, 86252, 855, 1959, 1961, 857, 86253, 851,
-        859, 860, 862, 1964, 864, 865, 866, 867, 1965, 1966, 1967,
-        1968, 1969, 86254, 1971, 1972, 1973, 1974, 1975, 1976, 1977,
-        841, 1954, 842, 2978, 846, 847, 849, 850, 852, 1956, 17452,
-        71941, 86255, 86256, 73665, 1471, 13690, 185, 503, 504, 2342,
-        505, 506, 4378, 508, 4379, 17313, 510, 511, 512, 520, 513,
-        4384, 17314, 514, 515, 46158, 17317, 518, 34269, 519, 4386,
-        523, 524, 525, 46159, 528, 529, 17319, 531, 532, 533, 534, 535,
-        7482, 537, 538, 5267, 536, 539, 541, 540, 19858, 17320, 17321,
-        906, 907, 908, 17322, 910, 17323, 912, 15850, 913, 4398, 17324,
-        86257, 278, 2948, 2949, 2950, 3007, 2951, 2952, 2953, 2954,
-        2955, 3013, 35352, 3014, 3015, 2962, 3016, 33505, 39118, 3017,
-        3018, 20492, 4000, 3021, 3022, 35353, 39293, 3024, 18443, 3029,
-        9467, 20529, 39119, 8380, 2965, 3030, 3043, 22714, 39120, 2956,
-        3035, 39121, 3037, 3038, 2688, 86258, 36675, 30894, 24505,
-        8888, 13541, 49728, 27660, 9082, 27661, 365, 366, 2232, 76098,
-        7233, 1494, 17391, 606, 607, 611, 610, 612, 614, 615, 613, 616,
-        9117, 617, 618, 21155, 1789, 619, 620, 7636, 12019, 621, 622,
-        1793, 623, 625, 624, 631, 626, 627, 21578, 21103, 628, 21579,
-        629, 9122, 9123, 12189, 9289, 3168, 3169, 630, 632, 634, 21580,
-        9121, 635, 636, 637, 21581, 12781, 1801, 638, 639, 1559, 24343,
-        9419, 9420, 795, 796, 1611, 86259, 1612, 21551, 21552, 3741,
-        1617, 3742, 1615, 1619, 1620, 6301, 3744, 1622, 67685, 8521,
-        55937, 9025, 27663, 8881, 13581, 86260, 11592, 44720, 86261,
-        63231, 50873, 42925, 52332, 86262, 72706, 17705, 17707, 17708,
-        3401, 40217, 1248, 40218, 86263, 7098, 86264, 86265, 1264,
-        86266, 1266, 1267, 1268, 1269, 86267, 1271, 1272, 1273, 1274,
-        2556, 1275, 1276, 1277, 1278, 1279, 1280, 1282, 1283, 22680,
-        11889, 86268, 45662, 7038, 86269, 19315, 45663, 45664, 86270,
-        5855, 34002, 49245, 10447, 5663, 86271, 15429, 53877, 49249,
-        86272, 86273, 86274, 60128, 60453, 60129, 5552, 31923, 43407,
-        4287, 17980, 64977, 86275, 86276, 8234, 86277, 3649, 8240,
-        1330, 11999, 1332, 27618, 1334, 1335, 340, 3651, 25640, 18165,
-        1343, 4618, 1474, 3653, 75921, 1349, 53519, 1779, 45454, 22778,
-        40153, 67677, 63826, 45455, 15128, 67678, 67679, 1792, 67680,
-        3171, 47816, 45457, 9288, 59891, 67681, 25703, 35731, 35732,
-        369, 35713, 35714, 35715, 34652, 35716, 31681, 35717, 12779,
-        35718, 35719, 11992, 806, 807, 808, 43499, 43500, 810, 776,
-        812, 813, 814, 241, 43501, 43502, 816, 755, 43503, 818, 819,
-        820, 43504, 821, 822, 823, 824, 825, 826, 43505, 43506, 43507,
-        828, 829, 20083, 43508, 43509, 832, 833, 834, 835, 86278,
-        19984, 19985, 86279, 24125, 19986, 86280, 19988, 86281, 5414,
-        86282, 85808, 5479, 5420, 5421, 5422, 5423, 63800, 86283,
-        86284, 30965, 86285, 416, 1510, 5740, 5741, 81991, 86286,
-        28938, 50149, 1003, 55512, 14306, 6960, 688, 86287, 14307,
-        5399, 5400, 17783, 24118, 720, 86288, 44913, 24557, 667, 24876,
-        6529, 24877, 24878, 24879, 24880, 31847, 20671, 4011, 171, 580,
-        86289, 3863, 914, 2202, 916, 917, 918, 919, 921, 922, 923,
-        7585, 925, 7586, 926, 927, 928, 7588, 929, 930, 931, 932, 933,
-        934, 1875, 1876, 7589, 7590, 1878, 1879, 7591, 7592, 1882,
-        1883, 1884, 2212, 7593, 1887, 1888, 1889, 1890, 1891, 1892,
-        1893, 1894, 1895, 1896, 1897, 1898, 2217, 1900, 7594, 1902,
-        2219, 7595, 1905, 1906, 1907, 3323, 7596, 1911, 1912, 7597,
-        1914, 1915, 1916, 2226, 1919, 7598, 2227, 1920, 1921, 7599,
-        7600, 4708, 1923, 355, 356, 1549, 358, 32077, 360, 32078,
-        21117, 362, 19043, 71677, 5716, 86290, 49790, 86291, 86292,
-        86293, 49792, 86294, 86295, 49794, 86296, 86297, 86298, 86299,
-        11882, 86300, 49798, 86301, 49800, 49801, 49802, 49803, 453,
-        49804, 8591, 6794, 49806, 18989, 49807, 49808, 16308, 49809,
-        86302, 86303, 10105, 86304, 5285, 10106, 10107, 6557, 86305,
-        23571, 10109, 38883, 10110, 5401, 86306, 67557, 16430, 67558,
-        40171, 16433, 25878, 86307, 21762, 23, 86308, 86309, 21766,
-        86310, 86311, 5149, 3926, 21768, 21769, 47826, 942, 46985,
-        6588, 58867, 6589, 6590, 86312, 6592, 6006, 53855, 9565, 359,
-        86313, 2845, 876, 879, 27556, 27557, 885, 27558, 888, 2847,
-        27559, 2115, 2116, 2117, 53962, 57839, 315, 316, 317, 318, 319,
-        86314, 321, 322, 2122, 323, 2123, 324, 325, 328, 326, 327,
-        40542, 329, 330, 18079, 18080, 331, 1790, 7382, 332, 7380,
-        7236, 23413, 23414, 18924, 18925, 333, 335, 336, 39750, 337,
-        86315, 339, 341, 342, 343, 16264, 16265, 6615, 86316, 86317,
-        86318, 86319, 16269, 10538, 33226, 86320, 16272, 5824, 16273,
-        16274, 16276, 16277, 16278, 16279, 16280, 14517, 1547, 6463,
-        3394, 49677, 659, 10380, 30013, 10382, 10378, 10379, 10383,
-        10384, 10385, 86321, 4139, 13370, 13371, 86322, 86323, 11878,
-        64509, 15141, 15142, 15143, 32737, 14183, 15144, 39101, 42768,
-        5645, 32738, 801, 803, 804, 86324, 14707, 86325, 6601, 12402,
-        712, 12403, 2936, 1447, 15477, 1410, 44872, 1550, 8614, 15478,
-        15479, 15480, 15481, 4811, 3752, 1442, 15482, 8818, 1445, 5006,
-        16304, 32277, 16305, 16306, 86326, 16307, 53691, 69305, 809,
-        86327, 815, 26724, 69307, 43484, 63904, 86328, 13498, 827,
-        86329, 831, 2857, 836, 86330, 86331, 837, 838, 839, 840, 228,
-        229, 43722, 230, 231, 43723, 234, 235, 236, 237, 238, 239,
-        2745, 2746, 240, 242, 243, 244, 43724, 19788, 246, 247, 21134,
-        248, 250, 251, 252, 253, 254, 255, 256, 257, 258, 43725, 43726,
-        41, 43727, 262, 43728, 2751, 264, 265, 266, 267, 268, 269, 270,
-        271, 272, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 1031, 1032,
-        1033, 1034, 43729, 1035, 43730, 1037, 21821, 2926, 14388,
-        10432, 14389, 14390, 14391, 14392, 86332, 14394, 14395, 2035,
-        2169, 86333, 14397, 14398, 14399, 14400, 52, 14401, 14402,
-        7077, 21822, 14405, 14406, 14396, 86334, 17356, 17357, 84679,
-        84680, 76383, 17360, 17361, 86335, 38801, 2060, 30850, 12963,
-        1684, 1687, 2061, 14978, 1694, 43387, 1697, 1699, 2067, 1701,
-        1702, 1706, 43388, 43389, 76325, 1716, 1718, 26832, 1719, 1723,
-        2081, 2063, 1728, 39059, 76326, 1731, 86336, 1736, 76327, 1738,
-        19657, 6579, 6581, 6582, 6583, 6584, 6585, 29979, 1818, 28239,
-        68, 69, 3391, 86337, 10266, 63528, 86338, 10269, 10270, 10271,
-        10272, 86339, 86340, 63530, 63531, 63532, 63533, 10273, 63534,
-        86341, 10681, 10682, 86342, 9673, 86343, 10683, 460, 461, 462,
-        467, 4464, 4466, 3729, 471, 472, 468, 81634, 474, 81635, 475,
-        476, 477, 479, 480, 81636, 81637, 482, 17442, 81638, 81639,
-        484, 485, 486, 4473, 488, 489, 490, 493, 466, 494, 495, 496,
-        497, 499, 500, 501, 502, 34376, 86344, 63836, 56281, 1707,
-        20416, 61452, 56282, 1755, 56283, 56284, 18508, 53650, 63444,
-        86345, 3579, 63445, 3677, 1979, 1980, 1981, 3132, 3147, 34090,
-        1987, 12770, 1329, 80818, 80819, 1988, 23522, 1986, 15880,
-        1985, 32975, 1992, 1993, 7165, 3141, 3143, 86346, 1982, 1984,
-        3145, 86347, 78064, 55453, 2656, 2657, 35634, 35635, 2167,
-        43479,
-        // ensure there is a representative number for any # of int bytes
-        1, 1 << 8 + 1, 1 << 16 + 1, 1 << 24 + 1 };
-//    data = new int[]{1, 2, 3, 4};
-    for (int value : data) {
-      dataSet.add(new Long(value));
-    }
+  public void testSortingUniqueDGapNOnes3() throws Exception {
+    encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(3)))), data, uniqueSortedData);
   }
 
 }

Modified: lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java (original)
+++ lucene/dev/branches/lucene4547/lucene/misc/src/java/org/apache/lucene/store/NativeUnixDirectory.java Fri Jan 11 14:39:45 2013
@@ -73,7 +73,7 @@ public class NativeUnixDirectory extends
   private final static long ALIGN = 512;
   private final static long ALIGN_NOT_MASK = ~(ALIGN-1);
   
-  /** Default buffer size before writing to disk (256 MB);
+  /** Default buffer size before writing to disk (256 KB);
    *  larger means less IO load but more RAM and direct
    *  buffer storage space consumed during merging. */
 

Modified: lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java (original)
+++ lucene/dev/branches/lucene4547/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionBuilder.java Fri Jan 11 14:39:45 2013
@@ -237,7 +237,7 @@ public class FSTCompletionBuilder {
     final Object empty = outputs.getNoOutput();
     final Builder<Object> builder = new Builder<Object>(
         FST.INPUT_TYPE.BYTE1, 0, 0, true, true, 
-        shareMaxTailLength, outputs, null, false);
+        shareMaxTailLength, outputs, null, false, true);
     
     BytesRef scratch = new BytesRef();
     BytesRef entry;

Modified: lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java?rev=1432065&r1=1432064&r2=1432065&view=diff
==============================================================================
--- lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java (original)
+++ lucene/dev/branches/lucene4547/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingCodec.java Fri Jan 11 14:39:45 2013
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.asserti
 
 import org.apache.lucene.codecs.FilterCodec;
 import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.lucene41.Lucene41Codec;
 
@@ -29,6 +30,7 @@ public final class AssertingCodec extend
 
   private final PostingsFormat postings = new AssertingPostingsFormat();
   private final TermVectorsFormat vectors = new AssertingTermVectorsFormat();
+  private final StoredFieldsFormat storedFields = new AssertingStoredFieldsFormat();
 
   public AssertingCodec() {
     super("Asserting", new Lucene41Codec());
@@ -44,4 +46,8 @@ public final class AssertingCodec extend
     return vectors;
   }
 
+  @Override
+  public StoredFieldsFormat storedFieldsFormat() {
+    return storedFields;
+  }
 }