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

svn commit: r1360169 - in /lucene/dev/branches/pforcodec_3892/lucene/core/src: java/org/apache/lucene/codecs/pfor/ test/org/apache/lucene/codecs/pfor/

Author: mikemccand
Date: Wed Jul 11 13:29:11 2012
New Revision: 1360169

URL: http://svn.apache.org/viewvc?rev=1360169&view=rev
Log:
LUCENE-3892: docs/cleanup

Modified:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForFactory.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForPostingsFormat.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForUtil.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PackedIntsDecompress.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py
    lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestPForUtil.java

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,5 @@
 package org.apache.lucene.codecs.pfor;
-/**
+/*
  * 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.
@@ -32,12 +32,15 @@ import org.apache.lucene.codecs.intblock
 import org.apache.lucene.codecs.intblock.FixedIntBlockIndexOutput;
 
 /** 
- * Stuff to pass to PostingsReader/WriterBase.
- * Things really make sense are: flushBlock() and readBlock()
+ * Used to plug to PostingsReader/WriterBase.
+ * Encoder and decoder in lower layers are called by 
+ * flushBlock() and readBlock()
  */
 
 public final class ForFactory extends IntStreamFactory {
-  private final int blockSize;
+
+  /* number of ints for each block */
+  private final int blockSize; 
 
   public ForFactory() {
     this.blockSize = ForPostingsFormat.DEFAULT_BLOCK_SIZE;
@@ -53,6 +56,8 @@ public final class ForFactory extends In
       return ret;
     } finally {
       if (!success) {
+        // For some cases (e.g. disk full), the IntIndexOutput may not be 
+        // properly created. So we should close those opened files. 
         IOUtils.closeWhileHandlingException(out);
       }
     }
@@ -63,7 +68,10 @@ public final class ForFactory extends In
     return new ForIndexInput(dir.openInput(fileName, context));
   }
 
-  // wrap input and output with buffer support
+  /**
+   * Here we'll hold both input buffer and output buffer for 
+   * encoder/decoder.
+   */
   private class ForIndexInput extends FixedIntBlockIndexInput {
 
     ForIndexInput(final IndexInput in) throws IOException {
@@ -77,7 +85,10 @@ public final class ForFactory extends In
       private final IntBuffer encodedBuffer;
 
       ForBlockReader(final IndexInput in, final int[] buffer) {
-        this.encoded = new byte[blockSize*8+4];
+        // upperbound for encoded value should include:
+        // 1. blockSize of normal value when numFrameBits=32(4x bytes); 
+        // 2. header (4bytes);
+        this.encoded = new byte[blockSize*4+4]; 
         this.in = in;
         this.buffer = buffer;
         this.encodedBuffer = ByteBuffer.wrap(encoded).asIntBuffer();
@@ -87,7 +98,7 @@ public final class ForFactory extends In
       @Override
       public void readBlock() throws IOException {
         final int numBytes = in.readInt();
-        assert numBytes <= blockSize*8+4;
+        assert numBytes <= blockSize*4+4;
         in.readBytes(encoded,0,numBytes);
         ForUtil.decompress(encodedBuffer,buffer);
       }
@@ -105,7 +116,7 @@ public final class ForFactory extends In
 
     ForIndexOutput(IndexOutput out, int blockSize) throws IOException {
       super(out,blockSize);
-      this.encoded = new byte[blockSize*8+4];
+      this.encoded = new byte[blockSize*4+4];
       this.encodedBuffer=ByteBuffer.wrap(encoded).asIntBuffer();
     }
 

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,6 @@
 package org.apache.lucene.codecs.pfor;
-/**
+
+/*
  * 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.
@@ -37,10 +38,10 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
+
 /**
- * This class actually only pass the ForFactory
- * to a PostingsWriter/ReaderBase, and get customized
- * format plugged.
+ * Pass ForFactory to a PostingsWriter/ReaderBase, and get 
+ * customized postings format plugged.
  */
 public final class ForPostingsFormat extends PostingsFormat {
   private final int blockSize;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,5 @@
 package org.apache.lucene.codecs.pfor;
-/**
+/*
  * 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.
@@ -15,12 +15,15 @@ package org.apache.lucene.codecs.pfor;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-// nocommit: this is only a test verison, change from PForUtil.java
+
 import java.nio.IntBuffer;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
-// Encode all values in normal area, based on the bit size for max value
+/**
+ * Encode all values in normal area with fixed bit width, 
+ * which is determined by the max value in this block.
+ */
 public class ForUtil {
   public static final int HEADER_INT_SIZE=1;
   protected static final int[] MASK = {   0x00000000,
@@ -31,26 +34,52 @@ public class ForUtil {
     0x01ffffff, 0x03ffffff, 0x07ffffff, 0x0fffffff, 0x1fffffff, 0x3fffffff,
     0x7fffffff, 0xffffffff};
 
+  /** Compress given int[] into Integer buffer, with For format
+   *
+   * @param data        uncompressed data
+   * @param size        num of ints to compress
+   * @param intBuffer   integer buffer to hold compressed data
+   */
   public static int compress(final int[] data, int size, IntBuffer intBuffer) {
     int numBits=getNumBits(data,size);
   
     for (int i=0; i<size; ++i) {
       encodeNormalValue(intBuffer,i,data[i], numBits);
     }
-    // encode header
     encodeHeader(intBuffer, size, numBits);
 
     return (HEADER_INT_SIZE+(size*numBits+31)/32)*4;
   }
-  
+
+  /** Decompress given Integer buffer into int array.
+   *
+   * @param intBuffer   integer buffer to hold compressed data
+   * @param data        int array to hold uncompressed data
+   */
   public static int decompress(IntBuffer intBuffer, int[] data) {
+
+    // since this buffer is reused at upper level, rewind first
     intBuffer.rewind();
-    int header = intBuffer.get();
 
+    int header = intBuffer.get();
     int numInts = (header & MASK[8]) + 1;
     int numBits = ((header >> 8) & MASK[5]) + 1;
 
-    // TODO: PackedIntsDecompress is hardewired to size==128 only
+    decompressCore(intBuffer, data, numBits);
+
+    return numInts;
+  }
+
+  /**
+   * IntBuffer will not be rewinded in this method, therefore
+   * caller should ensure that the position is set to the first
+   * encoded int before decoding.
+   */
+  static void decompressCore(IntBuffer intBuffer, int[] data, int numBits) {
+    assert numBits<=32;
+    assert numBits>=1;
+
+    // TODO: PackedIntsDecompress is hardewired to size==129 only
     switch(numBits) {
       case 1: PackedIntsDecompress.decode1(intBuffer, data); break;
       case 2: PackedIntsDecompress.decode2(intBuffer, data); break;
@@ -84,10 +113,7 @@ public class ForUtil {
       case 30: PackedIntsDecompress.decode30(intBuffer, data); break;
       case 31: PackedIntsDecompress.decode31(intBuffer, data); break;
       case 32: PackedIntsDecompress.decode32(intBuffer, data); break;
-      default:
-        throw new IllegalStateException("Unknown numFrameBits " + numBits);
     }
-    return numInts;
   }
 
   static void encodeHeader(IntBuffer intBuffer, int numInts, int numBits) {
@@ -96,9 +122,9 @@ public class ForUtil {
   }
 
   static void encodeNormalValue(IntBuffer intBuffer, int pos, int value, int numBits) {
-    final int globalBitPos = numBits*pos;         // position in bit stream
-    final int localBitPos = globalBitPos & 31;    // position inside an int
-    int intPos = HEADER_INT_SIZE + globalBitPos/32;   // which integer to locate 
+    final int globalBitPos = numBits*pos;           // position in bit stream
+    final int localBitPos = globalBitPos & 31;      // position inside an int
+    int intPos = HEADER_INT_SIZE + globalBitPos/32; // which integer to locate 
     setBufferIntBits(intBuffer, intPos, localBitPos, numBits, value);
     if ((localBitPos + numBits) > 32) { // value does not fit in this int, fill tail
       setBufferIntBits(intBuffer, intPos+1, 0, 
@@ -115,7 +141,9 @@ public class ForUtil {
           | (value << firstBitPos));
   }
 
-  // TODO: shall we use 32 NumBits directly if it exceeds 28 bits?
+  /**
+   * Estimate best num of frame bits according to the largest value.
+   */
   static int getNumBits(final int[] data, int size) {
     int optBits=1;
     for (int i=0; i<size; ++i) {
@@ -125,12 +153,12 @@ public class ForUtil {
     }
     return optBits;
   }
-  /** The 4 byte header (32 bits) contains (from lsb to msb):
+
+  /** 
+   * Generate the 4 byte header, which contains (from lsb to msb):
    *
    * - 8 bits for uncompressed int num - 1 (use up to 7 bits i.e 128 actually)
-   *
    * - 5 bits for num of frame bits - 1
-   *
    * - other bits unused
    *
    */
@@ -138,38 +166,4 @@ public class ForUtil {
     return  (numInts-1)
           | ((numBits-1) << 8);
   }
-
-  static void println(String format, Object... args) {
-    System.out.println(String.format(format,args)); 
-  }
-  static void print(String format, Object... args) {
-    System.out.print(String.format(format,args)); 
-  }
-  static void eprintln(String format, Object... args) {
-    System.err.println(String.format(format,args)); 
-  }
-  public static String getHex( byte [] raw, int sz ) {
-    final String HEXES = "0123456789ABCDEF";
-    if ( raw == null ) return null;
-    final StringBuilder hex = new StringBuilder( 2 * raw.length );
-    for ( int i=0; i<sz; i++ ) {
-      if (i>0 && (i)%16 == 0)
-        hex.append("\n");
-      byte b=raw[i];
-      hex.append(HEXES.charAt((b & 0xF0) >> 4))
-         .append(HEXES.charAt((b & 0x0F)))
-         .append(" ");
-    }
-    return hex.toString();
-  }
-  public static String getHex( int [] raw, int sz ) {
-    if ( raw == null ) return null;
-    final StringBuilder hex = new StringBuilder( 4 * raw.length );
-    for ( int i=0; i<sz; i++ ) {
-      if (i>0 && i%8 == 0)
-        hex.append("\n");
-      hex.append(String.format("%08x ",raw[i]));
-    }
-    return hex.toString();
-  }
 }

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForFactory.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForFactory.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForFactory.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,5 @@
 package org.apache.lucene.codecs.pfor;
-/**
+/*
  * 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.
@@ -32,11 +32,14 @@ import org.apache.lucene.codecs.intblock
 import org.apache.lucene.codecs.intblock.FixedIntBlockIndexOutput;
 
 /** 
- * Stuff to pass to PostingsReader/WriterBase.
- * Things really make sense are: flushBlock() and readBlock()
+ * Used to plug to PostingsReader/WriterBase.
+ * Encoder and decoder in lower layers are called by 
+ * flushBlock() and readBlock()
  */
 
 public final class PForFactory extends IntStreamFactory {
+
+  /* number of ints for each block */
   private final int blockSize;
 
   public PForFactory() {
@@ -53,6 +56,8 @@ public final class PForFactory extends I
       return ret;
     } finally {
       if (!success) {
+        // For some cases (e.g. disk full), the IntIndexOutput may not be 
+        // properly created. So we should close those opened files. 
         IOUtils.closeWhileHandlingException(out);
       }
     }
@@ -63,7 +68,10 @@ public final class PForFactory extends I
     return new PForIndexInput(dir.openInput(fileName, context));
   }
 
-  // wrap input and output with buffer support
+  /**
+   * Here we'll hold both input buffer and output buffer for 
+   * encoder/decoder.
+   */
   private class PForIndexInput extends FixedIntBlockIndexInput {
 
     PForIndexInput(final IndexInput in) throws IOException {
@@ -77,6 +85,10 @@ public final class PForFactory extends I
       private final IntBuffer encodedBuffer;
 
       PForBlockReader(final IndexInput in, final int[] buffer) {
+        // upperbound for encoded value should include:
+        // 1. blockSize of normal value (4x bytes); 
+        // 2. blockSize of exception value (4x bytes);
+        // 3. header (4bytes);
         this.encoded = new byte[blockSize*8+4];
         this.in = in;
         this.buffer = buffer;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForPostingsFormat.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForPostingsFormat.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForPostingsFormat.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,6 @@
 package org.apache.lucene.codecs.pfor;
-/**
+
+/*
  * 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.
@@ -37,10 +38,10 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
+
 /**
- * This class actually only pass the PForFactory
- * to a PostingsWriter/ReaderBase, and get customized
- * format plugged.
+ * Pass PForFactory to a PostingsWriter/ReaderBase, and get 
+ * customized postings format plugged.
  */
 public final class PForPostingsFormat extends PostingsFormat {
   private final int blockSize;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForUtil.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PForUtil.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,5 @@
 package org.apache.lucene.codecs.pfor;
-/**
+/*
  * 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.
@@ -20,26 +20,45 @@ import java.nio.IntBuffer;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
-// Encode all small values and exception pointers in normal area, 
-// Encode large values in exception area.
-// Size per exception is variable, possibly: 1byte, 2bytes, or 4bytes
+/**
+ * Encode all small values and exception pointers in normal area;
+ * Encode large values in exception area;
+ * Size per exception is variable, possibly: 1byte, 2bytes, or 4bytes
+ */
 public final class PForUtil extends ForUtil {
+
   protected static final int[] PER_EXCEPTION_SIZE = {1,2,4};
 
+  /** Compress given int[] into Integer buffer, with PFor format
+   *
+   * @param data        uncompressed data
+   * @param size        num of ints to compress
+   * @param intBuffer   integer buffer to hold compressed data
+   */
   public static int compress(final int[] data, int size, IntBuffer intBuffer) {
+    /** estimate minimum compress size to determine numFrameBits */
     int numBits=getNumBits(data,size);
   
     int[] excValues = new int[size];
-    int excNum = 0, excLastPos = -1, excFirstPos = -1;
-    int excLastNonForcePos = -1; 
-    int excNumBase = 0;       // num of exception until the last non-force exception
-    int excBytes = 1;         // bytes per exception
-    int excByteOffset = 0;    // bytes of preceeding codes like header and normal area
-    long maxChain = (1<<8) - 2;  // header bits limits this to 254
+    int excNum = 0, excLastPos = -1, excFirstPos = -1, excLastNonForcePos = -1; 
+
+    // num of exception until the last non-forced exception
+    int excNumBase = 0;          
+
+    // bytes per exception
+    int excBytes = 1;
+
+    // bytes before exception area, e.g. header and normal area
+    int excByteOffset = 0;
+
+    // the max value possible for current exception pointer, 
+    // value of the first pointer is limited by header as 254
+    long maxChain = (1<<8) - 2;  
+
     boolean conValue, conForce, conEnd;
     int i=0;
 
-    // estimate exceptions
+    /** estimate exceptions */
     for (i=0; i<size; ++i) {
       conValue = ((data[i] & MASK[numBits]) != data[i]); // value exception
       conForce = (i >= maxChain + excLastPos);           // force exception
@@ -57,7 +76,7 @@ public final class PForUtil extends ForU
       }
     }
 
-    // encode normal area, record exception positions
+    /** encode normal area, record exception positions */
     i=0;
     excNum = 0;
     if (excFirstPos < 0) { // no exception 
@@ -90,7 +109,7 @@ public final class PForUtil extends ForU
       }
     }
   
-    // encode exception area
+    /** encode exception area */
     i=0;
     for (; i<excNum; ++i) {
       if (excBytes < 2 && (excValues[i] & ~MASK[8]) != 0) {
@@ -103,60 +122,33 @@ public final class PForUtil extends ForU
     excByteOffset = HEADER_INT_SIZE*4 + (size*numBits + 7)/8;
     encodeExcValues(intBuffer, excValues, excNum, excBytes, excByteOffset);
 
-    // encode header
+    /** encode header */
     encodeHeader(intBuffer, size, numBits, excNum, excFirstPos, excBytes);
 
     return (excByteOffset + excBytes*excNum + 3)/4*4;
   }
   
+  /** Decompress given Integer buffer into int array.
+   *
+   * @param intBuffer   integer buffer to hold compressed data
+   * @param data        int array to hold uncompressed data
+   */
   public static int decompress(IntBuffer intBuffer, int[] data) {
+
+    // since this buffer is reused at upper level, rewind first
     intBuffer.rewind();
-    int header = intBuffer.get();
 
+    int header = intBuffer.get();
     int numInts = (header & MASK[8]) + 1;
     int excNum = ((header >> 8) & MASK[8]) + 1;
     int excFirstPos = ((header >> 16) & MASK[8]) - 1;
     int excBytes = PER_EXCEPTION_SIZE[(header >> 29) & MASK[2]];
     int numBits = ((header >> 24) & MASK[5]) + 1;
 
-    // TODO: PackedIntsDecompress is hardewired to size==128 only
-    switch(numBits) {
-      case 1: PackedIntsDecompress.decode1(intBuffer, data); break;
-      case 2: PackedIntsDecompress.decode2(intBuffer, data); break;
-      case 3: PackedIntsDecompress.decode3(intBuffer, data); break;
-      case 4: PackedIntsDecompress.decode4(intBuffer, data); break;
-      case 5: PackedIntsDecompress.decode5(intBuffer, data); break;
-      case 6: PackedIntsDecompress.decode6(intBuffer, data); break;
-      case 7: PackedIntsDecompress.decode7(intBuffer, data); break;
-      case 8: PackedIntsDecompress.decode8(intBuffer, data); break;
-      case 9: PackedIntsDecompress.decode9(intBuffer, data); break;
-      case 10: PackedIntsDecompress.decode10(intBuffer, data); break;
-      case 11: PackedIntsDecompress.decode11(intBuffer, data); break;
-      case 12: PackedIntsDecompress.decode12(intBuffer, data); break;
-      case 13: PackedIntsDecompress.decode13(intBuffer, data); break;
-      case 14: PackedIntsDecompress.decode14(intBuffer, data); break;
-      case 15: PackedIntsDecompress.decode15(intBuffer, data); break;
-      case 16: PackedIntsDecompress.decode16(intBuffer, data); break;
-      case 17: PackedIntsDecompress.decode17(intBuffer, data); break;
-      case 18: PackedIntsDecompress.decode18(intBuffer, data); break;
-      case 19: PackedIntsDecompress.decode19(intBuffer, data); break;
-      case 20: PackedIntsDecompress.decode20(intBuffer, data); break;
-      case 21: PackedIntsDecompress.decode21(intBuffer, data); break;
-      case 22: PackedIntsDecompress.decode22(intBuffer, data); break;
-      case 23: PackedIntsDecompress.decode23(intBuffer, data); break;
-      case 24: PackedIntsDecompress.decode24(intBuffer, data); break;
-      case 25: PackedIntsDecompress.decode25(intBuffer, data); break;
-      case 26: PackedIntsDecompress.decode26(intBuffer, data); break;
-      case 27: PackedIntsDecompress.decode27(intBuffer, data); break;
-      case 28: PackedIntsDecompress.decode28(intBuffer, data); break;
-      case 29: PackedIntsDecompress.decode29(intBuffer, data); break;
-      case 30: PackedIntsDecompress.decode30(intBuffer, data); break;
-      case 31: PackedIntsDecompress.decode31(intBuffer, data); break;
-      case 32: PackedIntsDecompress.decode32(intBuffer, data); break;
-      default:
-        throw new IllegalStateException("Unknown numFrameBits " + numBits);
-    }
+    decompressCore(intBuffer, data, numBits);
+
     patchException(intBuffer,data,excNum,excFirstPos,excBytes);
+
     return numInts;
   }
 
@@ -196,9 +188,19 @@ public final class PForUtil extends ForU
     }
   }
 
-  // TODO: since numInts===128, we don't need to rewind intBuffer.
-  // however, tail of normal area may share a same int with head of exception area
-  // which means patchException may lose heading exceptions.
+  /**
+   * Decode exception values base on the exception pointers in normal area,
+   * and values in exception area.
+   * As for current implementation, numInts is hardwired as 128, so the
+   * tail of normal area is naturally aligned to 32 bits, and we don't need to
+   * rewind intBuffer here.
+   * However, the normal area may share a same int with exception area, 
+   * when numFrameBits * numInts % 32 != 0,
+   * In this case we should preprocess patch several heading exceptions, 
+   * before calling this method.
+   *
+   * TODO: blockSize is hardewired to size==128 only
+   */
   public static void patchException(IntBuffer intBuffer, int[] data, int excNum, int excFirstPos, int excBytes) {
     if (excFirstPos == -1) {
       return;
@@ -206,7 +208,7 @@ public final class PForUtil extends ForU
     int curPos=excFirstPos;
     int i,j;
 
-    if (excBytes == 1) {
+    if (excBytes == 1) { // each exception consumes 1 byte
       for (i=0; i+3<excNum; i+=4) {
         final int curInt = intBuffer.get();
         curPos = patch(data, curPos, (curInt) & MASK[8]);
@@ -214,13 +216,13 @@ public final class PForUtil extends ForU
         curPos = patch(data, curPos, (curInt >>> 16) & MASK[8]);
         curPos = patch(data, curPos, (curInt >>> 24) & MASK[8]);
       }
-      if (i<excNum) { 
+      if (i<excNum) {
         final int curInt = intBuffer.get();
         for (j=0; j<32 && i<excNum; j+=8,i++) {
           curPos = patch(data, curPos, (curInt >>> j) & MASK[8]);
         }
       }
-    } else if (excBytes == 2) {
+    } else if (excBytes == 2) { // each exception consumes 2 bytes
       for (i=0; i+1<excNum; i+=2) {
         final int curInt = intBuffer.get();
         curPos = patch(data, curPos, (curInt) & MASK[16]);
@@ -230,7 +232,7 @@ public final class PForUtil extends ForU
         final int curInt = intBuffer.get();
         curPos = patch(data, curPos, (curInt) & MASK[16]);
       }
-    } else if (excBytes == 4) {
+    } else if (excBytes == 4) { // each exception consumes 4 bytes
       for (i=0; i<excNum; i++) {
         curPos = patch(data, curPos, intBuffer.get());
       }
@@ -244,7 +246,10 @@ public final class PForUtil extends ForU
     return nextPos;
   }
 
-  // TODO: shall we use 32 NumBits directly if it exceeds 28 bits?
+  /**
+   * Estimate best number of frame bits according to minimum compressed size.
+   * It will run 32 times.
+   */
   static int getNumBits(final int[] data, int size) {
     int optBits=1;
     int optSize=estimateCompressedSize(data,size,1);
@@ -258,8 +263,11 @@ public final class PForUtil extends ForU
     return optBits;
   }
 
-  // loosely estimate int size of each compressed block, based on parameter b
-  // ignore force exceptions
+  /**
+   * Iterate the whole block to get maximum exception bits, 
+   * and estimate compressed size without forced exception.
+   * TODO: foresee forced exception for better estimation
+   */
   static int estimateCompressedSize(final int[] data, int size, int numBits) {
     int totalBytes=(numBits*size+7)/8;   // always round to byte
     int excNum=0;
@@ -285,17 +293,19 @@ public final class PForUtil extends ForU
 
     return totalBytes/4*4+HEADER_INT_SIZE;  // round up to ints
   }
-  /** The 4 byte header (32 bits) contains (from lsb to msb):
+
+  /** 
+   * Generate the 4 byte header, which contains (from lsb to msb):
    *
-   * - 8 bits for uncompressed int num - 1 (use up to 7 bits i.e 128 actually)
+   * 8 bits for uncompressed int num - 1 (use up to 7 bits i.e 128 actually)
    *
-   * - 8 bits for exception num - 1 (when no exceptions, this is undefined)
+   * 8 bits for exception num - 1 (when no exceptions, this is undefined)
    *
-   * - 8 bits for the index of the first exception + 1 (when no exception, this is 0)
+   * 8 bits for the index of the first exception + 1 (when no exception, this is 0)
    *
-   * - 5 bits for num of frame bits - 1
-   * - 2 bits for the exception code: 00: byte, 01: short, 10: int
-   * - 1 bit unused
+   * 5 bits for num of frame bits - 1
+   * 2 bits for the exception code: 00: byte, 01: short, 10: int
+   * 1 bit unused
    *
    */
   static int getHeader(int numInts, int numBits, int excNum, int excFirstPos, int excBytes) {

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PackedIntsDecompress.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PackedIntsDecompress.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PackedIntsDecompress.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/PackedIntsDecompress.java Wed Jul 11 13:29:11 2012
@@ -1,5 +1,5 @@
 package org.apache.lucene.codecs.pfor;
-/**
+/*
  * 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.
@@ -15,7 +15,8 @@ package org.apache.lucene.codecs.pfor;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
- /* This code is generated, do not modify. See gendecompress.py */
+ 
+/* This code is generated, do not modify. See gendecompress.py */
 
 import java.nio.IntBuffer;
 
@@ -24,6 +25,7 @@ final class PackedIntsDecompress {
   // nocommit: assess perf of this to see if specializing is really needed
 
   // NOTE: hardwired to blockSize == 128
+
   public static void decode1(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 1;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -65,8 +67,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode2(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 2;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -109,8 +109,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode3(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 3;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -154,8 +152,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode4(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 4;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -200,8 +196,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode5(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 5;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -247,8 +241,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode6(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 6;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -295,8 +287,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode7(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 7;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -344,8 +334,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode8(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 8;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -394,8 +382,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode9(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 9;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -445,8 +431,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode10(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 10;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -497,8 +481,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode11(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 11;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -550,8 +532,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode12(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 12;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -604,8 +584,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode13(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 13;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -659,8 +637,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode14(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 14;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -715,8 +691,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode15(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 15;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -772,8 +746,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode16(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 16;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -830,8 +802,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode17(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 17;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -889,8 +859,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode18(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 18;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -949,8 +917,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode19(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 19;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1010,8 +976,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode20(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 20;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1072,8 +1036,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode21(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 21;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1135,8 +1097,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode22(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 22;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1199,8 +1159,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode23(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 23;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1264,8 +1222,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode24(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 24;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1330,8 +1286,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode25(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 25;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1397,8 +1351,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode26(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 26;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1465,8 +1417,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode27(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 27;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1534,8 +1484,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode28(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 28;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1604,8 +1552,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode29(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 29;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1675,8 +1621,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode30(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 30;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1747,8 +1691,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode31(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 31;
     final int mask = (int) ((1L<<numFrameBits) - 1);
@@ -1820,8 +1762,6 @@ final class PackedIntsDecompress {
       outputOffset += 32;
     }
   }
-
-  // NOTE: hardwired to blockSize == 128
   public static void decode32(final IntBuffer compressedBuffer, final int[] output) {
     final int numFrameBits = 32;
     final int mask = (int) ((1L<<numFrameBits) - 1);

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py Wed Jul 11 13:29:11 2012
@@ -17,19 +17,14 @@
 """
 
 """
-Generate source code for java classes for FOR decompression.
+Generate source code for java classes for For or PFor decompression.
 """
 
-USE_SCRATCH = False
-
 def bitsExpr(i, numFrameBits):
   framePos = i * numFrameBits
   intValNum = (framePos / 32)
   bitPos = framePos % 32
-  if USE_SCRATCH:
-    bitsInInt = "inputInts[" + str(intValNum) + "]"
-  else:
-    bitsInInt = "intValue" + str(intValNum)
+  bitsInInt = "intValue" + str(intValNum)
   needBrackets = 0
   if bitPos > 0:
     bitsInInt +=  " >>> " + str(bitPos)
@@ -37,10 +32,7 @@ def bitsExpr(i, numFrameBits):
   if bitPos + numFrameBits > 32:
     if needBrackets:
       bitsInInt = "(" + bitsInInt + ")"
-    if USE_SCRATCH:
-      bitsInInt += " | (inputInts[" + str(intValNum+1) + "] << "+ str(32 - bitPos) + ")"
-    else:
-      bitsInInt += " | (intValue" + str(intValNum+1) + " << "+ str(32 - bitPos) + ")"
+    bitsInInt += " | (intValue" + str(intValNum+1) + " << "+ str(32 - bitPos) + ")"
     needBrackets = 1
   if bitPos + numFrameBits != 32:
     if needBrackets:
@@ -57,7 +49,7 @@ def genDecompress():
   w = f.write
   try:
     w("package org.apache.lucene.codecs.pfor;\n")
-    w("""/**
+    w("""/*
  * 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.
@@ -75,50 +67,36 @@ def genDecompress():
  */
  """)
 
-    w("/* This code is generated, do not modify. See gendecompress.py */\n\n")
+    w("\n/* This code is generated, do not modify. See gendecompress.py */\n\n")
 
     w("import java.nio.IntBuffer;\n\n")
 
     w("final class PackedIntsDecompress {\n")
 
     w('\n  // nocommit: assess perf of this to see if specializing is really needed\n')
+    w('\n  // NOTE: hardwired to blockSize == 128\n\n')
 
-    # previous version only handle int less(or equal) than 31 bits
-    # try to support 32 bits here
     for numFrameBits in xrange(1, 33):
-
-      w('\n  // NOTE: hardwired to blockSize == 128\n')
-      if USE_SCRATCH:
-        w('  public static void decode%d(final IntBuffer compressedBuffer, final int[] output, final int[] scratch) {\n' % numFrameBits)
-      else:
-        w('  public static void decode%d(final IntBuffer compressedBuffer, final int[] output) {\n' % numFrameBits)
-
+      w('  public static void decode%d(final IntBuffer compressedBuffer, final int[] output) {\n' % numFrameBits)
       w('    final int numFrameBits = %d;\n' % numFrameBits)
       w('    final int mask = (int) ((1L<<numFrameBits) - 1);\n')
       w('    int outputOffset = 0;\n')
-      
       w('    for(int step=0;step<4;step++) {\n')
 
-      if USE_SCRATCH:
-        w('      compressedBuffer.get(scratch, 0, %d);\n' % numFrameBits)
-      else:
-        for i in range(numFrameBits): # declare int vars and init from buffer
-          w("      int intValue" + str(i) + " = compressedBuffer.get();\n")
+      for i in range(numFrameBits): # declare int vars and init from buffer
+        w("      int intValue" + str(i) + " = compressedBuffer.get();\n")
 
       for i in range(32): # set output from int vars
         w("      output[" + str(i) + " + outputOffset] = " + bitsExpr(i, numFrameBits) + ";\n")
+
       w('      outputOffset += 32;\n')
       w('    }\n')
       w('  }\n')
+
     w('}\n')
       
   finally:
     f.close()
 
-def genSwitch():
-  for numFrameBits in xrange(1, 33):
-    print '      case %d: PackedIntsDecompress.decode%d(compressedBuffer, encoded); break;' % (numFrameBits, numFrameBits)
-
 if __name__ == "__main__":
   genDecompress()
-  #genSwitch()

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestPForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestPForUtil.java?rev=1360169&r1=1360168&r2=1360169&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestPForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestPForUtil.java Wed Jul 11 13:29:11 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.codecs.pfor;
 
-/**
+/*
  * 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.
@@ -17,12 +17,21 @@ package org.apache.lucene.codecs.pfor;
  * limitations under the License.
  */
 
-import java.util.*;
-import java.io.*;
-import java.nio.*;
-import org.apache.lucene.codecs.pfor.*;
+import java.util.Collections;
+import java.util.Arrays;
+import java.util.Random;
+import java.nio.IntBuffer;
+import java.nio.ByteBuffer;
+import org.apache.lucene.codecs.pfor.PForUtil;
+import org.apache.lucene.codecs.pfor.ForPostingsFormat;
 import org.apache.lucene.util.LuceneTestCase;
 
+/**
+ * Test the core utility for PFor compress and decompress
+ * We don't provide test case for For encoder/decoder, since
+ * PFor is a extended version of For, and most methods will be reused 
+ * here.
+ */
 public class TestPForUtil extends LuceneTestCase {
   static final int[] MASK={ 0x00000000,
     0x00000001, 0x00000003, 0x00000007, 0x0000000f, 0x0000001f, 0x0000003f,
@@ -32,21 +41,18 @@ public class TestPForUtil extends Lucene
     0x01ffffff, 0x03ffffff, 0x07ffffff, 0x0fffffff, 0x1fffffff, 0x3fffffff,
     0x7fffffff, 0xffffffff};
   Random gen;
-  long seed=System.currentTimeMillis();
-  //long seed=1338528171959L;
   public void initRandom() {
-  //  println("Seed: "+seed);
-    this.gen = new Random(seed);
-  }
-  public void testCompress() throws Exception {
-    initRandom();
-    tryForcedException();
-    tryAllDistribution();
+    this.gen = random();
   }
 
-  // Test correctness of ignored forced exception
-  public void tryForcedException() throws Exception {
-    int sz=128;
+  /**
+   * Test correctness of ignored forced exception.
+   * The trailing forced exceptions shouldn't be reverted
+   * since they're not necessary. 
+   */
+  public void testForcedException() throws Exception {
+    initRandom();
+    int sz=ForPostingsFormat.DEFAULT_BLOCK_SIZE;
     Integer[] buff= new Integer[sz];
     int[] data = new int[sz];
     int[] copy = new int[sz];
@@ -56,16 +62,14 @@ public class TestPForUtil extends Lucene
       buff[i]=gen.nextInt() & 1;
     buff[sz-1]=gen.nextInt() & 0xffffffff;   // create only one exception
 
-    Collections.shuffle(Arrays.asList(buff),new Random(seed));
+    Collections.shuffle(Arrays.asList(buff),gen);
     for (int i=0; i<sz; ++i)
       data[i] = buff[i];
 
     int ensz = PForUtil.compress(data,sz,resBuffer);
 
-    if (ensz > sz*8+4) {
-      println("Excceed? "+ensz+">"+(sz*8+4));
-      ensz=sz*8+4;
-    }
+    assert (ensz <= sz*8+4);  // must not exceed the loose upperbound
+
     resBuffer.rewind();
     PForUtil.decompress(resBuffer,copy);
 
@@ -76,12 +80,17 @@ public class TestPForUtil extends Lucene
     assert cmp(data,sz,copy,sz)==true;
   }
 
-  // Test correctness of compressing and decompressing
-  public void tryAllDistribution() throws Exception {
+  /**
+   * Test correctness of compressing and decompressing.
+   * Here we randomly assign a rate of exception (i.e. 1-alpha), 
+   * and test different scale of normal/exception values.
+   */
+  public void testAllDistribution() throws Exception {
+    initRandom();
     for (int i=0; i<=32; ++i) { // try to test every kinds of distribution
       double alpha=gen.nextDouble(); // rate of normal value
       for (int j=0; j<=32; ++j) {
-        tryDistribution(128,alpha,MASK[i],MASK[j]);
+        tryDistribution(ForPostingsFormat.DEFAULT_BLOCK_SIZE,alpha,MASK[i],MASK[j]);
       }
     }
   }
@@ -95,16 +104,14 @@ public class TestPForUtil extends Lucene
       buff[i]=gen.nextInt() & masknorm;
     for (; i<sz; ++i)
       buff[i]=gen.nextInt() & maskexc;
-    Collections.shuffle(Arrays.asList(buff),new Random(seed));
+    Collections.shuffle(Arrays.asList(buff),gen);
     for (i=0; i<sz; ++i)
       data[i] = buff[i];
 
     int ensz = PForUtil.compress(data,sz,resBuffer);
     
-    if (ensz > sz*8+4) {
-      println("Excceed? "+ensz+">"+(sz*8+4));
-      ensz=sz*8+4;
-    }
+    assert (ensz <= sz*8+4);  // must not exceed the loose upperbound
+
     int[] copy = new int[sz];
     PForUtil.decompress(resBuffer,copy);