You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2012/08/21 14:14:42 UTC

svn commit: r1375495 [3/3] - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/analysis/ lucene/benchmark/ lucene/core/ lucene/core/src/java/org/apache/lucene/codecs/ lucene/core/src/java/org/apache/lucene/codecs/block/ lucene/core/src/ja...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Tue Aug 21 12:14:39 2012
@@ -17,12 +17,12 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import org.apache.lucene.store.DataInput;
-import org.apache.lucene.util.RamUsageEstimator;
-
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.util.RamUsageEstimator;
+
 /**
  * Space optimized random access capable array of values with a fixed number of
  * bits/value. Values are packed contiguously.
@@ -146,12 +146,12 @@ class Packed64 extends PackedInts.Mutabl
     assert off + len <= arr.length;
 
     final int originalIndex = index;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
+    final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
 
     // go to the next block where the value does not span across two blocks
-    final int offsetInBlocks = index % op.values();
+    final int offsetInBlocks = index % decoder.valueCount();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < op.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < decoder.valueCount() && len > 0; ++i) {
         arr[off++] = get(index++);
         --len;
       }
@@ -161,12 +161,12 @@ class Packed64 extends PackedInts.Mutabl
     }
 
     // bulk get
-    assert index % op.values() == 0;
+    assert index % decoder.valueCount() == 0;
     int blockIndex = (int) ((long) index * bitsPerValue) >>> BLOCK_BITS;
     assert (((long)index * bitsPerValue) & MOD_MASK) == 0;
-    final int iterations = len / op.values();
-    op.get(blocks, blockIndex, arr, off, iterations);
-    final int gotValues = iterations * op.values();
+    final int iterations = len / decoder.valueCount();
+    decoder.decode(blocks, blockIndex, arr, off, iterations);
+    final int gotValues = iterations * decoder.valueCount();
     index += gotValues;
     len -= gotValues;
     assert len >= 0;
@@ -210,12 +210,12 @@ class Packed64 extends PackedInts.Mutabl
     assert off + len <= arr.length;
 
     final int originalIndex = index;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
+    final PackedInts.Encoder encoder = BulkOperation.of(PackedInts.Format.PACKED, bitsPerValue);
 
     // go to the next block where the value does not span across two blocks
-    final int offsetInBlocks = index % op.values();
+    final int offsetInBlocks = index % encoder.valueCount();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < op.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < encoder.valueCount() && len > 0; ++i) {
         set(index++, arr[off++]);
         --len;
       }
@@ -224,13 +224,13 @@ class Packed64 extends PackedInts.Mutabl
       }
     }
 
-    // bulk get
-    assert index % op.values() == 0;
+    // bulk set
+    assert index % encoder.valueCount() == 0;
     int blockIndex = (int) ((long) index * bitsPerValue) >>> BLOCK_BITS;
     assert (((long)index * bitsPerValue) & MOD_MASK) == 0;
-    final int iterations = len / op.values();
-    op.set(blocks, blockIndex, arr, off, iterations);
-    final int setValues = iterations * op.values();
+    final int iterations = len / encoder.valueCount();
+    encoder.encode(arr, off, blocks, blockIndex, iterations);
+    final int setValues = iterations * encoder.valueCount();
     index += setValues;
     len -= setValues;
     assert len >= 0;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Tue Aug 21 12:14:39 2012
@@ -86,12 +86,12 @@ abstract class Packed64SingleBlock exten
 
     // bulk get
     assert index % valuesPerBlock == 0;
-    final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blocks() == 1;
-    assert op.values() == valuesPerBlock;
+    final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
+    assert decoder.blockCount() == 1;
+    assert decoder.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.get(blocks, blockIndex, arr, off, nblocks);
+    decoder.decode(blocks, blockIndex, arr, off, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 
@@ -131,11 +131,11 @@ abstract class Packed64SingleBlock exten
     // bulk set
     assert index % valuesPerBlock == 0;
     final BulkOperation op = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert op.blocks() == 1;
-    assert op.values() == valuesPerBlock;
+    assert op.blockCount() == 1;
+    assert op.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.set(blocks, blockIndex, arr, off, nblocks);
+    op.encode(arr, off, blocks, blockIndex, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Tue Aug 21 12:14:39 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.Closeable;
+import java.io.IOException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.DataInput;
@@ -25,8 +26,6 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.LongsRef;
 
-import java.io.IOException;
-
 /**
  * Simplistic compression for array of unsigned long values.
  * Each value is >= 0 and <= a specified maximum value.  The
@@ -66,6 +65,14 @@ public class PackedInts {
   public final static int VERSION_START = 0;
   public final static int VERSION_CURRENT = VERSION_START;
 
+  private static void checkVersion(int version) {
+    if (version < VERSION_START) {
+      throw new IllegalArgumentException("Version is too old, should be at least " + VERSION_START + " (got " + version + ")");
+    } else if (version > VERSION_CURRENT) {
+      throw new IllegalArgumentException("Version is too new, should be at most " + VERSION_CURRENT + " (got " + version + ")");
+    }
+  }
+
   /**
    * A format to write packed ints.
    *
@@ -242,6 +249,146 @@ public class PackedInts {
   }
 
   /**
+   * A decoder for packed integers.
+   */
+  public static interface Decoder {
+
+    /**
+     * The minimum number of long blocks to decode in a single call.
+     */
+    int blockCount();
+
+    /**
+     * The number of values that can be stored in <code>blockCount()</code> long
+     * blocks.
+     */
+    int valueCount();
+
+    /**
+     * Read <code>iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(long[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations);
+
+    /**
+     * Read <code>8 * iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(byte[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations);
+
+    /**
+     * Read <code>iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(long[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations);
+
+    /**
+     * Read <code>8 * iterations * blockCount()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * valueCount()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start reading blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start writing values
+     * @param iterations   controls how much data to decode
+     */
+    void decode(byte[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations);
+
+  }
+
+  /**
+   * An encoder for packed integers.
+   */
+  public static interface Encoder {
+
+    /**
+     * The minimum number of long blocks to encode in a single call.
+     */
+    int blockCount();
+
+    /**
+     * The number of values that can be stored in <code>blockCount()</code> long
+     * blocks.
+     */
+    int valueCount();
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(long[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations);
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>8 * iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations);
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(int[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations);
+
+    /**
+     * Read <code>iterations * valueCount()</code> values from <code>values</code>,
+     * encode them and write <code>8 * iterations * blockCount()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param blocksOffset the offset where to start writing blocks
+     * @param values       the values buffer
+     * @param valuesOffset the offset where to start reading values
+     * @param iterations   controls how much data to encode
+     */
+    void encode(int[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations);
+
+  }
+
+  /**
    * A read-only random access array of positive integers.
    * @lucene.internal
    */
@@ -490,8 +637,7 @@ public class PackedInts {
     protected final int valueCount;
     protected final int bitsPerValue;
 
-    protected Writer(DataOutput out, int valueCount, int bitsPerValue)
-      throws IOException {
+    protected Writer(DataOutput out, int valueCount, int bitsPerValue) {
       assert bitsPerValue <= 64;
       assert valueCount >= 0 || valueCount == -1;
       this.out = out;
@@ -529,6 +675,32 @@ public class PackedInts {
   }
 
   /**
+   * Get a {@link Decoder}.
+   *
+   * @param format         the format used to store packed ints
+   * @param version        the compatibility version
+   * @param bitsPerValue   the number of bits per value
+   * @return a decoder
+   */
+  public static Decoder getDecoder(Format format, int version, int bitsPerValue) {
+    checkVersion(version);
+    return BulkOperation.of(format, bitsPerValue);
+  }
+
+  /**
+   * Get an {@link Encoder}.
+   *
+   * @param format         the format used to store packed ints
+   * @param version        the compatibility version
+   * @param bitsPerValue   the number of bits per value
+   * @return an encoder
+   */
+  public static Encoder getEncoder(Format format, int version, int bitsPerValue) {
+    checkVersion(version);
+    return BulkOperation.of(format, bitsPerValue);
+  }
+
+  /**
    * Expert: Restore a {@link Reader} from a stream without reading metadata at
    * the beginning of the stream. This method is useful to restore data from
    * streams which have been created using
@@ -546,6 +718,7 @@ public class PackedInts {
    */
   public static Reader getReaderNoHeader(DataInput in, Format format, int version,
       int valueCount, int bitsPerValue) throws IOException {
+    checkVersion(version);
     switch (format) {
       case PACKED_SINGLE_BLOCK:
         return Packed64SingleBlock.create(in, valueCount, bitsPerValue);
@@ -612,7 +785,8 @@ public class PackedInts {
    * @lucene.internal
    */
   public static ReaderIterator getReaderIteratorNoHeader(DataInput in, Format format, int version,
-      int valueCount, int bitsPerValue, int mem) throws IOException {
+      int valueCount, int bitsPerValue, int mem) {
+    checkVersion(version);
     return new PackedReaderIterator(format, valueCount, bitsPerValue, in, mem);
   }
 
@@ -652,7 +826,8 @@ public class PackedInts {
    * @lucene.internal
    */
   public static Reader getDirectReaderNoHeader(IndexInput in, Format format,
-      int version, int valueCount, int bitsPerValue) throws IOException {
+      int version, int valueCount, int bitsPerValue) {
+    checkVersion(version);
     switch (format) {
       case PACKED:
         return new DirectPackedReader(bitsPerValue, valueCount, in);
@@ -784,7 +959,7 @@ public class PackedInts {
    * @lucene.internal
    */
   public static Writer getWriterNoHeader(
-      DataOutput out, Format format, int valueCount, int bitsPerValue, int mem) throws IOException {
+      DataOutput out, Format format, int valueCount, int bitsPerValue, int mem) {
     return new PackedWriter(format, out, valueCount, bitsPerValue, mem);
   }
 
@@ -894,4 +1069,4 @@ public class PackedInts {
     }
   }
 
-}
+}
\ No newline at end of file

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java Tue Aug 21 12:14:39 2012
@@ -38,10 +38,10 @@ final class PackedReaderIterator extends
     bulkOperation = BulkOperation.of(format, bitsPerValue);
     iterations = bulkOperation.computeIterations(valueCount, mem);
     assert iterations > 0;
-    nextBlocks = new long[iterations * bulkOperation.blocks()];
-    nextValues = new LongsRef(new long[iterations * bulkOperation.values()], 0, 0);
-    assert iterations * bulkOperation.values() == nextValues.longs.length;
-    assert iterations * bulkOperation.blocks() == nextBlocks.length;
+    nextBlocks = new long[iterations * bulkOperation.blockCount()];
+    nextValues = new LongsRef(new long[iterations * bulkOperation.valueCount()], 0, 0);
+    assert iterations * bulkOperation.valueCount() == nextValues.longs.length;
+    assert iterations * bulkOperation.blockCount() == nextBlocks.length;
     nextValues.offset = nextValues.longs.length;
     position = -1;
   }
@@ -70,7 +70,7 @@ final class PackedReaderIterator extends
         nextBlocks[i] = 0L;
       }
 
-      bulkOperation.get(nextBlocks, 0, nextValues.longs, 0, iterations);
+      bulkOperation.decode(nextBlocks, 0, nextValues.longs, 0, iterations);
       nextValues.offset = 0;
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java Tue Aug 21 12:14:39 2012
@@ -21,6 +21,7 @@ import org.apache.lucene.store.DataOutpu
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.util.Arrays;
 
 // Packs high order byte first, to match
 // IndexOutput.writeInt/Long/Short byte order
@@ -29,21 +30,20 @@ final class PackedWriter extends PackedI
 
   boolean finished;
   final PackedInts.Format format;
-  final BulkOperation bulkOperation;
+  final BulkOperation encoder;
   final long[] nextBlocks;
   final long[] nextValues;
   final int iterations;
   int off;
   int written;
 
-  PackedWriter(PackedInts.Format format, DataOutput out, int valueCount, int bitsPerValue, int mem)
-      throws IOException {
+  PackedWriter(PackedInts.Format format, DataOutput out, int valueCount, int bitsPerValue, int mem) {
     super(out, valueCount, bitsPerValue);
     this.format = format;
-    bulkOperation = BulkOperation.of(format, bitsPerValue);
-    iterations = bulkOperation.computeIterations(valueCount, mem);
-    nextBlocks = new long[iterations * bulkOperation.blocks()];
-    nextValues = new long[iterations * bulkOperation.values()];
+    encoder = BulkOperation.of(format, bitsPerValue);
+    iterations = encoder.computeIterations(valueCount, mem);
+    nextBlocks = new long[iterations * encoder.blockCount()];
+    nextValues = new long[iterations * encoder.valueCount()];
     off = 0;
     written = 0;
     finished = false;
@@ -63,8 +63,7 @@ final class PackedWriter extends PackedI
     }
     nextValues[off++] = v;
     if (off == nextValues.length) {
-      flush(nextValues.length);
-      off = 0;
+      flush();
     }
     ++written;
   }
@@ -77,16 +76,17 @@ final class PackedWriter extends PackedI
         add(0L);
       }
     }
-    flush(off);
+    flush();
     finished = true;
   }
 
-  private void flush(int nvalues) throws IOException {
-    bulkOperation.set(nextBlocks, 0, nextValues, 0, iterations);
-    final int blocks = format.nblocks(bitsPerValue, nvalues);
+  private void flush() throws IOException {
+    encoder.encode(nextValues, 0, nextBlocks, 0, iterations);
+    final int blocks = format.nblocks(bitsPerValue, off);
     for (int i = 0; i < blocks; ++i) {
       out.writeLong(nextBlocks[i]);
     }
+    Arrays.fill(nextValues, 0L);
     off = 0;
   }
 

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py Tue Aug 21 12:14:39 2012
@@ -42,22 +42,52 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import java.util.EnumMap;
+import java.nio.LongBuffer;
+import java.nio.ByteBuffer;
 
 /**
  * Efficient sequential read/write of packed integers.
  */
-abstract class BulkOperation {
+enum BulkOperation implements PackedInts.Decoder, PackedInts.Encoder {
+"""
 
-  static final EnumMap<PackedInts.Format, BulkOperation[]> BULK_OPERATIONS = new EnumMap<PackedInts.Format, BulkOperation[]>(PackedInts.Format.class);
+FOOTER="""
 
-  public static BulkOperation of(PackedInts.Format format, int bitsPerValue) {
-    assert bitsPerValue > 0 && bitsPerValue <= 64;
-    BulkOperation[] ops = BULK_OPERATIONS.get(format);
-    if (ops == null || ops[bitsPerValue] == null) {
-      throw new IllegalArgumentException("format: " + format + ", bitsPerValue: " + bitsPerValue);
+  private static long[] toLongArray(int[] ints, int offset, int length) {
+    long[] arr = new long[length];
+    for (int i = 0; i < length; ++i) {
+      arr[i] = ints[offset + i];
     }
-    return ops[bitsPerValue];
+    return arr;
+  }
+
+  @Override
+  public void decode(long[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void decode(byte[] blocks, int blocksOffset, int[] values, int valuesOffset, int iterations) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void encode(int[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations) {
+    encode(toLongArray(values, valuesOffset, iterations * valueCount()), 0, blocks, blocksOffset, iterations);
+  }
+
+  @Override
+  public void encode(long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
+    final long[] longBLocks = new long[blockCount() * iterations];
+    encode(values, valuesOffset, longBLocks, 0, iterations);
+    ByteBuffer.wrap(blocks, blocksOffset, 8 * iterations * blockCount()).asLongBuffer().put(longBLocks);
+  }
+
+  @Override
+  public void encode(int[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations) {
+    final long[] longBLocks = new long[blockCount() * iterations];
+    encode(values, valuesOffset, longBLocks, 0, iterations);
+    ByteBuffer.wrap(blocks, blocksOffset, 8 * iterations * blockCount()).asLongBuffer().put(longBLocks);
   }
 
   /**
@@ -67,7 +97,7 @@ abstract class BulkOperation {
    *  - 16 bits per value -> b=1, v=4
    *  - 24 bits per value -> b=3, v=8
    *  - 50 bits per value -> b=25, v=32
-   *  - 63 bits per value -> b=63, v = 64
+   *  - 63 bits per value -> b=63, v=64
    *  - ...
    *
    * A bulk read consists in copying <code>iterations*v</code> values that are
@@ -79,87 +109,155 @@ abstract class BulkOperation {
    * <code>ramBudget / (8 * (b + v))</code> (since a long is 8 bytes).
    */
   public final int computeIterations(int valueCount, int ramBudget) {
-    final int iterations = (ramBudget >>> 3) / (blocks() + values());
+    final int iterations = (ramBudget >>> 3) / (blockCount() + valueCount());
     if (iterations == 0) {
       // at least 1
       return 1;
-    } else if ((iterations - 1) * blocks() >= valueCount) {
+    } else if ((iterations - 1) * blockCount() >= valueCount) {
       // don't allocate for more than the size of the reader
-      return (int) Math.ceil((double) valueCount / values());
+      return (int) Math.ceil((double) valueCount / valueCount());
     } else {
       return iterations;
     }
   }
-
-  /**
-   * The minimum number of blocks required to perform a bulk get/set.
-   */
-  public abstract int blocks();
-
-  /**
-   * The number of values that can be stored in <code>blocks()</code> blocks.
-   */
-  public abstract int values();
-
-  /**
-   * Get <code>n * values()</code> values from <code>n * blocks()</code> blocks.
-   */
-  public abstract void get(long[] blocks, int blockIndex, long[] values, int valuesIndex, int iterations);
-
-  /**
-   * Set <code>n * values()</code> values into <code>n * blocks()</code> blocks.
-   */
-  public abstract void set(long[] blocks, int blockIndex, long[] values, int valuesIndex, int iterations);
-
+}
 """
 
-FOOTER = "}"
+def casts(typ):
+  cast_start = "(%s) (" %typ
+  cast_end = ")"
+  if typ == "long":
+    cast_start = ""
+    cast_end = ""
+  return cast_start, cast_end
+
+def masks(bits):
+  if bits == 64:
+    return "", ""
+  return "(", " & %sL)" %(hex((1 << bits) - 1))
+
+def get_type(bits):
+  if bits == 8:
+    return "byte"
+  elif bits == 16:
+    return "short"
+  elif bits == 32:
+    return "int"
+  elif bits == 64:
+    return "long"
+  else:
+    assert False
 
 def packed64singleblock(bpv, f):
   values = 64 / bpv
-  f.write("\n  static final class Packed64SingleBlockBulkOperation%d extends BulkOperation {\n\n" %bpv)
-  f.write("    public int blocks() {\n")
+  f.write("\n  PACKED_SINGLE_BLOCK_%d {\n\n" %bpv)
+  f.write("    public int blockCount() {\n")
   f.write("      return 1;\n")
   f.write("     }\n\n")
-  f.write("    public int values() {\n")
+  f.write("    public int valueCount() {\n")
   f.write("      return %d;\n" %values)
   f.write("    }\n\n")
+  p64sb_decode(bpv, 32)
+  p64sb_decode(bpv, 64)
+  p64sb_encode(bpv, 32)
+  p64sb_encode(bpv, 64)
+  f.write("  }")
 
-  f.write("    public void get(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
+def p64sb_decode(bpv, bits):
+  values = 64 / bpv
+  typ = get_type(bits)
+  cast_start, cast_end = casts(typ)
+  f.write("    public void decode(long[] blocks, int blocksOffset, %s[] values, int valuesOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("      throw new UnsupportedOperationException();\n")
+    f.write("    }\n\n")
+    return 
+  f.write("      assert blocksOffset + iterations * blockCount() <= blocks.length;\n")
+  f.write("      assert valuesOffset + iterations * valueCount() <= values.length;\n")
   f.write("      for (int i = 0; i < iterations; ++i) {\n")
-  f.write("        final long block = blocks[bi++];\n")
+  f.write("        final long block = blocks[blocksOffset++];\n")
   mask = (1 << bpv) - 1
   for i in xrange(values):
     block_offset = i / values
     offset_in_block = i % values
     if i == 0:
-      f.write("        values[vi++] = block & %dL;\n" %mask)
+      f.write("        values[valuesOffset++] = %sblock & %dL%s;\n" %(cast_start, mask, cast_end))
     elif i == values - 1:
-      f.write("        values[vi++] = block >>> %d;\n" %(i * bpv))
+      f.write("        values[valuesOffset++] = %sblock >>> %d%s;\n" %(cast_start, i * bpv, cast_end))
+    else:
+      f.write("        values[valuesOffset++] = %s(block >>> %d) & %dL%s;\n" %(cast_start, i * bpv, mask, cast_end))
+  f.write("      }\n")
+  f.write("    }\n\n")
+
+  f.write("    public void decode(byte[] blocks, int blocksOffset, %s[] values, int valuesOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("      throw new UnsupportedOperationException();\n")
+    f.write("    }\n\n")
+  f.write("      assert blocksOffset + 8 * iterations * blockCount() <= blocks.length;\n")
+  f.write("      assert valuesOffset + iterations * valueCount() <= values.length;\n")
+  f.write("      for (int i = 0; i < iterations; ++i) {\n")
+  if bpv >= 32 and bits > 32:
+    for i in xrange(7, -1, -1):
+      f.write("        final long byte%d = blocks[blocksOffset++] & 0xFF;\n" %i)
+  else:
+    for i in xrange(7, -1, -1):
+      f.write("        final int byte%d = blocks[blocksOffset++] & 0xFF;\n" %i)
+  for i in xrange(values):
+    byte_start = (i * bpv) / 8
+    bit_start = (i * bpv) % 8
+    byte_end = ((i + 1) * bpv - 1) / 8
+    bit_end = ((i + 1) * bpv - 1) % 8
+    f.write("        values[valuesOffset++] =")
+    if byte_start == byte_end:
+      # only one byte
+      if bit_start == 0:
+        if bit_end == 7:
+          f.write(" byte%d" %byte_start)
+        else:
+          f.write(" byte%d & %d" %(byte_start, mask))
+      else:
+        if bit_end == 7:
+          f.write(" byte%d >>> %d" %(byte_start, bit_start))
+        else:
+          f.write(" (byte%d >>> %d) & %d" %(byte_start, bit_start, mask))
     else:
-      f.write("        values[vi++] = (block >>> %d) & %dL;\n" %(i * bpv, mask))
+      if bit_start == 0:
+        f.write(" byte%d" %byte_start)
+      else:
+        f.write(" (byte%d >>> %d)" %(byte_start, bit_start))
+      for b in xrange(byte_start + 1, byte_end):
+        f.write(" | (byte%d << %d)" %(b, 8 * (b - byte_start) - bit_start))
+      if bit_end == 7:
+        f.write(" | (byte%d << %d)" %(byte_end, 8 * (byte_end - byte_start) - bit_start))
+      else:
+        f.write(" | ((byte%d & %d) << %d)" %(byte_end, 2 ** (bit_end + 1) - 1, 8 * (byte_end - byte_start) - bit_start))
+    f.write(";\n")
   f.write("      }\n")
   f.write("    }\n\n")
 
-  f.write("    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
+def p64sb_encode(bpv, bits):
+  values = 64 / bpv
+  typ = get_type(bits)
+  mask_start, mask_end = masks(bits)
+  f.write("    public void encode(%s[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("      throw new UnsupportedOperationException();\n")
+    f.write("    }\n\n")
+    return
+  f.write("      assert blocksOffset + iterations * blockCount() <= blocks.length;\n")
+  f.write("      assert valuesOffset + iterations * valueCount() <= values.length;\n")
   f.write("      for (int i = 0; i < iterations; ++i) {\n")
   for i in xrange(values):
     block_offset = i / values
     offset_in_block = i % values
     if i == 0:
-      f.write("        blocks[bi++] = values[vi++]")
+      f.write("        blocks[blocksOffset++] = %svalues[valuesOffset++]%s" %(mask_start, mask_end))
     else:
-      f.write(" | (values[vi++] << %d)" %(i * bpv))
+      f.write(" | (%svalues[valuesOffset++]%s << %d)" %(mask_start, mask_end, i * bpv))
       if i == values - 1:
         f.write(";\n")
   f.write("      }\n")
-  f.write("    }\n")
-
-  f.write("  }\n")
+  f.write("    }\n\n")
 
 def packed64(bpv, f):
   blocks = bpv
@@ -169,96 +267,180 @@ def packed64(bpv, f):
     values /= 2
   assert values * bpv == 64 * blocks, "%d values, %d blocks, %d bits per value" %(values, blocks, bpv)
   mask = (1 << bpv) - 1
-  f.write("  static final class Packed64BulkOperation%d extends BulkOperation {\n\n" %bpv)
-  f.write("    public int blocks() {\n")
+  f.write("  PACKED_%d {\n\n" %bpv)
+  f.write("    public int blockCount() {\n")
   f.write("      return %d;\n" %blocks)
   f.write("    }\n\n")
-  f.write("    public int values() {\n")
+  f.write("    public int valueCount() {\n")
   f.write("      return %d;\n" %values)
   f.write("    }\n\n")
 
   if bpv == 64:
-    f.write("""    public void get(long[] blocks, int bi, long[] values, int vi, int iterations) {
-      System.arraycopy(blocks, bi, values, vi, iterations);
+    f.write("""    public void decode(long[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations) {
+      System.arraycopy(blocks, blocksOffset, values, valuesOffset, valueCount() * iterations);
     }
 
-    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {
-      System.arraycopy(values, bi, blocks, vi, iterations);
+    public void decode(byte[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations) {
+      LongBuffer.wrap(values, valuesOffset, iterations * valueCount()).put(ByteBuffer.wrap(blocks, blocksOffset, 8 * iterations * blockCount()).asLongBuffer());
     }
+
+    public void encode(long[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations) {
+      System.arraycopy(values, valuesOffset, blocks, blocksOffset, valueCount() * iterations);
+    }
+
   }
 """)
+  else:
+    p64_decode(bpv, 32, values)
+    p64_decode(bpv, 64, values)
+    p64_encode(bpv, 32, values)
+    p64_encode(bpv, 64, values)
+    f.write("  }\n")
+
+def p64_decode(bpv, bits, values):
+  typ = get_type(bits)
+  cast_start, cast_end = casts(typ)
+
+  f.write("    public void decode(long[] blocks, int blocksOffset, %s[] values, int valuesOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("      throw new UnsupportedOperationException();\n")
+    f.write("    }\n\n")
     return
-
-  f.write("    public void get(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
+  f.write("      assert blocksOffset + iterations * blockCount() <= blocks.length;\n")
+  f.write("      assert valuesOffset + iterations * valueCount() <= values.length;\n")
   f.write("      for (int i = 0; i < iterations; ++i) {\n")
+  mask = (1 << bpv) - 1
   for i in xrange(0, values):
     block_offset = i * bpv / 64
     bit_offset = (i * bpv) % 64
     if bit_offset == 0:
       # start of block
-      f.write("        final long block%d = blocks[bi++];\n" %block_offset);
-      f.write("        values[vi++] = block%d >>> %d;\n" %(block_offset, 64 - bpv))
+      f.write("        final long block%d = blocks[blocksOffset++];\n" %block_offset);
+      f.write("        values[valuesOffset++] = %sblock%d >>> %d%s;\n" %(cast_start, block_offset, 64 - bpv, cast_end))
     elif bit_offset + bpv == 64:
       # end of block
-      f.write("        values[vi++] = block%d & %dL;\n" %(block_offset, mask))
+      f.write("        values[valuesOffset++] = %sblock%d & %dL%s;\n" %(cast_start, block_offset, mask, cast_end))
     elif bit_offset + bpv < 64:
       # middle of block
-      f.write("        values[vi++] = (block%d >>> %d) & %dL;\n" %(block_offset, 64 - bit_offset - bpv, mask))
+      f.write("        values[valuesOffset++] = %s(block%d >>> %d) & %dL%s;\n" %(cast_start, block_offset, 64 - bit_offset - bpv, mask, cast_end))
     else:
       # value spans across 2 blocks
       mask1 = (1 << (64 - bit_offset)) -1
       shift1 = bit_offset + bpv - 64
       shift2 = 64 - shift1
-      f.write("        final long block%d = blocks[bi++];\n" %(block_offset + 1));
-      f.write("        values[vi++] = ((block%d & %dL) << %d) | (block%d >>> %d);\n" %(block_offset, mask1, shift1, block_offset + 1, shift2))
+      f.write("        final long block%d = blocks[blocksOffset++];\n" %(block_offset + 1));
+      f.write("        values[valuesOffset++] = %s((block%d & %dL) << %d) | (block%d >>> %d)%s;\n" %(cast_start, block_offset, mask1, shift1, block_offset + 1, shift2, cast_end))
+  f.write("      }\n")
+  f.write("    }\n\n")
+
+  f.write("    public void decode(byte[] blocks, int blocksOffset, %s[] values, int valuesOffset, int iterations) {\n" %typ)
+  if bits < bpv:
+    f.write("      throw new UnsupportedOperationException();\n")
+    f.write("    }\n\n")
+    return
+  f.write("      assert blocksOffset + 8 * iterations * blockCount() <= blocks.length;\n")
+  f.write("      assert valuesOffset + iterations * valueCount() <= values.length;\n")
+  f.write("      for (int i = 0; i < iterations; ++i) {\n")
+  blocks = values * bpv / 8
+  for i in xrange(0, values):
+    byte_start = i * bpv / 8
+    bit_start = (i * bpv) % 8
+    byte_end = ((i + 1) * bpv - 1) / 8
+    bit_end = ((i + 1) * bpv - 1) % 8
+    shift = lambda b: 8 * (byte_end - b - 1) + 1 + bit_end
+    if bit_start == 0:
+      f.write("        final %s byte%d = blocks[blocksOffset++] & 0xFF;\n" %(typ, byte_start))
+    for b in xrange(byte_start + 1, byte_end + 1):
+      f.write("        final %s byte%d = blocks[blocksOffset++] & 0xFF;\n" %(typ, b))
+    f.write("        values[valuesOffset++] =")
+    if byte_start == byte_end:
+      if bit_start == 0:
+        if bit_end == 7:
+          f.write(" byte%d" %byte_start)
+        else:
+          f.write(" byte%d >>> %d" %(byte_start, 7 - bit_end))
+      else:
+        if bit_end == 7:
+          f.write(" byte%d & %d" %(byte_start, 2 ** (8 - bit_start) - 1))
+        else:
+          f.write(" (byte%d >>> %d) & %d" %(byte_start, 7 - bit_end, 2 ** (bit_end - bit_start + 1) - 1))
+    else:
+      if bit_start == 0:
+        f.write(" (byte%d << %d)" %(byte_start, shift(byte_start)))
+      else:
+        f.write(" ((byte%d & %d) << %d)" %(byte_start, 2 ** (8 - bit_start) - 1, shift(byte_start)))
+      for b in xrange(byte_start + 1, byte_end):
+        f.write(" | (byte%d << %d)" %(b, shift(b)))
+      if bit_end == 7:
+        f.write(" | byte%d" %byte_end)
+      else:
+        f.write(" | (byte%d >>> %d)" %(byte_end, 7 - bit_end))
+    f.write(";\n")
   f.write("      }\n")
   f.write("    }\n\n")
 
-  f.write("    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {\n")
-  f.write("      assert bi + iterations * blocks() <= blocks.length;\n")
-  f.write("      assert vi + iterations * values() <= values.length;\n")
+def p64_encode(bpv, bits, values):
+  typ = get_type(bits)
+  mask_start, mask_end = masks(bits)
+  f.write("    public void encode(%s[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations) {\n" %typ)
+  f.write("      assert blocksOffset + iterations * blockCount() <= blocks.length;\n")
+  f.write("      assert valuesOffset + iterations * valueCount() <= values.length;\n")
   f.write("      for (int i = 0; i < iterations; ++i) {\n")
   for i in xrange(0, values):
     block_offset = i * bpv / 64
     bit_offset = (i * bpv) % 64
     if bit_offset == 0:
       # start of block
-      f.write("        blocks[bi++] = (values[vi++] << %d)" %(64 - bpv))
+      f.write("        blocks[blocksOffset++] = (%svalues[valuesOffset++]%s << %d)" %(mask_start, mask_end, 64 - bpv))
     elif bit_offset + bpv == 64:
       # end of block
-      f.write(" | values[vi++];\n")
+      f.write(" | %svalues[valuesOffset++]%s;\n" %(mask_start, mask_end))
     elif bit_offset + bpv < 64:
       # inside a block
-      f.write(" | (values[vi++] << %d)" %(64 - bit_offset - bpv))
+      f.write(" | (%svalues[valuesOffset++]%s << %d)" %(mask_start, mask_end, 64 - bit_offset - bpv))
     else:
       # value spans across 2 blocks
       right_bits = bit_offset + bpv - 64
-      f.write(" | (values[vi] >>> %d);\n" %right_bits)
-      f.write("        blocks[bi++] = (values[vi++] << %d)" %(64 - right_bits))
+      f.write(" | (%svalues[valuesOffset]%s >>> %d);\n" %(mask_start, mask_end, right_bits))
+      f.write("        blocks[blocksOffset++] = (%svalues[valuesOffset++]%s << %d)" %(mask_start, mask_end, 64 - right_bits))
   f.write("      }\n")
-  f.write("    }\n")
-
-  f.write("  }\n\n")
-
+  f.write("    }\n\n")
 
 
 if __name__ == '__main__':
   p64_bpv = [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 12, 16, 21, 32]
   f = open(OUTPUT_FILE, 'w')
   f.write(HEADER)
-  f.write("  static {\n")
-  f.write("    BULK_OPERATIONS.put(PackedInts.Format.PACKED, new BulkOperation[65]);")
-  for bpv in xrange(1, 65):
-    f.write("    BULK_OPERATIONS.get(PackedInts.Format.PACKED)[%d] = new Packed64BulkOperation%d();\n" %(bpv, bpv))
-  f.write("    BULK_OPERATIONS.put(PackedInts.Format.PACKED_SINGLE_BLOCK, new BulkOperation[65]);\n")
-  for bpv in PACKED_64_SINGLE_BLOCK_BPV:
-    f.write("    BULK_OPERATIONS.get(PackedInts.Format.PACKED_SINGLE_BLOCK)[%d] = new Packed64SingleBlockBulkOperation%d();\n" %(bpv, bpv))
-  f.write("  }\n")
   for bpv in xrange(1, 65):
     packed64(bpv, f)
+    f.write("  ,\n")
   for bpv in PACKED_64_SINGLE_BLOCK_BPV:
+    if bpv != PACKED_64_SINGLE_BLOCK_BPV[0]:
+      f.write("  ,\n")
     packed64singleblock(bpv,f)
+  f.write("  ;\n\n")
+  f.write("  public static BulkOperation of(PackedInts.Format format, int bitsPerValue) {\n")
+  f.write("    switch (format) {\n")
+
+  f.write("    case PACKED:\n")
+  f.write("      switch (bitsPerValue) {\n")
+  for i in xrange(1, 65):
+    f.write("      case %d:\n" %i)
+    f.write("        return PACKED_%d;\n" %i)
+  f.write("      default:\n")
+  f.write("        throw new AssertionError();\n")
+  f.write("      }\n")
+  f.write("    case PACKED_SINGLE_BLOCK:\n")
+  f.write("      switch (bitsPerValue) {\n")
+  for i in PACKED_64_SINGLE_BLOCK_BPV:
+    f.write("      case %d:\n" %i)
+    f.write("        return PACKED_SINGLE_BLOCK_%d;\n" %i)
+  f.write("      default:\n")
+  f.write("        throw new AssertionError();\n")
+  f.write("      }\n")
+  f.write("    default:\n")
+  f.write("      throw new AssertionError();\n")
+  f.write("    }\n")
+  f.write("  }\n")
   f.write(FOOTER)
   f.close()

Modified: lucene/dev/branches/branch_4x/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Tue Aug 21 12:14:39 2012
@@ -19,3 +19,4 @@ org.apache.lucene.codecs.simpletext.Simp
 org.apache.lucene.codecs.memory.MemoryPostingsFormat
 org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
 org.apache.lucene.codecs.memory.DirectPostingsFormat
+org.apache.lucene.codecs.block.BlockPostingsFormat

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsFormat.java Tue Aug 21 12:14:39 2012
@@ -933,6 +933,7 @@ public class TestPostingsFormat extends 
     // NOTE: you can also test "weaker" index options than
     // you indexed with:
     testTerms(fieldsProducer, EnumSet.allOf(Option.class), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
+    //testTerms(fieldsProducer, EnumSet.complementOf(EnumSet.of(Option.THREADS)), IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS);
 
     fieldsProducer.close();
     dir.close();

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java Tue Aug 21 12:14:39 2012
@@ -444,9 +444,9 @@ public class TestPostingsOffsets extends
         makeToken("foo", 1, 0, 3),
         makeToken("foo", 0, 0, 3),
         makeToken("foo", 0, 0, 3)
-     });
+      });
   }
-  
+
   public void testLegalbutVeryLargeOffsets() throws Exception {
     Directory dir = newDirectory();
     IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null));

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java Tue Aug 21 12:14:39 2012
@@ -18,17 +18,24 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.LongBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
 import java.util.Random;
 
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.store.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util._TestUtil;
 import org.apache.lucene.util.packed.PackedInts.Reader;
 
 @Slow
@@ -622,4 +629,106 @@ public class TestPackedInts extends Luce
     }
   }
 
+  public void testEncodeDecode() {
+    for (PackedInts.Format format : PackedInts.Format.values()) {
+      for (int bpv = 1; bpv <= 64; ++bpv) {
+        if (!format.isSupported(bpv)) {
+          continue;
+        }
+        String msg = format + " " + bpv;
+
+        final PackedInts.Encoder encoder = PackedInts.getEncoder(format, PackedInts.VERSION_CURRENT, bpv);
+        final PackedInts.Decoder decoder = PackedInts.getDecoder(format, PackedInts.VERSION_CURRENT, bpv);
+        final int blockCount = encoder.blockCount();
+        final int valueCount = encoder.valueCount();
+        assertEquals(blockCount, decoder.blockCount());
+        assertEquals(valueCount, decoder.valueCount());
+
+        final int iterations = random().nextInt(100);
+        final int blocksOffset = random().nextInt(100);
+        final int valuesOffset = random().nextInt(100);
+        final int blocksOffset2 = random().nextInt(100);
+        final int blocksLen = iterations * blockCount;
+
+        // 1. generate random inputs
+        final long[] blocks = new long[blocksOffset + blocksLen];
+        for (int i = 0; i < blocks.length; ++i) {
+          blocks[i] = random().nextLong();
+          if (format == PackedInts.Format.PACKED_SINGLE_BLOCK && 64 % bpv != 0) {
+            // clear highest bits for packed
+            final int toClear = 64 % bpv;
+            blocks[i] = (blocks[i] << toClear) >>> toClear;
+          }
+        }
+
+        // 2. decode
+        final long[] values = new long[valuesOffset + iterations * valueCount];
+        decoder.decode(blocks, blocksOffset, values, valuesOffset, iterations);
+        for (long value : values) {
+          assertTrue(value <= PackedInts.maxValue(bpv));
+        }
+        // test decoding to int[]
+        final int[] intValues;
+        if (bpv <= 32) {
+          intValues = new int[values.length];
+          decoder.decode(blocks, blocksOffset, intValues, valuesOffset, iterations);
+          assertTrue(equals(intValues, values));
+        } else {
+          intValues = null;
+        }
+
+        // 3. re-encode
+        final long[] blocks2 = new long[blocksOffset2 + blocksLen];
+        encoder.encode(values, valuesOffset, blocks2, blocksOffset2, iterations);
+        assertArrayEquals(msg, Arrays.copyOfRange(blocks, blocksOffset, blocks.length),
+            Arrays.copyOfRange(blocks2, blocksOffset2, blocks2.length));
+        // test encoding from int[]
+        if (bpv <= 32) {
+          final long[] blocks3 = new long[blocks2.length];
+          encoder.encode(intValues, valuesOffset, blocks3, blocksOffset2, iterations);
+          assertArrayEquals(msg, blocks2, blocks3);
+        }
+
+        // 4. byte[] decoding
+        final byte[] byteBlocks = new byte[8 * blocks.length];
+        ByteBuffer.wrap(byteBlocks).asLongBuffer().put(blocks);
+        final long[] values2 = new long[valuesOffset + iterations * valueCount];
+        decoder.decode(byteBlocks, blocksOffset * 8, values2, valuesOffset, iterations);
+        for (long value : values2) {
+          assertTrue(msg, value <= PackedInts.maxValue(bpv));
+        }
+        assertArrayEquals(msg, values, values2);
+        // test decoding to int[]
+        if (bpv <= 32) {
+          final int[] intValues2 = new int[values2.length];
+          decoder.decode(byteBlocks, blocksOffset * 8, intValues2, valuesOffset, iterations);
+          assertTrue(msg, equals(intValues2, values2));
+        }
+
+        // 5. byte[] encoding
+        final byte[] blocks3 = new byte[8 * (blocksOffset2 + blocksLen)];
+        encoder.encode(values, valuesOffset, blocks3, 8 * blocksOffset2, iterations);
+        assertEquals(msg, LongBuffer.wrap(blocks2), ByteBuffer.wrap(blocks3).asLongBuffer());
+        // test encoding from int[]
+        if (bpv <= 32) {
+          final byte[] blocks4 = new byte[blocks3.length];
+          encoder.encode(intValues, valuesOffset, blocks4, 8 * blocksOffset2, iterations);
+          assertArrayEquals(msg, blocks3, blocks4);
+        }
+      }
+    }
+  }
+
+  private static boolean equals(int[] ints, long[] longs) {
+    if (ints.length != longs.length) {
+      return false;
+    }
+    for (int i = 0; i < ints.length; ++i) {
+      if ((ints[i] & 0xFFFFFFFFL) != longs[i]) {
+        return false;
+      }
+    }
+    return true;
+  }
+
 }

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSingleIntIndexInput.java Tue Aug 21 12:14:39 2012
@@ -87,7 +87,7 @@ public class MockSingleIntIndexInput ext
     }
 
     @Override
-    public void set(IntIndexInput.Index other) {
+    public void copyFrom(IntIndexInput.Index other) {
       fp = ((MockSingleIntIndexInputIndex) other).fp;
     }
 

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1375495&r1=1375494&r2=1375495&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Tue Aug 21 12:14:39 2012
@@ -311,9 +311,7 @@ public abstract class LuceneTestCase ext
     "MockFixedIntBlock",
     "MockVariableIntBlock",
     "MockSep",
-    "MockRandom",
-    "For",
-    "PFor"
+    "MockRandom"
   ));
   
   // -----------------------------------------------------------------