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/07/21 21:26:46 UTC

svn commit: r1364157 [3/3] - in /lucene/dev/branches/pforcodec_3892/lucene/core/src: 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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -21,6 +21,7 @@ import org.apache.lucene.store.DataInput
 import org.apache.lucene.util.RamUsageEstimator;
 
 import java.io.IOException;
+import java.nio.LongBuffer;
 import java.util.Arrays;
 
 /**
@@ -146,12 +147,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.values();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < op.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < decoder.values() && len > 0; ++i) {
         arr[off++] = get(index++);
         --len;
       }
@@ -161,12 +162,15 @@ class Packed64 extends PackedInts.Mutabl
     }
 
     // bulk get
-    assert index % op.values() == 0;
+    assert index % decoder.values() == 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.values();
+    decoder.decode(
+        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
+        LongBuffer.wrap(arr, off, arr.length - off),
+        iterations);
+    final int gotValues = iterations * decoder.values();
     index += gotValues;
     len -= gotValues;
     assert len >= 0;
@@ -210,12 +214,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.values();
     if (offsetInBlocks != 0) {
-      for (int i = offsetInBlocks; i < op.values() && len > 0; ++i) {
+      for (int i = offsetInBlocks; i < encoder.values() && len > 0; ++i) {
         set(index++, arr[off++]);
         --len;
       }
@@ -224,13 +228,16 @@ class Packed64 extends PackedInts.Mutabl
       }
     }
 
-    // bulk get
-    assert index % op.values() == 0;
+    // bulk set
+    assert index % encoder.values() == 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.values();
+    encoder.encode(
+        LongBuffer.wrap(arr, off, arr.length - off),
+        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
+        iterations);
+    final int setValues = iterations * encoder.values();
     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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
+import java.nio.LongBuffer;
 import java.util.Arrays;
 
 import org.apache.lucene.store.DataInput;
@@ -86,12 +87,15 @@ 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.blocks() == 1;
+    assert decoder.values() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.get(blocks, blockIndex, arr, off, nblocks);
+    decoder.decode(
+        LongBuffer.wrap(blocks, blockIndex, blocks.length - blockIndex),
+        LongBuffer.wrap(arr, off, arr.length - off),
+        nblocks);
     final int diff = nblocks * valuesPerBlock;
     index += diff; len -= diff;
 
@@ -135,7 +139,10 @@ abstract class Packed64SingleBlock exten
     assert op.values() == valuesPerBlock;
     final int blockIndex = index / valuesPerBlock;
     final int nblocks = (index + len) / valuesPerBlock - blockIndex;
-    op.set(blocks, blockIndex, arr, off, nblocks);
+    op.encode(
+        LongBuffer.wrap(arr, off, arr.length - off),
+        LongBuffer.wrap(blocks, blockIndex, blocks.length - 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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -26,6 +26,8 @@ import org.apache.lucene.store.IndexInpu
 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.
@@ -242,6 +244,88 @@ 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 blocks();
+
+    /**
+     * The number of values that can be stored in <code>blocks()</code> long
+     * blocks.
+     */
+    int values();
+
+    /**
+     * Read <code>iterations * blocks()</code> blocks from <code>blocks</code>,
+     * decode them and write <code>iterations * values()</code> values into
+     * <code>values</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param values       the values buffer
+     * @param iterations   controls how much data to decode
+     */
+    void decode(LongBuffer blocks, LongBuffer values, 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.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param values       the values buffer
+     * @param iterations   controls how much data to decode
+     */
+    void decode(LongBuffer blocks, IntBuffer values, int iterations);
+
+  }
+
+  /**
+   * An encoder for packed integers.
+   */
+  public static interface Encoder {
+
+    /**
+     * The minimum number of long blocks to encode in a single call.
+     */
+    int blocks();
+
+    /**
+     * The number of values that can be stored in <code>blocks()</code> long
+     * blocks.
+     */
+    int values();
+
+    /**
+     * Read <code>iterations * values()</code> values from <code>values</code>,
+     * encode them and write <code>iterations * blocks()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param values       the values buffer
+     * @param iterations   controls how much data to encode
+     */
+    void encode(LongBuffer values, LongBuffer blocks, int iterations);
+
+    /**
+     * Read <code>iterations * values()</code> values from <code>values</code>,
+     * encode them and write <code>iterations * blocks()</code> blocks into
+     * <code>blocks</code>.
+     *
+     * @param blocks       the long blocks that hold packed integer values
+     * @param values       the values buffer
+     * @param iterations   controls how much data to encode
+     */
+    void encode(IntBuffer values, LongBuffer blocks, int iterations);
+
+  }
+
+  /**
    * A read-only random access array of positive integers.
    * @lucene.internal
    */
@@ -490,8 +574,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 +612,30 @@ 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) {
+    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) {
+    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
@@ -612,7 +719,7 @@ 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) {
     return new PackedReaderIterator(format, valueCount, bitsPerValue, in, mem);
   }
 
@@ -652,7 +759,7 @@ 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) {
     switch (format) {
       case PACKED:
         return new DirectPackedReader(bitsPerValue, valueCount, in);
@@ -784,7 +891,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);
   }
 

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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -19,6 +19,7 @@ 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;
@@ -27,8 +28,9 @@ final class PackedReaderIterator extends
 
   final PackedInts.Format format;
   final BulkOperation bulkOperation;
-  final long[] nextBlocks;
+  final LongBuffer nextBlocks;
   final LongsRef nextValues;
+  final LongBuffer nextValuesBuffer;
   final int iterations;
   int position;
 
@@ -38,10 +40,11 @@ final class PackedReaderIterator extends
     bulkOperation = BulkOperation.of(format, bitsPerValue);
     iterations = bulkOperation.computeIterations(valueCount, mem);
     assert iterations > 0;
-    nextBlocks = new long[iterations * bulkOperation.blocks()];
+    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.length;
+    assert iterations * bulkOperation.blocks() == nextBlocks.capacity();
     nextValues.offset = nextValues.longs.length;
     position = -1;
   }
@@ -51,7 +54,9 @@ 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;
@@ -70,7 +75,9 @@ final class PackedReaderIterator extends
         nextBlocks[i] = 0L;
       }
 
-      bulkOperation.get(nextBlocks, 0, nextValues.longs, 0, iterations);
+      this.nextBlocks.rewind();
+      nextValuesBuffer.clear();
+      bulkOperation.decode(this.nextBlocks, nextValuesBuffer, 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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -21,6 +21,7 @@ import org.apache.lucene.store.DataOutpu
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.nio.LongBuffer;
 
 // Packs high order byte first, to match
 // IndexOutput.writeInt/Long/Short byte order
@@ -30,21 +31,18 @@ final class PackedWriter extends PackedI
   boolean finished;
   final PackedInts.Format format;
   final BulkOperation bulkOperation;
-  final long[] nextBlocks;
-  final long[] nextValues;
+  final LongBuffer nextBlocks;
+  final LongBuffer 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()];
-    off = 0;
+    nextBlocks = LongBuffer.allocate(iterations * bulkOperation.blocks());
+    nextValues = LongBuffer.allocate(iterations * bulkOperation.values());
     written = 0;
     finished = false;
   }
@@ -61,10 +59,9 @@ final class PackedWriter extends PackedI
     if (valueCount != -1 && written >= valueCount) {
       throw new EOFException("Writing past end of stream");
     }
-    nextValues[off++] = v;
-    if (off == nextValues.length) {
-      flush(nextValues.length);
-      off = 0;
+    nextValues.put(v);
+    if (nextValues.remaining() == 0) {
+      flush();
     }
     ++written;
   }
@@ -77,17 +74,21 @@ 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);
+  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();
     for (int i = 0; i < blocks; ++i) {
-      out.writeLong(nextBlocks[i]);
+      out.writeLong(nextBlocks.get());
     }
-    off = 0;
+    nextValues.clear();
   }
 
   @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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -42,12 +42,14 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import java.nio.IntBuffer;
+import java.nio.LongBuffer;
 import java.util.EnumMap;
 
 /**
  * Efficient sequential read/write of packed integers.
  */
-abstract class BulkOperation {
+abstract class BulkOperation implements PackedInts.Decoder, PackedInts.Encoder {
 
   static final EnumMap<PackedInts.Format, BulkOperation[]> BULK_OPERATIONS = new EnumMap<PackedInts.Format, BulkOperation[]>(PackedInts.Format.class);
 
@@ -91,30 +93,35 @@ abstract class BulkOperation {
     }
   }
 
-  /**
-   * 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)
@@ -124,42 +131,59 @@ def packed64singleblock(bpv, f):
   f.write("    public int values() {\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("  }\n")
 
-  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)
+  buf = typ.title() + "Buffer"
+  cast_start, cast_end = casts(typ)
+  f.write("    public void decode(LongBuffer blocks, %s values, int iterations) {\n" %buf)
+  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("      for (int i = 0; i < iterations; ++i) {\n")
-  f.write("        final long block = blocks[bi++];\n")
+  f.write("        final long block = blocks.get();\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.put(%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.put(%sblock >>> %d%s);\n" %(cast_start, i * bpv, cast_end))
     else:
-      f.write("        values[vi++] = (block >>> %d) & %dL;\n" %(i * bpv, mask))
+      f.write("        values.put(%s(block >>> %d) & %dL%s);\n" %(cast_start, i * bpv, mask, cast_end))
   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)
+  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("      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.put(%svalues.get()%s" %(mask_start, mask_end))
     else:
-      f.write(" | (values[vi++] << %d)" %(i * bpv))
+      f.write(" | (%svalues.get()%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")
+  f.write("    }\n\n")
 
 def packed64(bpv, f):
   blocks = bpv
@@ -178,70 +202,101 @@ def packed64(bpv, f):
   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(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();
     }
 
-    public void set(long[] blocks, int bi, long[] values, int vi, int iterations) {
-      System.arraycopy(values, bi, blocks, vi, 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(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, 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)
+  buf = typ.title() + "Buffer"
+  cast_start, cast_end = casts(typ)
+  f.write("    public void decode(LongBuffer blocks, %s values, int iterations) {\n" %buf)
+  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 blocks.position() + iterations * blocks() <= blocks.limit();\n")
+  f.write("      assert values.position() + iterations * values() <= values.limit();\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.get();\n" %block_offset);
+      f.write("        values.put(%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.put(%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.put(%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.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("      }\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)
+  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("      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.put((%svalues.get()%s << %d)" %(mask_start, mask_end, 64 - bpv))
     elif bit_offset + bpv == 64:
       # end of block
-      f.write(" | values[vi++];\n")
+      f.write(" | %svalues.get()%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.get()%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.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("      }\n")
-  f.write("    }\n")
-
-  f.write("  }\n\n")
-
+  f.write("    }\n\n")
 
 
 if __name__ == '__main__':
@@ -249,7 +304,7 @@ if __name__ == '__main__':
   f = open(OUTPUT_FILE, 'w')
   f.write(HEADER)
   f.write("  static {\n")
-  f.write("    BULK_OPERATIONS.put(PackedInts.Format.PACKED, new BulkOperation[65]);")
+  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")

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=1364157&r1=1364156&r2=1364157&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 Sat Jul 21 19:26:45 2012
@@ -18,6 +18,8 @@ package org.apache.lucene.util.packed;
  */
 
 import java.io.IOException;
+import java.nio.IntBuffer;
+import java.nio.LongBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Locale;
@@ -622,4 +624,63 @@ 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;
+        }
+        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);
+        }
+      }
+    }
+  }
+
 }