You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2012/08/07 15:56:31 UTC

svn commit: r1370267 [3/3] - in /lucene/dev/branches/pforcodec_3892/lucene/core/src: java/org/apache/lucene/codecs/block/ java/org/apache/lucene/codecs/blockpacked/ java/org/apache/lucene/util/packed/ test/org/apache/lucene/util/packed/

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64.java Tue Aug  7 13:56:30 2012
@@ -17,13 +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.nio.LongBuffer;
 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.
@@ -150,9 +149,9 @@ class Packed64 extends PackedInts.Mutabl
     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 % decoder.values();
+    final int offsetInBlocks = index % decoder.valueCount();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < decoder.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < decoder.valueCount() && len > 0; ++i) {
         arr[off++] = get(index++);
         --len;
       }
@@ -162,15 +161,12 @@ class Packed64 extends PackedInts.Mutabl
     }
 
     // bulk get
-    assert index % decoder.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 / decoder.values();
-    decoder.decode(
-        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
-        LongBuffer.wrap(arr, off, arr.length - off),
-        iterations);
-    final int gotValues = iterations * decoder.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;
@@ -217,9 +213,9 @@ class Packed64 extends PackedInts.Mutabl
     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 % encoder.values();
+    final int offsetInBlocks = index % encoder.valueCount();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < encoder.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < encoder.valueCount() && len > 0; ++i) {
         set(index++, arr[off++]);
         --len;
       }
@@ -229,15 +225,12 @@ class Packed64 extends PackedInts.Mutabl
     }
 
     // bulk set
-    assert index % encoder.values() == 0;
+    assert index % encoder.valueCount() == 0;
     int blockIndex = (int) ((long) index * bitsPerValue) >>> BLOCK_BITS;
     assert (((long)index * bitsPerValue) & MOD_MASK) == 0;
-    final int iterations = len / encoder.values();
-    encoder.encode(
-        LongBuffer.wrap(arr, off, arr.length - off),
-        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
-        iterations);
-    final int setValues = iterations * encoder.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/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java Tue Aug  7 13:56:30 2012
@@ -20,7 +20,6 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
-import java.nio.LongBuffer;
 import java.util.Arrays;
 
 import org.apache.lucene.store.DataInput;
@@ -88,14 +87,11 @@ abstract class Packed64SingleBlock exten
     // bulk get
     assert index % valuesPerBlock == 0;
     final PackedInts.Decoder decoder = BulkOperation.of(PackedInts.Format.PACKED_SINGLE_BLOCK, bitsPerValue);
-    assert decoder.blocks() == 1;
-    assert decoder.values() == valuesPerBlock;
+    assert decoder.blockCount() == 1;
+    assert decoder.valueCount() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    decoder.decode(
-        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
-        LongBuffer.wrap(arr, off, arr.length - off),
-        nblocks);
+    decoder.decode(blocks, blockIndex, arr, off, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 
@@ -135,14 +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.encode(
-        LongBuffer.wrap(arr, off, arr.length - off),
-        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
-        nblocks);
+    op.encode(arr, off, blocks, blockIndex, nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Tue Aug  7 13:56:30 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,10 +26,6 @@ import org.apache.lucene.store.DataOutpu
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.LongsRef;
 
-import java.io.IOException;
-import java.nio.IntBuffer;
-import java.nio.LongBuffer;
-
 /**
  * Simplistic compression for array of unsigned long values.
  * Each value is >= 0 and <= a specified maximum value.  The
@@ -251,37 +248,39 @@ public class PackedInts {
     /**
      * The minimum number of long blocks to decode in a single call.
      */
-    int blocks();
+    int blockCount();
 
     /**
-     * The number of values that can be stored in <code>blocks()</code> long
+     * The number of values that can be stored in <code>blockCount()</code> long
      * blocks.
      */
-    int values();
+    int valueCount();
 
     /**
-     * Read <code>iterations * blocks()</code> blocks from <code>blocks</code>,
-     * decode them and write <code>iterations * values()</code> values into
+     * 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(LongBuffer blocks, LongBuffer values, int iterations);
+    void decode(long[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations);
 
     /**
-     * Read <code>iterations * blocks()</code> blocks from <code>blocks</code>,
-     * decode them and write <code>iterations * values()</code> values into
-     * <code>values</code>. This method will throw an
-     * {@link UnsupportedOperationException} if the values require more than
-     * 32 bits of storage.
+     * 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(LongBuffer blocks, IntBuffer values, int iterations);
+    void decode(byte[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations);
 
   }
 
@@ -293,35 +292,39 @@ public class PackedInts {
     /**
      * The minimum number of long blocks to encode in a single call.
      */
-    int blocks();
+    int blockCount();
 
     /**
-     * The number of values that can be stored in <code>blocks()</code> long
+     * The number of values that can be stored in <code>blockCount()</code> long
      * blocks.
      */
-    int values();
+    int valueCount();
 
     /**
-     * Read <code>iterations * values()</code> values from <code>values</code>,
-     * encode them and write <code>iterations * blocks()</code> blocks into
+     * 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(LongBuffer values, LongBuffer blocks, int iterations);
+    void encode(long[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations);
 
     /**
-     * Read <code>iterations * values()</code> values from <code>values</code>,
-     * encode them and write <code>iterations * blocks()</code> blocks into
+     * 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(IntBuffer values, LongBuffer blocks, int iterations);
+    void encode(long[] values, int valuesOffset, byte[] blocks, int blocksOffset, int iterations);
 
   }
 
@@ -1001,4 +1004,4 @@ public class PackedInts {
     }
   }
 
-}
+}
\ No newline at end of file

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java Tue Aug  7 13:56:30 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.util.packed;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.nio.LongBuffer;
 
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.LongsRef;
@@ -28,9 +27,8 @@ final class PackedReaderIterator extends
 
   final PackedInts.Format format;
   final BulkOperation bulkOperation;
-  final LongBuffer nextBlocks;
+  final long[] nextBlocks;
   final LongsRef nextValues;
-  final LongBuffer nextValuesBuffer;
   final int iterations;
   int position;
 
@@ -40,11 +38,10 @@ final class PackedReaderIterator extends
     bulkOperation = BulkOperation.of(format, bitsPerValue);
     iterations = bulkOperation.computeIterations(valueCount, mem);
     assert iterations > 0;
-    nextBlocks = LongBuffer.allocate(iterations * bulkOperation.blocks());
-    nextValues = new LongsRef(new long[iterations * bulkOperation.values()], 0, 0);
-    nextValuesBuffer = LongBuffer.wrap(nextValues.longs);
-    assert iterations * bulkOperation.values() == nextValues.longs.length;
-    assert iterations * bulkOperation.blocks() == nextBlocks.capacity();
+    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;
   }
@@ -54,9 +51,7 @@ final class PackedReaderIterator extends
     assert nextValues.length >= 0;
     assert count > 0;
     assert nextValues.offset + nextValues.length <= nextValues.longs.length;
-
-    final long[] nextBlocks = this.nextBlocks.array();
-
+    
     nextValues.offset += nextValues.length;
 
     final int remaining = valueCount - position - 1;
@@ -75,9 +70,7 @@ final class PackedReaderIterator extends
         nextBlocks[i] = 0L;
       }
 
-      this.nextBlocks.rewind();
-      nextValuesBuffer.clear();
-      bulkOperation.decode(this.nextBlocks, nextValuesBuffer, iterations);
+      bulkOperation.decode(nextBlocks, 0, nextValues.longs, 0, iterations);
       nextValues.offset = 0;
     }
 

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.java Tue Aug  7 13:56:30 2012
@@ -21,7 +21,7 @@ import org.apache.lucene.store.DataOutpu
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.nio.LongBuffer;
+import java.util.Arrays;
 
 // Packs high order byte first, to match
 // IndexOutput.writeInt/Long/Short byte order
@@ -30,19 +30,21 @@ final class PackedWriter extends PackedI
 
   boolean finished;
   final PackedInts.Format format;
-  final BulkOperation bulkOperation;
-  final LongBuffer nextBlocks;
-  final LongBuffer nextValues;
+  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) {
     super(out, valueCount, bitsPerValue);
     this.format = format;
-    bulkOperation = BulkOperation.of(format, bitsPerValue);
-    iterations = bulkOperation.computeIterations(valueCount, mem);
-    nextBlocks = LongBuffer.allocate(iterations * bulkOperation.blocks());
-    nextValues = LongBuffer.allocate(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;
   }
@@ -59,8 +61,8 @@ final class PackedWriter extends PackedI
     if (valueCount != -1 && written >= valueCount) {
       throw new EOFException("Writing past end of stream");
     }
-    nextValues.put(v);
-    if (nextValues.remaining() == 0) {
+    nextValues[off++] = v;
+    if (off == nextValues.length) {
       flush();
     }
     ++written;
@@ -79,16 +81,13 @@ final class PackedWriter extends PackedI
   }
 
   private void flush() throws IOException {
-    final int nvalues = nextValues.position();
-    nextValues.rewind();
-    nextBlocks.clear();
-    bulkOperation.encode(nextValues, nextBlocks, iterations);
-    final int blocks = format.nblocks(bitsPerValue, nvalues);
-    nextBlocks.rewind();
+    encoder.encode(nextValues, 0, nextBlocks, 0, iterations);
+    final int blocks = format.nblocks(bitsPerValue, off);
     for (int i = 0; i < blocks; ++i) {
-      out.writeLong(nextBlocks.get());
+      out.writeLong(nextBlocks[i]);
     }
-    nextValues.clear();
+    Arrays.fill(nextValues, 0L);
+    off = 0;
   }
 
   @Override

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py Tue Aug  7 13:56:30 2012
@@ -42,24 +42,34 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import java.nio.IntBuffer;
 import java.nio.LongBuffer;
-import java.util.EnumMap;
+import java.nio.ByteBuffer;
 
 /**
  * Efficient sequential read/write of packed integers.
  */
-abstract class BulkOperation implements PackedInts.Decoder, PackedInts.Encoder {
+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);
-    }
-    return ops[bitsPerValue];
+  private static long[] toLongArray(byte[] bytes, int offset, int length) {
+    final int longsLen = length >>> 3;
+    LongBuffer longs = LongBuffer.allocate(longsLen);
+    longs.put(ByteBuffer.wrap(bytes, offset, length).asLongBuffer());
+    return longs.array();
+  }
+
+  @Override
+  public void decode(byte[] blocks, int blocksOffset, long[] values, int valuesOffset, int iterations) {
+    decode(toLongArray(blocks, blocksOffset, 8 * iterations * blockCount()), 0, values, valuesOffset, 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);
   }
 
   /**
@@ -69,7 +79,7 @@ abstract class BulkOperation implements 
    *  - 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
@@ -81,22 +91,20 @@ abstract class BulkOperation implements 
    * <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;
     }
   }
-
+}
 """
 
-FOOTER = "}"
-
 def casts(typ):
   cast_start = "(%s) (" %typ
   cast_end = ")"
@@ -124,64 +132,62 @@ def get_type(bits):
 
 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, 32)
   p64sb_decode(bpv, 64)
-  p64sb_encode(bpv, 32)
+  #p64sb_encode(bpv, 32)
   p64sb_encode(bpv, 64)
-  f.write("  }\n")
+  f.write("  }")
 
 def p64sb_decode(bpv, bits):
   values = 64 / bpv
   typ = get_type(bits)
-  buf = typ.title() + "Buffer"
   cast_start, cast_end = casts(typ)
-  f.write("    public void decode(LongBuffer blocks, %s values, int iterations) {\n" %buf)
+  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 blocks.position() + iterations * blocks() <= blocks.limit();\n")
-  f.write("      assert values.position() + iterations * values() <= values.limit();\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")
-  f.write("        final long block = blocks.get();\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.put(%sblock & %dL%s);\n" %(cast_start, mask, cast_end))
+      f.write("        values[valuesOffset++] = %sblock & %dL%s;\n" %(cast_start, mask, cast_end))
     elif i == values - 1:
-      f.write("        values.put(%sblock >>> %d%s);\n" %(cast_start, i * bpv, cast_end))
+      f.write("        values[valuesOffset++] = %sblock >>> %d%s;\n" %(cast_start, i * bpv, cast_end))
     else:
-      f.write("        values.put(%s(block >>> %d) & %dL%s);\n" %(cast_start, i * bpv, mask, cast_end))
+      f.write("        values[valuesOffset++] = %s(block >>> %d) & %dL%s;\n" %(cast_start, i * bpv, mask, cast_end))
   f.write("      }\n")
   f.write("    }\n\n")
 
 def p64sb_encode(bpv, bits):
   values = 64 / bpv
   typ = get_type(bits)
-  buf = typ.title() + "Buffer"
   mask_start, mask_end = masks(bits)
-  f.write("    public void encode(%s values, LongBuffer blocks, int iterations) {\n" %buf)
-  f.write("      assert blocks.position() + iterations * blocks() <= blocks.limit();\n")
-  f.write("      assert values.position() + iterations * values() <= values.limit();\n")
+  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(values):
     block_offset = i / values
     offset_in_block = i % values
     if i == 0:
-      f.write("        blocks.put(%svalues.get()%s" %(mask_start, mask_end))
+      f.write("        blocks[blocksOffset++] = %svalues[valuesOffset++]%s" %(mask_start, mask_end))
     else:
-      f.write(" | (%svalues.get()%s << %d)" %(mask_start, mask_end, 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\n")
 
@@ -193,58 +199,43 @@ 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 decode(LongBuffer blocks, LongBuffer values, int iterations) {
-      final int originalLimit = blocks.limit();
-      blocks.limit(blocks.position() + iterations * blocks());
-      values.put(blocks);
-      blocks.limit(originalLimit);
-    }
-
-    public void decode(LongBuffer blocks, IntBuffer values, int iterations) {
-      throw new UnsupportedOperationException();
+    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 encode(LongBuffer values, LongBuffer blocks, int iterations) {
-      final int originalLimit = values.limit();
-      values.limit(values.position() + iterations * values());
-      blocks.put(values);
-      values.limit(originalLimit);
+    public void encode(long[] values, int valuesOffset, long[] blocks, int blocksOffset, int iterations) {
+      System.arraycopy(values, valuesOffset, blocks, blocksOffset, valueCount() * iterations);
     }
 
-    public void encode(IntBuffer values, LongBuffer blocks, int iterations) {
-      for (int i = values.position(), end = values.position() + iterations, j = blocks.position(); i < end; ++i, ++j) {
-        blocks.put(j, values.get(i));
-      }
-    }
   }
 """)
   else:
-    p64_decode(bpv, 32, values)
+    #p64_decode(bpv, 32, values)
     p64_decode(bpv, 64, values)
-    p64_encode(bpv, 32, values)
+    #p64_encode(bpv, 32, values)
     p64_encode(bpv, 64, values)
     f.write("  }\n")
 
 def p64_decode(bpv, bits, values):
   typ = get_type(bits)
-  buf = typ.title() + "Buffer"
   cast_start, cast_end = casts(typ)
-  f.write("    public void decode(LongBuffer blocks, %s values, int iterations) {\n" %buf)
+
+  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 blocks.position() + iterations * blocks() <= blocks.limit();\n")
-  f.write("      assert values.position() + iterations * values() <= values.limit();\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):
@@ -252,49 +243,94 @@ def p64_decode(bpv, bits, values):
     bit_offset = (i * bpv) % 64
     if bit_offset == 0:
       # start of block
-      f.write("        final long block%d = blocks.get();\n" %block_offset);
-      f.write("        values.put(%sblock%d >>> %d%s);\n" %(cast_start, block_offset, 64 - bpv, cast_end))
+      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.put(%sblock%d & %dL%s);\n" %(cast_start, block_offset, mask, cast_end))
+      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.put(%s(block%d >>> %d) & %dL%s);\n" %(cast_start, block_offset, 64 - bit_offset - bpv, mask, cast_end))
+      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.get();\n" %(block_offset + 1));
-      f.write("        values.put(%s((block%d & %dL) << %d) | (block%d >>> %d)%s);\n" %(cast_start, block_offset, mask1, shift1, block_offset + 1, shift2, cast_end))
+      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")
 
 def p64_encode(bpv, bits, values):
   typ = get_type(bits)
-  buf = typ.title() + "Buffer"
   mask_start, mask_end = masks(bits)
-  f.write("    public void encode(%s values, LongBuffer blocks, int iterations) {\n" %buf)
-  f.write("      assert blocks.position() + iterations * blocks() <= blocks.limit();\n")
-  f.write("      assert values.position() + iterations * values() <= values.limit();\n")
+  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.put((%svalues.get()%s << %d)" %(mask_start, mask_end, 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(" | %svalues.get()%s);\n" %(mask_start, mask_end))
+      f.write(" | %svalues[valuesOffset++]%s;\n" %(mask_start, mask_end))
     elif bit_offset + bpv < 64:
       # inside a block
-      f.write(" | (%svalues.get()%s << %d)" %(mask_start, mask_end, 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(" | (%svalues.get(values.position())%s >>> %d));\n" %(mask_start, mask_end, right_bits))
-      f.write("        blocks.put((%svalues.get()%s << %d)" %(mask_start, mask_end, 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\n")
 
@@ -303,17 +339,36 @@ 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]);\n")
-  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/pforcodec_3892/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java Tue Aug  7 13:56:30 2012
@@ -18,19 +18,24 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
-import java.nio.IntBuffer;
+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
@@ -630,55 +635,48 @@ public class TestPackedInts extends Luce
         if (!format.isSupported(bpv)) {
           continue;
         }
-        PackedInts.Encoder encoder = PackedInts.getEncoder(format, PackedInts.VERSION_CURRENT, bpv);
-        PackedInts.Decoder decoder = PackedInts.getDecoder(format, PackedInts.VERSION_CURRENT, bpv);
-        final int nblocks = encoder.blocks();
-        final int nvalues = encoder.values();
-        assertEquals(nblocks, decoder.blocks());
-        assertEquals(nvalues, decoder.values());
-        final int iterations = _TestUtil.nextInt(random(), 1, 9);
-        assertEquals(format.nblocks(bpv, nvalues * iterations), nblocks * iterations);
-        final int blocksPosition = random().nextInt(10);
-        final int valuesPosition = random().nextInt(10);
-        final LongBuffer blocks = LongBuffer.allocate(blocksPosition + nblocks * iterations);
-        blocks.position(blocksPosition);
-        final LongBuffer values = LongBuffer.allocate(valuesPosition + nvalues * iterations);
-        values.position(valuesPosition);
-
-        for (int i = 0; i < iterations * nblocks; ++i) {
-          blocks.put(blocks.position() + i, random().nextLong());
-        }
-        decoder.decode(blocks, values, iterations);
-
-        final int restoredBlocksPosition = random().nextInt(10);
-        final LongBuffer restoredBlocks = LongBuffer.allocate(restoredBlocksPosition + nblocks * iterations);
-        values.position(valuesPosition);
-        restoredBlocks.position(restoredBlocksPosition);
-        encoder.encode(values, restoredBlocks, iterations);
-
-        blocks.position(blocksPosition);
-        blocks.limit(blocksPosition + nblocks * iterations);
-        restoredBlocks.position(restoredBlocksPosition);
-        restoredBlocks.limit(restoredBlocksPosition + nblocks * iterations);
-        assertEquals(blocks, restoredBlocks);
-
-        if (bpv <= 32) {
-          final IntBuffer intValues = IntBuffer.allocate(valuesPosition + nvalues * iterations);
-          intValues.position(valuesPosition);
-
-          blocks.position(blocksPosition);
-          decoder.decode(blocks, intValues, iterations);
-
-          intValues.position(valuesPosition);
-          restoredBlocks.position(restoredBlocksPosition);
-          encoder.encode(intValues, restoredBlocks, iterations);
-
-          blocks.position(blocksPosition);
-          blocks.limit(blocksPosition + nblocks * iterations);
-          restoredBlocks.position(restoredBlocksPosition);
-          restoredBlocks.limit(restoredBlocksPosition + nblocks * iterations);
-          assertEquals(blocks, restoredBlocks);
-        }
+        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();
+        }
+
+        // 2. decode
+        final long[] values = new long[valuesOffset + iterations * valueCount];
+        decoder.decode(blocks, blocksOffset, values, valuesOffset, iterations);
+
+        // 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));
+
+        // 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);
+        assertArrayEquals(msg, values, 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());
       }
     }
   }