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/08 16:15:23 UTC

svn commit: r1370781 - in /lucene/dev/branches/pforcodec_3892/lucene/core/src: java/org/apache/lucene/codecs/blockpacked/ java/org/apache/lucene/util/packed/ test/org/apache/lucene/codecs/blockpacked/ test/org/apache/lucene/util/packed/

Author: jpountz
Date: Wed Aug  8 14:15:22 2012
New Revision: 1370781

URL: http://svn.apache.org/viewvc?rev=1370781&view=rev
Log:
LUCENE-3892: Ability to select the right format based on an `acceptableOverheadRatio`.

The `acceptableOverheadRatio` is currenlty configurable through
BlockPackedPostingsWriter's constructor and defaults to PackedInts.DEFAULT.

Added:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/TestForUtil.java
Modified:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/BulkOperation.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/gen_BulkOperation.py
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/util/packed/TestPackedInts.java

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java?rev=1370781&r1=1370780&r2=1370781&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsReader.java Wed Aug  8 14:15:22 2012
@@ -18,6 +18,8 @@ package org.apache.lucene.codecs.blockpa
  */
 
 import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.blockpacked.ForUtil.MIN_DATA_SIZE;
+import static org.apache.lucene.codecs.blockpacked.ForUtil.MIN_ENCODED_SIZE;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -25,7 +27,6 @@ import java.util.Arrays;
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.codecs.blockpacked.BlockPackedSkipReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
@@ -57,6 +58,8 @@ public final class BlockPackedPostingsRe
   private final IndexInput posIn;
   private final IndexInput payIn;
 
+  private final ForUtil forUtil;
+
   public static boolean DEBUG = false;
 
   // nocommit
@@ -75,6 +78,7 @@ public final class BlockPackedPostingsRe
                             BlockPackedPostingsWriter.DOC_CODEC,
                             BlockPackedPostingsWriter.VERSION_START,
                             BlockPackedPostingsWriter.VERSION_START);
+      forUtil = new ForUtil(docIn);
 
       if (fieldInfos.hasProx()) {
         posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, BlockPackedPostingsFormat.POS_EXTENSION),
@@ -294,8 +298,8 @@ public final class BlockPackedPostingsRe
   final class BlockDocsEnum extends DocsEnum {
     private final byte[] encoded;
     
-    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
-    private final long[] freqBuffer = new long[BLOCK_SIZE];
+    private final long[] docDeltaBuffer = new long[MIN_DATA_SIZE];
+    private final long[] freqBuffer = new long[MIN_DATA_SIZE];
 
     private int docBufferUpto;
 
@@ -333,7 +337,7 @@ public final class BlockPackedPostingsRe
       indexHasPos = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       indexHasPayloads = fieldInfo.hasPayloads();
-      encoded = new byte[BLOCK_SIZE * 4];    
+      encoded = new byte[MIN_ENCODED_SIZE];    
     }
 
     public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) {
@@ -383,13 +387,13 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("    fill doc block from fp=" + docIn.getFilePointer());
         }
-        ForUtil.readBlock(docIn, encoded, docDeltaBuffer);
+        forUtil.readBlock(docIn, encoded, docDeltaBuffer);
 
         if (indexHasFreq) {
           if (DEBUG) {
             System.out.println("    fill freq block from fp=" + docIn.getFilePointer());
           }
-          ForUtil.readBlock(docIn, encoded, freqBuffer);
+          forUtil.readBlock(docIn, encoded, freqBuffer);
         }
       } else {
         // Read vInts:
@@ -540,9 +544,9 @@ public final class BlockPackedPostingsRe
     
     private final byte[] encoded;
 
-    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
-    private final long[] freqBuffer = new long[BLOCK_SIZE];
-    private final long[] posDeltaBuffer = new long[BLOCK_SIZE];
+    private final long[] docDeltaBuffer = new long[MIN_DATA_SIZE];
+    private final long[] freqBuffer = new long[MIN_DATA_SIZE];
+    private final long[] posDeltaBuffer = new long[MIN_DATA_SIZE];
 
     private int docBufferUpto;
     private int posBufferUpto;
@@ -599,7 +603,7 @@ public final class BlockPackedPostingsRe
       this.startDocIn = BlockPackedPostingsReader.this.docIn;
       this.docIn = (IndexInput) startDocIn.clone();
       this.posIn = (IndexInput) BlockPackedPostingsReader.this.posIn.clone();
-      encoded = new byte[BLOCK_SIZE*4];
+      encoded = new byte[MIN_ENCODED_SIZE];
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       indexHasPayloads = fieldInfo.hasPayloads();
     }
@@ -657,11 +661,11 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("    fill doc block from fp=" + docIn.getFilePointer());
         }
-        ForUtil.readBlock(docIn, encoded, docDeltaBuffer);
+        forUtil.readBlock(docIn, encoded, docDeltaBuffer);
         if (DEBUG) {
           System.out.println("    fill freq block from fp=" + docIn.getFilePointer());
         }
-        ForUtil.readBlock(docIn, encoded, freqBuffer);
+        forUtil.readBlock(docIn, encoded, freqBuffer);
       } else {
         // Read vInts:
         if (DEBUG) {
@@ -704,7 +708,7 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("        bulk pos block @ fp=" + posIn.getFilePointer());
         }
-        ForUtil.readBlock(posIn, encoded, posDeltaBuffer);
+        forUtil.readBlock(posIn, encoded, posDeltaBuffer);
       }
     }
 
@@ -873,7 +877,7 @@ public final class BlockPackedPostingsRe
             System.out.println("        skip whole block @ fp=" + posIn.getFilePointer());
           }
           assert posIn.getFilePointer() != lastPosBlockFP;
-          ForUtil.skipBlock(posIn);
+          forUtil.skipBlock(posIn);
           toSkip -= BLOCK_SIZE;
         }
         refillPositions();
@@ -945,9 +949,9 @@ public final class BlockPackedPostingsRe
     
     private final byte[] encoded;
 
-    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
-    private final long[] freqBuffer = new long[BLOCK_SIZE];
-    private final long[] posDeltaBuffer = new long[BLOCK_SIZE];
+    private final long[] docDeltaBuffer = new long[MIN_DATA_SIZE];
+    private final long[] freqBuffer = new long[MIN_DATA_SIZE];
+    private final long[] posDeltaBuffer = new long[MIN_DATA_SIZE];
 
     private final long[] payloadLengthBuffer;
     private final long[] offsetStartDeltaBuffer;
@@ -1023,11 +1027,11 @@ public final class BlockPackedPostingsRe
       this.docIn = (IndexInput) startDocIn.clone();
       this.posIn = (IndexInput) BlockPackedPostingsReader.this.posIn.clone();
       this.payIn = (IndexInput) BlockPackedPostingsReader.this.payIn.clone();
-      encoded = new byte[BLOCK_SIZE*4];
+      encoded = new byte[MIN_ENCODED_SIZE];
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       if (indexHasOffsets) {
-        offsetStartDeltaBuffer = new long[BLOCK_SIZE];
-        offsetLengthBuffer = new long[BLOCK_SIZE];
+        offsetStartDeltaBuffer = new long[MIN_DATA_SIZE];
+        offsetLengthBuffer = new long[MIN_DATA_SIZE];
       } else {
         offsetStartDeltaBuffer = null;
         offsetLengthBuffer = null;
@@ -1037,7 +1041,7 @@ public final class BlockPackedPostingsRe
 
       indexHasPayloads = fieldInfo.hasPayloads();
       if (indexHasPayloads) {
-        payloadLengthBuffer = new long[BLOCK_SIZE];
+        payloadLengthBuffer = new long[MIN_DATA_SIZE];
         payloadBytes = new byte[128];
         payload = new BytesRef();
       } else {
@@ -1102,11 +1106,11 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("    fill doc block from fp=" + docIn.getFilePointer());
         }
-        ForUtil.readBlock(docIn, encoded, docDeltaBuffer);
+        forUtil.readBlock(docIn, encoded, docDeltaBuffer);
         if (DEBUG) {
           System.out.println("    fill freq block from fp=" + docIn.getFilePointer());
         }
-        ForUtil.readBlock(docIn, encoded, freqBuffer);
+        forUtil.readBlock(docIn, encoded, freqBuffer);
       } else {
         if (DEBUG) {
           System.out.println("    fill last vInt doc block from fp=" + docIn.getFilePointer());
@@ -1166,13 +1170,13 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("        bulk pos block @ fp=" + posIn.getFilePointer());
         }
-        ForUtil.readBlock(posIn, encoded, posDeltaBuffer);
+        forUtil.readBlock(posIn, encoded, posDeltaBuffer);
 
         if (indexHasPayloads) {
           if (DEBUG) {
             System.out.println("        bulk payload block @ pay.fp=" + payIn.getFilePointer());
           }
-          ForUtil.readBlock(payIn, encoded, payloadLengthBuffer);
+          forUtil.readBlock(payIn, encoded, payloadLengthBuffer);
           int numBytes = payIn.readVInt();
           if (DEBUG) {
             System.out.println("        " + numBytes + " payload bytes @ pay.fp=" + payIn.getFilePointer());
@@ -1188,8 +1192,8 @@ public final class BlockPackedPostingsRe
           if (DEBUG) {
             System.out.println("        bulk offset block @ pay.fp=" + payIn.getFilePointer());
           }
-          ForUtil.readBlock(payIn, encoded, offsetStartDeltaBuffer);
-          ForUtil.readBlock(payIn, encoded, offsetLengthBuffer);
+          forUtil.readBlock(payIn, encoded, offsetStartDeltaBuffer);
+          forUtil.readBlock(payIn, encoded, offsetLengthBuffer);
         }
       }
     }
@@ -1355,11 +1359,11 @@ public final class BlockPackedPostingsRe
             System.out.println("        skip whole block @ fp=" + posIn.getFilePointer());
           }
           assert posIn.getFilePointer() != lastPosBlockFP;
-          ForUtil.skipBlock(posIn);
+          forUtil.skipBlock(posIn);
 
           if (indexHasPayloads) {
             // Skip payloadLength block:
-            ForUtil.skipBlock(payIn);
+            forUtil.skipBlock(payIn);
 
             // Skip payloadBytes block:
             int numBytes = payIn.readVInt();
@@ -1369,8 +1373,8 @@ public final class BlockPackedPostingsRe
           if (indexHasOffsets) {
             // Must load offset blocks merely to sum
             // up into lastStartOffset:
-            ForUtil.readBlock(payIn, encoded, offsetStartDeltaBuffer);
-            ForUtil.readBlock(payIn, encoded, offsetLengthBuffer);
+            forUtil.readBlock(payIn, encoded, offsetStartDeltaBuffer);
+            forUtil.readBlock(payIn, encoded, offsetLengthBuffer);
             for(int i=0;i<BLOCK_SIZE;i++) {
               lastStartOffset += offsetStartDeltaBuffer[i] + offsetLengthBuffer[i];
             }

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java?rev=1370781&r1=1370780&r2=1370781&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsWriter.java Wed Aug  8 14:15:22 2012
@@ -18,6 +18,9 @@ package org.apache.lucene.codecs.blockpa
  */
 
 import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsReader.DEBUG;
+import static org.apache.lucene.codecs.blockpacked.ForUtil.MIN_DATA_SIZE;
+import static org.apache.lucene.codecs.blockpacked.ForUtil.MIN_ENCODED_SIZE;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -36,6 +39,7 @@ import org.apache.lucene.store.RAMOutput
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
 
 
 /**
@@ -49,8 +53,6 @@ import org.apache.lucene.util.IOUtils;
  */
 public final class BlockPackedPostingsWriter extends PostingsWriterBase {
 
-  private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
-
   // nocommit move these constants to the PF:
 
   static final int maxSkipLevels = 10;
@@ -108,9 +110,10 @@ public final class BlockPackedPostingsWr
 
   final byte[] encoded;
 
+  private final ForUtil forUtil;
   private final BlockPackedSkipWriter skipWriter;
   
-  public BlockPackedPostingsWriter(SegmentWriteState state) throws IOException {
+  public BlockPackedPostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
     super();
 
     docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPackedPostingsFormat.DOC_EXTENSION),
@@ -120,23 +123,24 @@ public final class BlockPackedPostingsWr
     boolean success = false;
     try {
       CodecUtil.writeHeader(docOut, DOC_CODEC, VERSION_CURRENT);
+      forUtil = new ForUtil(acceptableOverheadRatio, docOut);
       if (state.fieldInfos.hasProx()) {
-        posDeltaBuffer = new long[BLOCK_SIZE];
+        posDeltaBuffer = new long[MIN_DATA_SIZE];
         posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPackedPostingsFormat.POS_EXTENSION),
                                               state.context);
         CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
 
         if (state.fieldInfos.hasPayloads()) {
           payloadBytes = new byte[128];
-          payloadLengthBuffer = new long[BLOCK_SIZE];
+          payloadLengthBuffer = new long[MIN_DATA_SIZE];
         } else {
           payloadBytes = null;
           payloadLengthBuffer = null;
         }
 
         if (state.fieldInfos.hasOffsets()) {
-          offsetStartDeltaBuffer = new long[BLOCK_SIZE];
-          offsetLengthBuffer = new long[BLOCK_SIZE];
+          offsetStartDeltaBuffer = new long[MIN_DATA_SIZE];
+          offsetLengthBuffer = new long[MIN_DATA_SIZE];
         } else {
           offsetStartDeltaBuffer = null;
           offsetLengthBuffer = null;
@@ -163,8 +167,8 @@ public final class BlockPackedPostingsWr
       }
     }
 
-    docDeltaBuffer = new long[BLOCK_SIZE];
-    freqBuffer = new long[BLOCK_SIZE];
+    docDeltaBuffer = new long[MIN_DATA_SIZE];
+    freqBuffer = new long[MIN_DATA_SIZE];
 
     skipWriter = new BlockPackedSkipWriter(maxSkipLevels,
                                      BlockPackedPostingsFormat.BLOCK_SIZE, 
@@ -173,7 +177,11 @@ public final class BlockPackedPostingsWr
                                      posOut,
                                      payOut);
 
-    encoded = new byte[BLOCK_SIZE*4];
+    encoded = new byte[MIN_ENCODED_SIZE];
+  }
+
+  public BlockPackedPostingsWriter(SegmentWriteState state) throws IOException {
+    this(state, PackedInts.DEFAULT);
   }
 
   @Override
@@ -236,12 +244,12 @@ public final class BlockPackedPostingsWr
       if (DEBUG) {
         System.out.println("  write docDelta block @ fp=" + docOut.getFilePointer());
       }
-      ForUtil.writeBlock(docDeltaBuffer, encoded, docOut);
+      forUtil.writeBlock(docDeltaBuffer, encoded, docOut);
       if (fieldHasFreqs) {
         if (DEBUG) {
           System.out.println("  write freq block @ fp=" + docOut.getFilePointer());
         }
-        ForUtil.writeBlock(freqBuffer, encoded, docOut);
+        forUtil.writeBlock(freqBuffer, encoded, docOut);
       }
       // NOTE: don't set docBufferUpto back to 0 here;
       // finishDoc will do so (because it needs to see that
@@ -288,17 +296,17 @@ public final class BlockPackedPostingsWr
       if (DEBUG) {
         System.out.println("  write pos bulk block @ fp=" + posOut.getFilePointer());
       }
-      ForUtil.writeBlock(posDeltaBuffer, encoded, posOut);
+      forUtil.writeBlock(posDeltaBuffer, encoded, posOut);
 
       if (fieldHasPayloads) {
-        ForUtil.writeBlock(payloadLengthBuffer, encoded, payOut);
+        forUtil.writeBlock(payloadLengthBuffer, encoded, payOut);
         payOut.writeVInt(payloadByteUpto);
         payOut.writeBytes(payloadBytes, 0, payloadByteUpto);
         payloadByteUpto = 0;
       }
       if (fieldHasOffsets) {
-        ForUtil.writeBlock(offsetStartDeltaBuffer, encoded, payOut);
-        ForUtil.writeBlock(offsetLengthBuffer, encoded, payOut);
+        forUtil.writeBlock(offsetStartDeltaBuffer, encoded, payOut);
+        forUtil.writeBlock(offsetLengthBuffer, encoded, payOut);
       }
       posBufferUpto = 0;
     }

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java?rev=1370781&r1=1370780&r2=1370781&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java Wed Aug  8 14:15:22 2012
@@ -21,36 +21,122 @@ import static org.apache.lucene.codecs.b
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
 
 /**
  * Encode all values in normal area with fixed bit width, 
  * which is determined by the max value in this block.
  */
-public class ForUtil {
+final class ForUtil {
 
   /**
    * Special number of bits per value used whenever all values to encode are equal.
    */
   private static final int ALL_VALUES_EQUAL = 0;
+  private static final int PACKED_INTS_VERSION = 0; // nocommit: encode in the stream?
 
-  static final int PACKED_INTS_VERSION = 0; // nocommit: encode in the stream?
-  static final PackedInts.Encoder[] ENCODERS = new PackedInts.Encoder[33];
-  static final PackedInts.Decoder[] DECODERS = new PackedInts.Decoder[33];
-  static final int[] ITERATIONS = new int[33];
+  /**
+   * Minimum length of the buffer that holds encoded bytes.
+   */
+  static final int MIN_ENCODED_SIZE = BLOCK_SIZE * 4;
+
+  /**
+   * Minimum length of the buffer that holds data.
+   */
+  static final int MIN_DATA_SIZE;
   static {
-    for (int i = 1; i <= 32; ++i) {
-      ENCODERS[i] = PackedInts.getEncoder(
-          PackedInts.Format.PACKED, PACKED_INTS_VERSION, i);
-      DECODERS[i] = PackedInts.getDecoder(
-          PackedInts.Format.PACKED, PACKED_INTS_VERSION, i);
-      ITERATIONS[i] = BLOCK_SIZE / DECODERS[i].valueCount();
+    int minDataSize = 0;
+    for (PackedInts.Format format : PackedInts.Format.values()) {
+      for (int bpv = 1; bpv <= 32; ++bpv) {
+        if (!format.isSupported(bpv)) {
+          continue;
+        }
+        final PackedInts.Decoder decoder = PackedInts.getDecoder(format, PACKED_INTS_VERSION, bpv);
+        final int iterations = (int) Math.ceil((float) BLOCK_SIZE / decoder.valueCount());
+        minDataSize = Math.max(minDataSize, iterations * decoder.valueCount());
+      }
+    }
+    MIN_DATA_SIZE = minDataSize;
+  }
+
+  private static int computeIterations(PackedInts.Decoder decoder) {
+    return (int) Math.ceil((float) BLOCK_SIZE / decoder.valueCount());
+  }
+
+  private final PackedInts.FormatAndBits[] formats;
+  private final PackedInts.Encoder[] encoders;
+  private final PackedInts.Decoder[] decoders;
+  private final int[] iterations;
+
+  /**
+   * Create a new {@link ForUtil} instance and save state into <code>out</code>.
+   */
+  ForUtil(float acceptableOverheadRatio, DataOutput out) throws IOException {
+    formats = new PackedInts.FormatAndBits[33];
+    encoders = new PackedInts.Encoder[33];
+    decoders = new PackedInts.Decoder[33];
+    iterations = new int[33];
+
+    for (int bpv = 1; bpv <= 32; ++bpv) {
+      final FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(
+          BLOCK_SIZE, bpv, acceptableOverheadRatio);
+      assert formatAndBits.format.isSupported(formatAndBits.bitsPerValue);
+      assert formatAndBits.bitsPerValue <= 32;
+      formats[bpv] = formatAndBits;
+      encoders[bpv] = PackedInts.getEncoder(
+          formatAndBits.format, PACKED_INTS_VERSION, formatAndBits.bitsPerValue);
+      decoders[bpv] = PackedInts.getDecoder(
+          formatAndBits.format, PACKED_INTS_VERSION, formatAndBits.bitsPerValue);
+      iterations[bpv] = computeIterations(decoders[bpv]);
+
+      out.writeVInt(formatAndBits.format.getId() << 5 | (formatAndBits.bitsPerValue - 1));
+    }
+  }
+
+  /**
+   * Restore a {@link ForUtil} from a {@link DataInput}.
+   */
+  ForUtil(DataInput in) throws IOException {
+    formats = new PackedInts.FormatAndBits[33];
+    encoders = new PackedInts.Encoder[33];
+    decoders = new PackedInts.Decoder[33];
+    iterations = new int[33];
+
+    for (int bpv = 1; bpv <= 32; ++bpv) {
+      final int code = in.readVInt();
+      final int formatId = code >>> 5;
+      final int bitsPerValue = (code & 31) + 1;
+
+      final PackedInts.Format format = PackedInts.Format.byId(formatId);
+      assert format.isSupported(bitsPerValue);
+      formats[bpv] = new PackedInts.FormatAndBits(format, bitsPerValue);
+      encoders[bpv] = PackedInts.getEncoder(
+          format, PACKED_INTS_VERSION, bitsPerValue);
+      decoders[bpv] = PackedInts.getDecoder(
+          format, PACKED_INTS_VERSION, bitsPerValue);
+      iterations[bpv] = computeIterations(decoders[bpv]);
     }
   }
 
   /**
+   * Compute the minimum size of the buffer that holds values. This method exists
+   * because {@link Decoder}s cannot decode less than a given amount of blocks
+   * at a time.
+   */
+  int getMinRequiredBufferSize() {
+    int minSize = 0;
+    for (int bpv = 1; bpv <= 32; ++bpv) {
+      minSize = Math.max(minSize, iterations[bpv] * decoders[bpv].valueCount());
+    }
+    return minSize;
+  }
+
+  /**
    * Write a block of data (<code>For</code> format).
    *
    * @param data     the data to write
@@ -58,7 +144,7 @@ public class ForUtil {
    * @param out      the destination output
    * @throws IOException
    */
-  static void writeBlock(long[] data, byte[] encoded, IndexOutput out) throws IOException {
+  void writeBlock(long[] data, byte[] encoded, IndexOutput out) throws IOException {
     if (isAllEqual(data)) {
       out.writeVInt(ALL_VALUES_EQUAL);
       out.writeInt((int) data[0]);
@@ -67,10 +153,11 @@ public class ForUtil {
 
     final int numBits = bitsRequired(data);
     assert numBits > 0 && numBits <= 32 : numBits;
-    final PackedInts.Encoder encoder = ENCODERS[numBits];
-    final int iters = ITERATIONS[numBits];
-    assert iters * encoder.valueCount() == BlockPackedPostingsFormat.BLOCK_SIZE;
+    final PackedInts.Encoder encoder = encoders[numBits];
+    final int iters = iterations[numBits];
+    assert iters * encoder.valueCount() >= BLOCK_SIZE;
     final int encodedSize = encodedSize(numBits);
+    assert (iters * encoder.blockCount()) << 3 >= encodedSize;
 
     out.writeVInt(numBits);
 
@@ -86,23 +173,22 @@ public class ForUtil {
    * @param decoded   where to write decoded data
    * @throws IOException
    */
-  static void readBlock(IndexInput in, byte[] encoded, long[] decoded) throws IOException {
+  void readBlock(IndexInput in, byte[] encoded, long[] decoded) throws IOException {
     final int numBits = in.readVInt();
     assert numBits <= 32 : numBits;
 
     if (numBits == ALL_VALUES_EQUAL) {
       final int value = in.readInt();
-      Arrays.fill(decoded, value);
+      Arrays.fill(decoded, 0, BLOCK_SIZE, value);
       return;
     }
 
     final int encodedSize = encodedSize(numBits);
     in.readBytes(encoded, 0, encodedSize);
 
-    final PackedInts.Decoder decoder = DECODERS[numBits];
-    final int iters = ITERATIONS[numBits];
-    assert iters * decoder.valueCount() == BLOCK_SIZE;
-    assert 8 * iters * decoder.blockCount() == encodedSize;
+    final PackedInts.Decoder decoder = decoders[numBits];
+    final int iters = iterations[numBits];
+    assert iters * decoder.valueCount() >= BLOCK_SIZE;
 
     decoder.decode(encoded, 0, decoded, 0, iters);
   }
@@ -113,8 +199,12 @@ public class ForUtil {
    * @param in      the input where to read data
    * @throws IOException
    */
-  static void skipBlock(IndexInput in) throws IOException {
+  void skipBlock(IndexInput in) throws IOException {
     final int numBits = in.readVInt();
+    if (numBits == ALL_VALUES_EQUAL) {
+      in.seek(in.getFilePointer() + 4);
+      return;
+    }
     assert numBits > 0 && numBits <= 32 : numBits;
     final int encodedSize = encodedSize(numBits);
     in.seek(in.getFilePointer() + encodedSize);
@@ -145,7 +235,7 @@ public class ForUtil {
   // nocommit: we must have a util function for this, hmm?
   private static boolean isAllEqual(final long[] data) {
     final long v = data[0];
-    for (int i = 1; i < data.length; ++i) {
+    for (int i = 1; i < BLOCK_SIZE; ++i) {
       if (data[i] != v) {
         return false;
       }
@@ -159,7 +249,7 @@ public class ForUtil {
    */
   private static int bitsRequired(final long[] data) {
     long or = 0;
-    for (int i = 0; i < data.length; ++i) {
+    for (int i = 0; i < BLOCK_SIZE; ++i) {
       or |= data[i];
     }
     return PackedInts.bitsRequired(or);
@@ -169,8 +259,9 @@ public class ForUtil {
    * Compute the number of bytes required to encode a block of values that require
    * <code>bitsPerValue</code> bits per value.
    */
-  private static int encodedSize(int bitsPerValue) {
-    return (BLOCK_SIZE * bitsPerValue) >>> 3;
+  private int encodedSize(int bitsPerValue) {
+    final FormatAndBits formatAndBits = formats[bitsPerValue];
+    return formatAndBits.format.nblocks(formatAndBits.bitsPerValue, BLOCK_SIZE) << 3;
   }
 
 }

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/BulkOperation.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/BulkOperation.java?rev=1370781&r1=1370780&r2=1370781&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/BulkOperation.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/BulkOperation.java Wed Aug  8 14:15:22 2012
@@ -21567,23 +21567,23 @@ enum BulkOperation implements PackedInts
         final int byte0 = blocks[blocksOffset++] & 0xFF;
         values[valuesOffset++] = byte0 & 7;
         values[valuesOffset++] = (byte0 >>> 3) & 7;
-        values[valuesOffset++] = (byte0 >>> 6) | ((byte1 & 1) << 10);
+        values[valuesOffset++] = (byte0 >>> 6) | ((byte1 & 1) << 2);
         values[valuesOffset++] = (byte1 >>> 1) & 7;
         values[valuesOffset++] = (byte1 >>> 4) & 7;
-        values[valuesOffset++] = (byte1 >>> 7) | ((byte2 & 3) << 9);
+        values[valuesOffset++] = (byte1 >>> 7) | ((byte2 & 3) << 1);
         values[valuesOffset++] = (byte2 >>> 2) & 7;
         values[valuesOffset++] = byte2 >>> 5;
         values[valuesOffset++] = byte3 & 7;
         values[valuesOffset++] = (byte3 >>> 3) & 7;
-        values[valuesOffset++] = (byte3 >>> 6) | ((byte4 & 1) << 10);
+        values[valuesOffset++] = (byte3 >>> 6) | ((byte4 & 1) << 2);
         values[valuesOffset++] = (byte4 >>> 1) & 7;
         values[valuesOffset++] = (byte4 >>> 4) & 7;
-        values[valuesOffset++] = (byte4 >>> 7) | ((byte5 & 3) << 9);
+        values[valuesOffset++] = (byte4 >>> 7) | ((byte5 & 3) << 1);
         values[valuesOffset++] = (byte5 >>> 2) & 7;
         values[valuesOffset++] = byte5 >>> 5;
         values[valuesOffset++] = byte6 & 7;
         values[valuesOffset++] = (byte6 >>> 3) & 7;
-        values[valuesOffset++] = (byte6 >>> 6) | ((byte7 & 1) << 10);
+        values[valuesOffset++] = (byte6 >>> 6) | ((byte7 & 1) << 2);
         values[valuesOffset++] = (byte7 >>> 1) & 7;
         values[valuesOffset++] = (byte7 >>> 4) & 7;
       }
@@ -21717,17 +21717,17 @@ enum BulkOperation implements PackedInts
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
         values[valuesOffset++] = byte0 & 31;
-        values[valuesOffset++] = (byte0 >>> 5) | ((byte1 & 3) << 11);
+        values[valuesOffset++] = (byte0 >>> 5) | ((byte1 & 3) << 3);
         values[valuesOffset++] = (byte1 >>> 2) & 31;
-        values[valuesOffset++] = (byte1 >>> 7) | ((byte2 & 15) << 9);
-        values[valuesOffset++] = (byte2 >>> 4) | ((byte3 & 1) << 12);
+        values[valuesOffset++] = (byte1 >>> 7) | ((byte2 & 15) << 1);
+        values[valuesOffset++] = (byte2 >>> 4) | ((byte3 & 1) << 4);
         values[valuesOffset++] = (byte3 >>> 1) & 31;
-        values[valuesOffset++] = (byte3 >>> 6) | ((byte4 & 7) << 10);
+        values[valuesOffset++] = (byte3 >>> 6) | ((byte4 & 7) << 2);
         values[valuesOffset++] = byte4 >>> 3;
         values[valuesOffset++] = byte5 & 31;
-        values[valuesOffset++] = (byte5 >>> 5) | ((byte6 & 3) << 11);
+        values[valuesOffset++] = (byte5 >>> 5) | ((byte6 & 3) << 3);
         values[valuesOffset++] = (byte6 >>> 2) & 31;
-        values[valuesOffset++] = (byte6 >>> 7) | ((byte7 & 15) << 9);
+        values[valuesOffset++] = (byte6 >>> 7) | ((byte7 & 15) << 1);
       }
     }
 
@@ -21782,15 +21782,15 @@ enum BulkOperation implements PackedInts
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
         values[valuesOffset++] = byte0 & 63;
-        values[valuesOffset++] = (byte0 >>> 6) | ((byte1 & 15) << 10);
-        values[valuesOffset++] = (byte1 >>> 4) | ((byte2 & 3) << 12);
+        values[valuesOffset++] = (byte0 >>> 6) | ((byte1 & 15) << 2);
+        values[valuesOffset++] = (byte1 >>> 4) | ((byte2 & 3) << 4);
         values[valuesOffset++] = byte2 >>> 2;
         values[valuesOffset++] = byte3 & 63;
-        values[valuesOffset++] = (byte3 >>> 6) | ((byte4 & 15) << 10);
-        values[valuesOffset++] = (byte4 >>> 4) | ((byte5 & 3) << 12);
+        values[valuesOffset++] = (byte3 >>> 6) | ((byte4 & 15) << 2);
+        values[valuesOffset++] = (byte4 >>> 4) | ((byte5 & 3) << 4);
         values[valuesOffset++] = byte5 >>> 2;
         values[valuesOffset++] = byte6 & 63;
-        values[valuesOffset++] = (byte6 >>> 6) | ((byte7 & 15) << 10);
+        values[valuesOffset++] = (byte6 >>> 6) | ((byte7 & 15) << 2);
       }
     }
 
@@ -21844,12 +21844,12 @@ enum BulkOperation implements PackedInts
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
         values[valuesOffset++] = byte0 & 127;
-        values[valuesOffset++] = (byte0 >>> 7) | ((byte1 & 63) << 9);
-        values[valuesOffset++] = (byte1 >>> 6) | ((byte2 & 31) << 10);
-        values[valuesOffset++] = (byte2 >>> 5) | ((byte3 & 15) << 11);
-        values[valuesOffset++] = (byte3 >>> 4) | ((byte4 & 7) << 12);
-        values[valuesOffset++] = (byte4 >>> 3) | ((byte5 & 3) << 13);
-        values[valuesOffset++] = (byte5 >>> 2) | ((byte6 & 1) << 14);
+        values[valuesOffset++] = (byte0 >>> 7) | ((byte1 & 63) << 1);
+        values[valuesOffset++] = (byte1 >>> 6) | ((byte2 & 31) << 2);
+        values[valuesOffset++] = (byte2 >>> 5) | ((byte3 & 15) << 3);
+        values[valuesOffset++] = (byte3 >>> 4) | ((byte4 & 7) << 4);
+        values[valuesOffset++] = (byte4 >>> 3) | ((byte5 & 3) << 5);
+        values[valuesOffset++] = (byte5 >>> 2) | ((byte6 & 1) << 6);
         values[valuesOffset++] = byte6 >>> 1;
         values[valuesOffset++] = byte7 & 127;
       }
@@ -21961,13 +21961,13 @@ enum BulkOperation implements PackedInts
         final int byte2 = blocks[blocksOffset++] & 0xFF;
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
-        values[valuesOffset++] = byte0 | ((byte1 & 1) << 16);
-        values[valuesOffset++] = (byte1 >>> 1) | ((byte2 & 3) << 15);
-        values[valuesOffset++] = (byte2 >>> 2) | ((byte3 & 7) << 14);
-        values[valuesOffset++] = (byte3 >>> 3) | ((byte4 & 15) << 13);
-        values[valuesOffset++] = (byte4 >>> 4) | ((byte5 & 31) << 12);
-        values[valuesOffset++] = (byte5 >>> 5) | ((byte6 & 63) << 11);
-        values[valuesOffset++] = (byte6 >>> 6) | ((byte7 & 127) << 10);
+        values[valuesOffset++] = byte0 | ((byte1 & 1) << 8);
+        values[valuesOffset++] = (byte1 >>> 1) | ((byte2 & 3) << 7);
+        values[valuesOffset++] = (byte2 >>> 2) | ((byte3 & 7) << 6);
+        values[valuesOffset++] = (byte3 >>> 3) | ((byte4 & 15) << 5);
+        values[valuesOffset++] = (byte4 >>> 4) | ((byte5 & 31) << 4);
+        values[valuesOffset++] = (byte5 >>> 5) | ((byte6 & 63) << 3);
+        values[valuesOffset++] = (byte6 >>> 6) | ((byte7 & 127) << 2);
       }
     }
 
@@ -22017,12 +22017,12 @@ enum BulkOperation implements PackedInts
         final int byte2 = blocks[blocksOffset++] & 0xFF;
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
-        values[valuesOffset++] = byte0 | ((byte1 & 3) << 16);
-        values[valuesOffset++] = (byte1 >>> 2) | ((byte2 & 15) << 14);
-        values[valuesOffset++] = (byte2 >>> 4) | ((byte3 & 63) << 12);
-        values[valuesOffset++] = (byte3 >>> 6) | (byte4 << 8);
-        values[valuesOffset++] = byte5 | ((byte6 & 3) << 16);
-        values[valuesOffset++] = (byte6 >>> 2) | ((byte7 & 15) << 14);
+        values[valuesOffset++] = byte0 | ((byte1 & 3) << 8);
+        values[valuesOffset++] = (byte1 >>> 2) | ((byte2 & 15) << 6);
+        values[valuesOffset++] = (byte2 >>> 4) | ((byte3 & 63) << 4);
+        values[valuesOffset++] = (byte3 >>> 6) | (byte4 << 2);
+        values[valuesOffset++] = byte5 | ((byte6 & 3) << 8);
+        values[valuesOffset++] = (byte6 >>> 2) | ((byte7 & 15) << 6);
       }
     }
 
@@ -22071,11 +22071,11 @@ enum BulkOperation implements PackedInts
         final int byte2 = blocks[blocksOffset++] & 0xFF;
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
-        values[valuesOffset++] = byte0 | ((byte1 & 15) << 16);
-        values[valuesOffset++] = (byte1 >>> 4) | (byte2 << 8);
-        values[valuesOffset++] = byte3 | ((byte4 & 15) << 16);
-        values[valuesOffset++] = (byte4 >>> 4) | (byte5 << 8);
-        values[valuesOffset++] = byte6 | ((byte7 & 15) << 16);
+        values[valuesOffset++] = byte0 | ((byte1 & 15) << 8);
+        values[valuesOffset++] = (byte1 >>> 4) | (byte2 << 4);
+        values[valuesOffset++] = byte3 | ((byte4 & 15) << 8);
+        values[valuesOffset++] = (byte4 >>> 4) | (byte5 << 4);
+        values[valuesOffset++] = byte6 | ((byte7 & 15) << 8);
       }
     }
 
@@ -22173,9 +22173,9 @@ enum BulkOperation implements PackedInts
         final int byte2 = blocks[blocksOffset++] & 0xFF;
         final int byte1 = blocks[blocksOffset++] & 0xFF;
         final int byte0 = blocks[blocksOffset++] & 0xFF;
-        values[valuesOffset++] = byte0 | (byte1 << 8) | ((byte2 & 31) << 32);
-        values[valuesOffset++] = (byte2 >>> 5) | (byte3 << 8) | (byte4 << 16) | ((byte5 & 3) << 43);
-        values[valuesOffset++] = (byte5 >>> 2) | (byte6 << 8) | ((byte7 & 127) << 30);
+        values[valuesOffset++] = byte0 | (byte1 << 8) | ((byte2 & 31) << 16);
+        values[valuesOffset++] = (byte2 >>> 5) | (byte3 << 3) | (byte4 << 11) | ((byte5 & 3) << 19);
+        values[valuesOffset++] = (byte5 >>> 2) | (byte6 << 6) | ((byte7 & 127) << 14);
       }
     }
 

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=1370781&r1=1370780&r2=1370781&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 Wed Aug  8 14:15:22 2012
@@ -208,11 +208,11 @@ def p64sb_decode(bpv, bits):
       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)))
+        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)))
+        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) + bpv - bit_end - 1))
+        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")

Added: lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/TestForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/TestForUtil.java?rev=1370781&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/TestForUtil.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/blockpacked/TestForUtil.java Wed Aug  8 14:15:22 2012
@@ -0,0 +1,94 @@
+package org.apache.lucene.codecs.blockpacked;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+import static org.apache.lucene.codecs.blockpacked.ForUtil.MIN_DATA_SIZE;
+import static org.apache.lucene.codecs.blockpacked.ForUtil.MIN_ENCODED_SIZE;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+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.LuceneTestCase;
+import org.apache.lucene.util.packed.PackedInts;
+
+import com.carrotsearch.randomizedtesting.generators.RandomInts;
+
+public class TestForUtil extends LuceneTestCase {
+
+  public void testEncodeDecode() throws IOException {
+    final int iterations = RandomInts.randomIntBetween(random(), 1, 1000);
+    final float acceptableOverheadRatio = random().nextFloat();
+    final long[] values = new long[iterations * BLOCK_SIZE + ForUtil.MIN_DATA_SIZE];
+    for (int i = 0; i < iterations; ++i) {
+      final int bpv = random().nextInt(32);
+      if (bpv == 0) {
+        final int value = RandomInts.randomIntBetween(random(), 0, Integer.MAX_VALUE);
+        for (int j = 0; j < BLOCK_SIZE; ++j) {
+          values[i * BLOCK_SIZE + j] = value;
+        }
+      } else {
+        for (int j = 0; j < BLOCK_SIZE; ++j) {
+          values[i * BLOCK_SIZE + j] = RandomInts.randomIntBetween(random(),
+              0, (int) PackedInts.maxValue(bpv));
+        }
+      }
+    }
+
+    final Directory d = new RAMDirectory();
+    final long endPointer;
+
+    {
+      // encode
+      IndexOutput out = d.createOutput("test.bin", IOContext.DEFAULT);
+      final ForUtil forUtil = new ForUtil(acceptableOverheadRatio, out);
+      
+      for (int i = 0; i < iterations; ++i) {
+        forUtil.writeBlock(
+            Arrays.copyOfRange(values, iterations * BLOCK_SIZE, values.length),
+            new byte[MIN_ENCODED_SIZE], out);
+      }
+      endPointer = out.getFilePointer();
+      out.close();
+    }
+
+    {
+      // decode
+      IndexInput in = d.openInput("test.bin", IOContext.READONCE);
+      final ForUtil forUtil = new ForUtil(in);
+      for (int i = 0; i < iterations; ++i) {
+        if (random().nextBoolean()) {
+          forUtil.skipBlock(in);
+          continue;
+        }
+        final long[] restored = new long[MIN_DATA_SIZE];
+        forUtil.readBlock(in, new byte[MIN_ENCODED_SIZE], restored);
+        assertArrayEquals(Arrays.copyOfRange(values, iterations * BLOCK_SIZE, (iterations + 1) * BLOCK_SIZE),
+            Arrays.copyOf(restored, BLOCK_SIZE));
+      }
+      assertEquals(endPointer, in.getFilePointer());
+      in.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=1370781&r1=1370780&r2=1370781&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 Wed Aug  8 14:15:22 2012
@@ -656,7 +656,7 @@ public class TestPackedInts extends Luce
           blocks[i] = random().nextLong();
           if (format == PackedInts.Format.PACKED_SINGLE_BLOCK && 64 % bpv != 0) {
             // clear highest bits for packed
-            final int toClear = 64 - 64 % bpv;
+            final int toClear = 64 % bpv;
             blocks[i] = (blocks[i] << toClear) >>> toClear;
           }
         }
@@ -664,6 +664,9 @@ public class TestPackedInts extends Luce
         // 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));
+        }
 
         // 3. re-encode
         final long[] blocks2 = new long[blocksOffset2 + blocksLen];
@@ -676,6 +679,9 @@ public class TestPackedInts extends Luce
         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);
 
         // 5. byte[] encoding