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/06/20 18:13:45 UTC

svn commit: r1352200 [3/3] - in /lucene/dev/branches/pforcodec_3892/lucene: core/src/java/org/apache/lucene/codecs/pfor/ core/src/resources/META-INF/services/ core/src/test/org/apache/lucene/codecs/pfor/ core/src/test/org/apache/lucene/index/ test-fram...

Added: 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=1352200&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForFactory.java Wed Jun 20 16:13:44 2012
@@ -0,0 +1,114 @@
+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.
+ * 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 java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.IntBuffer;
+
+import org.apache.lucene.util.IOUtils;
+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.codecs.sep.IntStreamFactory;
+import org.apache.lucene.codecs.sep.IntIndexInput;
+import org.apache.lucene.codecs.sep.IntIndexOutput;
+import org.apache.lucene.codecs.intblock.FixedIntBlockIndexInput;
+import org.apache.lucene.codecs.intblock.FixedIntBlockIndexOutput;
+
+/** 
+ * Stuff to pass to PostingsReader/WriterBase.
+ * Things really make sense are: flushBlock() and readBlock()
+ */
+
+public class ForFactory extends IntStreamFactory {
+  private final int blockSize;
+
+  public ForFactory() {
+    this.blockSize=ForPostingsFormat.DEFAULT_BLOCK_SIZE;
+  }
+
+  @Override
+  public IntIndexOutput createOutput(Directory dir, String fileName, IOContext context)  throws IOException {
+    IndexOutput out = dir.createOutput(fileName, context);
+    boolean success = false;
+    try {
+      FixedIntBlockIndexOutput ret = new  ForIndexOutput(out, blockSize);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        // TODO: why handle exception like this? 
+        // and why not use similar codes for read part?
+        IOUtils.closeWhileHandlingException(out);
+      }
+    }
+  }
+  @Override
+  public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
+    FixedIntBlockIndexInput ret = new ForIndexInput(dir.openInput(fileName, context));
+    return ret;
+  }
+
+  // wrap input and output with buffer support
+  private class ForIndexInput extends FixedIntBlockIndexInput {
+    ForIndexInput(final IndexInput in) throws IOException {
+      super(in);
+    }
+    class ForBlockReader implements FixedIntBlockIndexInput.BlockReader {
+      byte[] encoded;
+      int[] buffer;
+      IndexInput in;
+      IntBuffer encodedBuffer;
+      ForBlockReader(final IndexInput in, final int[] buffer) {
+        this.encoded = new byte[blockSize*8+4];
+        this.in=in;
+        this.buffer=buffer;
+        this.encodedBuffer=ByteBuffer.wrap(encoded).asIntBuffer();
+      }
+      public void seek(long pos) {}
+      // TODO: implement public void skipBlock() {} ?
+      public void readBlock() throws IOException {
+        final int numBytes = in.readInt();
+        assert numBytes <= blockSize*8+4;
+        in.readBytes(encoded,0,numBytes);
+        ForUtil.decompress(encodedBuffer,buffer);
+      }
+    }
+    @Override
+    protected BlockReader getBlockReader(final IndexInput in, final int[] buffer) throws IOException {
+      return new ForBlockReader(in,buffer);
+    }
+  }
+
+  private class ForIndexOutput extends FixedIntBlockIndexOutput {
+      private byte[] encoded;
+      private IntBuffer encodedBuffer;
+    ForIndexOutput(IndexOutput out, int blockSize) throws IOException {
+      super(out,blockSize);
+      this.encoded = new byte[blockSize*8+4];
+      this.encodedBuffer=ByteBuffer.wrap(encoded).asIntBuffer();
+    }
+    @Override
+    protected void flushBlock() throws IOException {
+      final int numBytes = ForUtil.compress(buffer,buffer.length,encodedBuffer);
+      out.writeInt(numBytes);
+      out.writeBytes(encoded, numBytes);
+    }
+  }
+}

Added: 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=1352200&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForPostingsFormat.java Wed Jun 20 16:13:44 2012
@@ -0,0 +1,117 @@
+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.
+ * 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 java.util.Set;
+import java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.BlockTreeTermsReader;
+import org.apache.lucene.codecs.TermsIndexReaderBase;
+import org.apache.lucene.codecs.TermsIndexWriterBase;
+import org.apache.lucene.codecs.FixedGapTermsIndexReader;
+import org.apache.lucene.codecs.FixedGapTermsIndexWriter;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.PostingsWriterBase;
+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.
+ */
+public class ForPostingsFormat extends PostingsFormat {
+  private final int blockSize;
+  private final int minBlockSize;
+  private final int maxBlockSize;
+  protected final static int DEFAULT_BLOCK_SIZE = 128;
+  protected final static int DEFAULT_TERM_CACHED_SIZE = 1024;
+
+  public ForPostingsFormat() {
+    super("For");
+    this.blockSize = DEFAULT_BLOCK_SIZE;
+    this.minBlockSize = BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE;
+    this.maxBlockSize = BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE;
+  }
+  public ForPostingsFormat(int minBlockSize, int maxBlockSize) {
+    super("For");
+    this.blockSize = DEFAULT_BLOCK_SIZE;
+    this.minBlockSize = minBlockSize;
+    assert minBlockSize > 1;
+    this.maxBlockSize = maxBlockSize;
+    assert minBlockSize <= maxBlockSize;
+  }
+
+  @Override
+  public String toString() {
+    return getName() + "(blocksize=" + blockSize + ")";
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    // TODO: implement a new PostingsWriterBase to improve skip-settings
+    PostingsWriterBase postingsWriter = new SepPostingsWriter(state, new ForFactory()); 
+    boolean success = false;
+    try {
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, 
+                                                    postingsWriter,
+                                                    minBlockSize, 
+                                                    maxBlockSize);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        postingsWriter.close();
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase postingsReader = new SepPostingsReader(state.dir,
+                                                              state.fieldInfos,
+                                                              state.segmentInfo,
+                                                              state.context,
+                                                              new ForFactory(),
+                                                              state.segmentSuffix);
+
+    boolean success = false;
+    try {
+      FieldsProducer ret = new BlockTreeTermsReader(state.dir,
+                                                    state.fieldInfos,
+                                                    state.segmentInfo.name,
+                                                    postingsReader,
+                                                    state.context,
+                                                    state.segmentSuffix,
+                                                    state.termsIndexDivisor);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        postingsReader.close();
+      }
+    }
+  }
+}

Added: 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=1352200&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/ForUtil.java Wed Jun 20 16:13:44 2012
@@ -0,0 +1,176 @@
+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.
+ * 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.
+ */
+// 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
+public final class ForUtil {
+  public static final int HEADER_INT_SIZE=1;
+  private static final int[] MASK = {   0x00000000,
+    0x00000001, 0x00000003, 0x00000007, 0x0000000f, 0x0000001f, 0x0000003f,
+    0x0000007f, 0x000000ff, 0x000001ff, 0x000003ff, 0x000007ff, 0x00000fff,
+    0x00001fff, 0x00003fff, 0x00007fff, 0x0000ffff, 0x0001ffff, 0x0003ffff,
+    0x0007ffff, 0x000fffff, 0x001fffff, 0x003fffff, 0x007fffff, 0x00ffffff,
+    0x01ffffff, 0x03ffffff, 0x07ffffff, 0x0fffffff, 0x1fffffff, 0x3fffffff,
+    0x7fffffff, 0xffffffff};
+  private static final int[] PER_EXCEPTION_SIZE = {1,2,4};
+
+  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;
+  }
+  
+  public static int decompress(IntBuffer intBuffer, int[] data) {
+    intBuffer.rewind();
+    int header = intBuffer.get();
+
+    int numInts = (header & MASK[8]) + 1;
+    int numBits = ((header >> 8) & MASK[5]) + 1;
+
+    // TODO: ForDecompressImpl is hardewired to size==128 only
+    switch(numBits) {
+      case 1: ForDecompressImpl.decode1(intBuffer, data); break;
+      case 2: ForDecompressImpl.decode2(intBuffer, data); break;
+      case 3: ForDecompressImpl.decode3(intBuffer, data); break;
+      case 4: ForDecompressImpl.decode4(intBuffer, data); break;
+      case 5: ForDecompressImpl.decode5(intBuffer, data); break;
+      case 6: ForDecompressImpl.decode6(intBuffer, data); break;
+      case 7: ForDecompressImpl.decode7(intBuffer, data); break;
+      case 8: ForDecompressImpl.decode8(intBuffer, data); break;
+      case 9: ForDecompressImpl.decode9(intBuffer, data); break;
+      case 10: ForDecompressImpl.decode10(intBuffer, data); break;
+      case 11: ForDecompressImpl.decode11(intBuffer, data); break;
+      case 12: ForDecompressImpl.decode12(intBuffer, data); break;
+      case 13: ForDecompressImpl.decode13(intBuffer, data); break;
+      case 14: ForDecompressImpl.decode14(intBuffer, data); break;
+      case 15: ForDecompressImpl.decode15(intBuffer, data); break;
+      case 16: ForDecompressImpl.decode16(intBuffer, data); break;
+      case 17: ForDecompressImpl.decode17(intBuffer, data); break;
+      case 18: ForDecompressImpl.decode18(intBuffer, data); break;
+      case 19: ForDecompressImpl.decode19(intBuffer, data); break;
+      case 20: ForDecompressImpl.decode20(intBuffer, data); break;
+      case 21: ForDecompressImpl.decode21(intBuffer, data); break;
+      case 22: ForDecompressImpl.decode22(intBuffer, data); break;
+      case 23: ForDecompressImpl.decode23(intBuffer, data); break;
+      case 24: ForDecompressImpl.decode24(intBuffer, data); break;
+      case 25: ForDecompressImpl.decode25(intBuffer, data); break;
+      case 26: ForDecompressImpl.decode26(intBuffer, data); break;
+      case 27: ForDecompressImpl.decode27(intBuffer, data); break;
+      case 28: ForDecompressImpl.decode28(intBuffer, data); break;
+      case 29: ForDecompressImpl.decode29(intBuffer, data); break;
+      case 30: ForDecompressImpl.decode30(intBuffer, data); break;
+      case 31: ForDecompressImpl.decode31(intBuffer, data); break;
+      case 32: ForDecompressImpl.decode32(intBuffer, data); break;
+      default:
+        throw new IllegalStateException("Unknown numFrameBits " + numBits);
+    }
+    return numInts;
+  }
+
+  static void encodeHeader(IntBuffer intBuffer, int numInts, int numBits) {
+    int header = getHeader(numInts,numBits);
+    intBuffer.put(0, header);
+  }
+
+  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 
+    setBufferIntBits(intBuffer, intPos, localBitPos, numBits, value);
+    if ((localBitPos + numBits) > 32) { // value does not fit in this int, fill tail
+      setBufferIntBits(intBuffer, intPos+1, 0, 
+                       (localBitPos+numBits-32), 
+                       (value >>> (32-localBitPos)));
+    }
+  }
+
+  static void setBufferIntBits(IntBuffer intBuffer, int intPos, int firstBitPos, int numBits, int value) {
+    assert (value & ~MASK[numBits]) == 0;
+    // safely discards those msb parts when firstBitPos+numBits>32
+    intBuffer.put(intPos,
+          (intBuffer.get(intPos) & ~(MASK[numBits] << firstBitPos)) 
+          | (value << firstBitPos));
+  }
+
+  // TODO: shall we use 32 NumBits directly if it exceeds 28 bits?
+  static int getNumBits(final int[] data, int size) {
+    int optBits=1;
+    for (int i=0; i<size; ++i) {
+      while ((data[i] & ~MASK[optBits]) != 0) {
+        optBits++;
+      }
+    }
+    return optBits;
+  }
+  /** The 4 byte header (32 bits) 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
+   *
+   */
+  static int getHeader(int numInts, int numBits) {
+    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();
+  }
+}

Added: 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=1352200&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/pfor/gendecompress.py Wed Jun 20 16:13:44 2012
@@ -0,0 +1,125 @@
+#!/usr/bin/env python2
+"""
+  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.
+"""
+
+"""
+Generate source code for java classes for FOR decompression.
+"""
+
+USE_SCRATCH = False
+#USE_SCRATCH = True 
+
+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)
+  needBrackets = 0
+  if bitPos > 0:
+    bitsInInt +=  " >>> " + str(bitPos)
+    needBrackets = 1
+  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) + ")"
+    needBrackets = 1
+  if bitPos + numFrameBits != 32:
+    if needBrackets:
+      bitsInInt = "(" + bitsInInt + ")"
+    bitsInInt += " & mask"
+  return bitsInInt
+
+
+def genDecompress():
+  className = "ForDecompressImpl"
+  fileName = className + ".java"
+  imports = "import java.nio.IntBuffer;\n"
+  f = open(fileName, 'w')
+  w = f.write
+  try:
+    w("package org.apache.lucene.codecs.pfor;\n")
+    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.
+ * 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.
+ */
+ """)
+
+    w("/* This code is generated, do not modify. See gendecompress.py */\n\n")
+
+    w("import java.nio.IntBuffer;\n\n")
+
+    w("final class ForDecompressImpl {\n")
+
+    w('\n  // nocommit: assess perf of this to see if specializing is really needed\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('    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(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: ForDecompressImpl.decode%d(compressedBuffer, encoded); break;' % (numFrameBits, numFrameBits)
+
+if __name__ == "__main__":
+  genDecompress()
+  #genSwitch()

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1352200&r1=1352199&r2=1352200&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Wed Jun 20 16:13:44 2012
@@ -17,3 +17,4 @@ org.apache.lucene.codecs.lucene40.Lucene
 org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat
 org.apache.lucene.codecs.simpletext.SimpleTextPostingsFormat
 org.apache.lucene.codecs.memory.MemoryPostingsFormat
+org.apache.lucene.codecs.pfor.ForPostingsFormat

Added: lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestForUtil.java?rev=1352200&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestForUtil.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/codecs/pfor/TestForUtil.java Wed Jun 20 16:13:44 2012
@@ -0,0 +1,163 @@
+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.
+ * 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 java.util.*;
+import java.io.*;
+import java.nio.*;
+import org.apache.lucene.codecs.pfor.*;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestForUtil extends LuceneTestCase {
+  static final int[] MASK={ 0x00000000,
+    0x00000001, 0x00000003, 0x00000007, 0x0000000f, 0x0000001f, 0x0000003f,
+    0x0000007f, 0x000000ff, 0x000001ff, 0x000003ff, 0x000007ff, 0x00000fff,
+    0x00001fff, 0x00003fff, 0x00007fff, 0x0000ffff, 0x0001ffff, 0x0003ffff,
+    0x0007ffff, 0x000fffff, 0x001fffff, 0x003fffff, 0x007fffff, 0x00ffffff,
+    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();
+  }
+
+  // Test correctness of ignored forced exception
+  public void tryForcedException() throws Exception {
+    int sz=128;
+    Integer[] buff= new Integer[sz];
+    int[] data = new int[sz];
+    int[] copy = new int[sz];
+    byte[] res = new byte[4+sz*8];
+    IntBuffer resBuffer = ByteBuffer.wrap(res).asIntBuffer();
+    for (int i=0; i<sz-1; ++i)
+      buff[i]=gen.nextInt() & 0;
+    buff[sz-1]=gen.nextInt() & 0xffffffff;   // create only one exception
+
+    Collections.shuffle(Arrays.asList(buff),new Random(seed));
+    for (int i=0; i<sz; ++i)
+      data[i] = buff[i];
+
+    int ensz = ForUtil.compress(data,sz,resBuffer);
+
+    if (ensz > sz*8+4) {
+      println("Excceed? "+ensz+">"+(sz*8+4));
+      ensz=sz*8+4;
+    }
+    resBuffer.rewind();
+    ForUtil.decompress(resBuffer,copy);
+
+//    println(getHex(data,sz)+"\n");
+//    println(getHex(res,ensz)+"\n");
+//    println(getHex(copy,sz)+"\n");
+    
+    assert cmp(data,sz,copy,sz)==true;
+  }
+
+  // Test correctness of compressing and decompressing
+  public void tryAllDistribution() throws Exception {
+    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]);
+      }
+    }
+  }
+  public void tryDistribution(int sz, double alpha, int masknorm, int maskexc) throws Exception {
+    Integer[] buff= new Integer[sz];
+    int[] data = new int[sz];
+    byte[] res = new byte[4+sz*8];      // loosely upperbound
+    IntBuffer resBuffer = ByteBuffer.wrap(res).asIntBuffer();
+    int i=0;
+    for (; i<sz*alpha; ++i)
+      buff[i]=gen.nextInt() & masknorm;
+    for (; i<sz; ++i)
+      buff[i]=gen.nextInt() & maskexc;
+    Collections.shuffle(Arrays.asList(buff),new Random(seed));
+    for (i=0; i<sz; ++i)
+      data[i] = buff[i];
+
+    int ensz = ForUtil.compress(data,sz,resBuffer);
+    
+    if (ensz > sz*8+4) {
+      println("Excceed? "+ensz+">"+(sz*8+4));
+      ensz=sz*8+4;
+    }
+    int[] copy = new int[sz];
+
+    ForUtil.decompress(resBuffer,copy);
+
+//    println(getHex(data,sz)+"\n");
+//    println(getHex(res,ensz)+"\n");
+//    println(getHex(copy,sz)+"\n");
+
+    assert cmp(data,sz,copy,sz)==true;
+  }
+  public boolean cmp(int[] a, int sza, int[] b, int szb) {
+    if (sza!=szb)
+      return false;
+    for (int i=0; i<sza; ++i) {
+      if (a[i]!=b[i]) {
+        System.err.println(String.format("! %08x != %08x in %d",a[i],b[i],i));
+        return false;
+      }
+    }
+    return true;
+  }
+  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();
+  }
+  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)); 
+  }
+}

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1352200&r1=1352199&r2=1352200&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Wed Jun 20 16:13:44 2012
@@ -76,7 +76,7 @@ import org.junit.BeforeClass;
 // we won't even be running the actual code, only the impostor
 // @SuppressCodecs("Lucene4x")
 // Sep codec cannot yet handle the offsets in our 4.x index!
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom","For"})
 public class TestBackwardsCompatibility extends LuceneTestCase {
 
   // Uncomment these cases & run them on an older Lucene

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java?rev=1352200&r1=1352199&r2=1352200&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/test/org/apache/lucene/index/TestPostingsOffsets.java Wed Jun 20 16:13:44 2012
@@ -49,7 +49,7 @@ import org.apache.lucene.util._TestUtil;
 // TODO: we really need to test indexingoffsets, but then getting only docs / docs + freqs.
 // not all codecs store prx separate...
 // TODO: fix sep codec to index offsets so we can greatly reduce this list!
-@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom"})
+@SuppressCodecs({"MockFixedIntBlock", "MockVariableIntBlock", "MockSep", "MockRandom","For"})
 public class TestPostingsOffsets extends LuceneTestCase {
   IndexWriterConfig iwc;
   
@@ -438,14 +438,6 @@ public class TestPostingsOffsets extends
     }
   }
   
-  public void testStackedTokens() throws Exception {
-    checkTokens(new Token[] { 
-        makeToken("foo", 1, 0, 3),
-        makeToken("foo", 0, 0, 3),
-        makeToken("foo", 0, 0, 3)
-     });
-  }
-  
   public void testLegalbutVeryLargeOffsets() throws Exception {
     Directory dir = newDirectory();
     IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, null));

Modified: lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1352200&r1=1352199&r2=1352200&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Wed Jun 20 16:13:44 2012
@@ -486,6 +486,7 @@ public abstract class BaseTokenStreamTes
     add("MockVariableIntBlock");
     add("MockSep");
     add("MockRandom");
+    add("For");
   }};
   
   private static void checkRandomData(Random random, Analyzer a, int iterations, int maxWordLength, boolean useCharFilter, boolean simple, boolean offsetsAreCorrect, RandomIndexWriter iw) throws IOException {

Modified: lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1352200&r1=1352199&r2=1352200&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Wed Jun 20 16:13:44 2012
@@ -1,6 +1,6 @@
 package org.apache.lucene.codecs.mockrandom;
 
-/*
+/**
  * 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.
@@ -61,6 +61,7 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
+import org.apache.lucene.codecs.pfor.*;
 
 /**
  * Randomly combines terms index impl w/ postings impls.
@@ -93,6 +94,7 @@ public class MockRandomPostingsFormat ex
       final int baseBlockSize = _TestUtil.nextInt(random, 1, 127);
       delegates.add(new MockVariableIntBlockPostingsFormat.MockIntFactory(baseBlockSize));
       // TODO: others
+      delegates.add(new ForFactory());
     }
 
     private static String getExtension(String fileName) {