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

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

Author: jpountz
Date: Tue Aug  7 13:56:30 2012
New Revision: 1370267

URL: http://svn.apache.org/viewvc?rev=1370267&view=rev
Log:
LUCENE-3892: performance improvements to the 'BlockPacked' codec.

 - backport of Mike's changes to the 'Block' codec,
 - revert of oal.util.packed.BulkOperation in order not to use java.nio.*Buffer,
 - add direct byte[] decoding and encoding to PackedInts.Decoder and PackedInts.Encoder.

Modified:
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java
    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/BlockPackedSkipReader.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.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/Packed64.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/Packed64SingleBlock.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedReaderIterator.java
    lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/util/packed/PackedWriter.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/block/ForUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/block/ForUtil.java Tue Aug  7 13:56:30 2012
@@ -75,7 +75,7 @@ public final class ForUtil {
     decompressCore(intBuffer, data, numBits);
   }
 
-  static void decompressCore(IntBuffer intBuffer, int[] data, int numBits) {
+  public static void decompressCore(IntBuffer intBuffer, int[] data, int numBits) {
     switch(numBits) {
       case 0: PackedIntsDecompress.decode0(intBuffer, data); break;
       case 1: PackedIntsDecompress.decode1(intBuffer, data); break;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedPostingsFormat.java Tue Aug  7 13:56:30 2012
@@ -31,8 +31,8 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.util.IOUtils;
 
 /**
- * Pass ForFactory to a PostingsWriter/ReaderBase, and get 
- * customized postings format plugged.
+ * Encodes/decode postings in packed int blocks for faster
+ * decode.
  */
 public final class BlockPackedPostingsFormat extends PostingsFormat {
   public static final String DOC_EXTENSION = "doc";
@@ -41,7 +41,7 @@ public final class BlockPackedPostingsFo
 
   private final int minTermBlockSize;
   private final int maxTermBlockSize;
-  public final static int DEFAULT_BLOCK_SIZE = 128;
+  public final static int BLOCK_SIZE = 128;
 
   public BlockPackedPostingsFormat() {
     this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
@@ -57,13 +57,12 @@ public final class BlockPackedPostingsFo
 
   @Override
   public String toString() {
-    return getName() + "(blocksize=" + DEFAULT_BLOCK_SIZE + ")";
+    return getName() + "(blocksize=" + BLOCK_SIZE + ")";
   }
 
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    // TODO: implement a new PostingsWriterBase to improve skip-settings
-    PostingsWriterBase postingsWriter = new BlockPackedPostingsWriter(state, 128);
+    PostingsWriterBase postingsWriter = new BlockPackedPostingsWriter(state);
 
     boolean success = false;
     try {
@@ -86,8 +85,7 @@ public final class BlockPackedPostingsFo
                                                                       state.fieldInfos,
                                                                       state.segmentInfo,
                                                                       state.context,
-                                                                      state.segmentSuffix,
-                                                                      128);
+                                                                      state.segmentSuffix);
     boolean success = false;
     try {
       FieldsProducer ret = new BlockTreeTermsReader(state.dir,

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=1370267&r1=1370266&r2=1370267&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 Tue Aug  7 13:56:30 2012
@@ -17,19 +17,19 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+
 import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.IntBuffer;
-import java.nio.LongBuffer;
 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.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -44,10 +44,13 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
-// nocommit can we share more w/ BlockPF?  like make the
-// ForUtil class pluggable then pass it in...
-
-// nocommit javadocs
+/**
+ * Concrete class that reads docId(maybe frq,pos,offset,payloads) list
+ * with postings format.
+ *
+ * @see BlockPackedSkipReader for details
+ *
+ */
 public final class BlockPackedPostingsReader extends PostingsReaderBase {
 
   private final IndexInput docIn;
@@ -59,10 +62,7 @@ public final class BlockPackedPostingsRe
   // nocommit
   final String segment;
 
-  // NOTE: not private to avoid access$NNN methods:
-  final int blockSize;
-
-  public BlockPackedPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix, int blockSize) throws IOException {
+  public BlockPackedPostingsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo segmentInfo, IOContext ioContext, String segmentSuffix) throws IOException {
     boolean success = false;
     segment = segmentInfo.name;
     IndexInput docIn = null;
@@ -103,8 +103,6 @@ public final class BlockPackedPostingsRe
         IOUtils.closeWhileHandlingException(docIn, posIn, payIn);
       }
     }
-
-    this.blockSize = blockSize;
   }
 
   @Override
@@ -115,22 +113,11 @@ public final class BlockPackedPostingsRe
                           BlockPackedPostingsWriter.VERSION_START,
                           BlockPackedPostingsWriter.VERSION_START);
     final int indexBlockSize = termsIn.readVInt();
-    if (indexBlockSize != blockSize) {
-      throw new IllegalStateException("index-time blockSize (" + indexBlockSize + ") != read-time blockSize (" + blockSize + ")");
+    if (indexBlockSize != BLOCK_SIZE) {
+      throw new IllegalStateException("index-time blockSize (" + indexBlockSize + ") != read-time blockSize (" + BLOCK_SIZE + ")");
     }
   }
 
-  static void readBlock(IndexInput in, byte[] encoded, LongBuffer encodedBuffer, LongBuffer buffer) throws IOException {
-    int header = in.readVInt();
-    in.readBytes(encoded, 0, ForUtil.getEncodedSize(header));
-    ForUtil.decompress(buffer, encodedBuffer, header);
-  }
-
-  static void skipBlock(IndexInput in) throws IOException {
-    int header = in.readVInt();
-    in.seek(in.getFilePointer() + ForUtil.getEncodedSize(header));
-  }
-
   // Must keep final because we do non-standard clone
   private final static class IntBlockTermState extends BlockTermState {
     long docStartFP;
@@ -216,12 +203,12 @@ public final class BlockPackedPostingsRe
       termState.docStartFP = in.readVLong();
       if (fieldHasPositions) {
         termState.posStartFP = in.readVLong();
-        if (termState.totalTermFreq > blockSize) {
+        if (termState.totalTermFreq > BLOCK_SIZE) {
           termState.lastPosBlockOffset = in.readVInt();
         } else {
           termState.lastPosBlockOffset = -1;
         }
-        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= blockSize) {
+        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
           termState.payStartFP = in.readVLong();
         } else {
           termState.payStartFP = -1;
@@ -231,12 +218,12 @@ public final class BlockPackedPostingsRe
       termState.docStartFP += in.readVLong();
       if (fieldHasPositions) {
         termState.posStartFP += in.readVLong();
-        if (termState.totalTermFreq > blockSize) {
+        if (termState.totalTermFreq > BLOCK_SIZE) {
           termState.lastPosBlockOffset = in.readVInt();
         } else {
           termState.lastPosBlockOffset = -1;
         }
-        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= blockSize) {
+        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
           long delta = in.readVLong();
           if (termState.payStartFP == -1) {
             termState.payStartFP = delta;
@@ -247,7 +234,7 @@ public final class BlockPackedPostingsRe
       }
     }
 
-    if (termState.docFreq > blockSize) {
+    if (termState.docFreq > BLOCK_SIZE) {
       termState.skipOffset = in.readVInt();
     } else {
       termState.skipOffset = -1;
@@ -276,10 +263,10 @@ public final class BlockPackedPostingsRe
     throws IOException {
 
     boolean indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-    boolean indexHasPayloasd = fieldInfo.hasPayloads();
+    boolean indexHasPayloads = fieldInfo.hasPayloads();
 
     if ((!indexHasOffsets || (flags & DocsAndPositionsEnum.FLAG_OFFSETS) == 0) &&
-        (!fieldInfo.hasPayloads() || (flags & DocsAndPositionsEnum.FLAG_PAYLOADS) == 0)) {
+        (!indexHasPayloads || (flags & DocsAndPositionsEnum.FLAG_PAYLOADS) == 0)) {
       BlockDocsAndPositionsEnum docsAndPositionsEnum;
       if (reuse instanceof BlockDocsAndPositionsEnum) {
         docsAndPositionsEnum = (BlockDocsAndPositionsEnum) reuse;
@@ -306,12 +293,9 @@ public final class BlockPackedPostingsRe
 
   final class BlockDocsEnum extends DocsEnum {
     private final byte[] encoded;
-    private final LongBuffer encodedBuffer;
     
-    private final long[] docDeltaBuffer = new long[blockSize];
-    private final long[] freqBuffer = new long[blockSize];
-    private final LongBuffer docDeltaLBuffer = LongBuffer.wrap(docDeltaBuffer);
-    private final LongBuffer freqLBuffer = LongBuffer.wrap(freqBuffer);
+    private final long[] docDeltaBuffer = new long[BLOCK_SIZE];
+    private final long[] freqBuffer = new long[BLOCK_SIZE];
 
     private int docBufferUpto;
 
@@ -349,8 +333,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[blockSize*4];
-      encodedBuffer = ByteBuffer.wrap(encoded).asLongBuffer();      
+      encoded = new byte[BLOCK_SIZE * 4];    
     }
 
     public boolean canReuse(IndexInput docIn, FieldInfo fieldInfo) {
@@ -369,14 +352,14 @@ public final class BlockPackedPostingsRe
       docTermStartFP = termState.docStartFP;
       docIn.seek(docTermStartFP);
       skipOffset = termState.skipOffset;
+
+      doc = -1;
       if (!indexHasFreq) {
         Arrays.fill(freqBuffer, 1);
       }
-
-      doc = -1;
       accum = 0;
       docUpto = 0;
-      docBufferUpto = blockSize;
+      docBufferUpto = BLOCK_SIZE;
       skipped = false;
       return this;
     }
@@ -392,50 +375,37 @@ public final class BlockPackedPostingsRe
     }
     
     private void refillDocs() throws IOException {
+      //System.out.println("["+docFreq+"]"+" refillDoc");
       final int left = docFreq - docUpto;
       assert left > 0;
 
-      if (left >= blockSize) {
+      if (left >= BLOCK_SIZE) {
         if (DEBUG) {
           System.out.println("    fill doc block from fp=" + docIn.getFilePointer());
         }
-        readBlock(docIn, encoded, encodedBuffer, docDeltaLBuffer);
+        ForUtil.readBlock(docIn, encoded, docDeltaBuffer);
 
         if (indexHasFreq) {
           if (DEBUG) {
             System.out.println("    fill freq block from fp=" + docIn.getFilePointer());
           }
-          readBlock(docIn, encoded, encodedBuffer, freqLBuffer);
+          ForUtil.readBlock(docIn, encoded, freqBuffer);
         }
       } else {
         // Read vInts:
         if (DEBUG) {
           System.out.println("    fill last vInt block from fp=" + docIn.getFilePointer());
         }
-        for(int i=0;i<left;i++) {
-          final int code = docIn.readVInt();
-          if (indexHasFreq) {
-            docDeltaBuffer[i] = code >>> 1;
-            if ((code & 1) != 0) {
-              freqBuffer[i] = 1;
-            } else {
-              freqBuffer[i] = docIn.readVInt();
-            }
-          } else {
-            docDeltaBuffer[i] = code;
-          }
-        }
+        ForUtil.readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, indexHasFreq);
       }
       docBufferUpto = 0;
     }
 
     @Override
     public int nextDoc() throws IOException {
-
       if (DEBUG) {
         System.out.println("\nFPR.nextDoc");
       }
-
       while (true) {
         if (DEBUG) {
           System.out.println("  docUpto=" + docUpto + " (of df=" + docFreq + ") docBufferUpto=" + docBufferUpto);
@@ -447,8 +417,8 @@ public final class BlockPackedPostingsRe
           }
           return doc = NO_MORE_DOCS;
         }
-
-        if (docBufferUpto == blockSize) {
+        //System.out.println("["+docFreq+"]"+" nextDoc");
+        if (docBufferUpto == BLOCK_SIZE) {
           refillDocs();
         }
 
@@ -460,31 +430,27 @@ public final class BlockPackedPostingsRe
 
         if (liveDocs == null || liveDocs.get(accum)) {
           doc = accum;
-          freq = (int)freqBuffer[docBufferUpto];
+          freq = (int) freqBuffer[docBufferUpto];
           docBufferUpto++;
           if (DEBUG) {
             System.out.println("  return doc=" + doc + " freq=" + freq);
           }
           return doc;
         }
-
         if (DEBUG) {
           System.out.println("  doc=" + accum + " is deleted; try next doc");
         }
-
         docBufferUpto++;
       }
     }
-    
+
     @Override
     public int advance(int target) throws IOException {
       // nocommit make frq block load lazy/skippable
 
-      // nocommit 2 is heuristic guess!!
-      // nocommit put cheating back!  does it help?
       // nocommit use skipper!!!  it has next last doc id!!
-      //if (docFreq > blockSize && target - (blockSize - docBufferUpto) - 2*blockSize > accum) {
-      if (docFreq > blockSize && target - accum > blockSize) {
+
+      if (docFreq > BLOCK_SIZE && target - accum > BLOCK_SIZE) {
 
         if (DEBUG) {
           System.out.println("load skipper");
@@ -494,7 +460,7 @@ public final class BlockPackedPostingsRe
           // Lazy init: first time this enum has ever been used for skipping
           skipper = new BlockPackedSkipReader((IndexInput) docIn.clone(),
                                         BlockPackedPostingsWriter.maxSkipLevels,
-                                        blockSize,
+                                        BLOCK_SIZE,
                                         indexHasPos,
                                         indexHasOffsets,
                                         indexHasPayloads);
@@ -512,36 +478,60 @@ public final class BlockPackedPostingsRe
 
         if (newDocUpto > docUpto) {
           // Skipper moved
-
           if (DEBUG) {
             System.out.println("skipper moved to docUpto=" + newDocUpto + " vs current=" + docUpto + "; docID=" + skipper.getDoc() + " fp=" + skipper.getDocPointer());
           }
-
-          assert newDocUpto % blockSize == (blockSize-1): "got " + newDocUpto;
+          assert newDocUpto % BLOCK_SIZE == (BLOCK_SIZE - 1): "got " + newDocUpto;
           docUpto = newDocUpto+1;
 
-          // Force block read next:
-          docBufferUpto = blockSize;
-          accum = skipper.getDoc();
-          docIn.seek(skipper.getDocPointer());
+          // Force to read next block
+          docBufferUpto = BLOCK_SIZE;
+          accum = skipper.getDoc();               // actually, this is just lastSkipEntry
+          docIn.seek(skipper.getDocPointer());    // now point to the block we want to search
         }
       }
 
-      // Now scan:
-      while (nextDoc() != NO_MORE_DOCS) {
-        if (doc >= target) {
-          if (DEBUG) {
-            System.out.println("  advance return doc=" + doc);
-          }
-          return doc;
+      // Now scan... this is an inlined/pared down version
+      // of nextDoc():
+      while (true) {
+        if (DEBUG) {
+          System.out.println("  scan doc=" + accum + " docBufferUpto=" + docBufferUpto);
+        }
+        if (docUpto == docFreq) {
+          return doc = NO_MORE_DOCS;
         }
-      }
 
-      if (DEBUG) {
-        System.out.println("  advance return doc=END");
+        // nocommit: in theory we should not hit this?  ie
+        // skipper should already have moved us to the block
+        // containing the doc?  yet assert false trips ... i
+        // think because if you advance w/o having done a
+        // nextDoc yet()... can we assert/remove this?
+        if (docBufferUpto == BLOCK_SIZE) {
+          refillDocs();
+        }
+        accum += docDeltaBuffer[docBufferUpto];
+        docUpto++;
+
+        if (accum >= target) {
+          break;
+        }
+        docBufferUpto++;
       }
 
-      return NO_MORE_DOCS;
+      if (liveDocs == null || liveDocs.get(accum)) {
+        if (DEBUG) {
+          System.out.println("  return doc=" + accum);
+        }
+        freq = (int) freqBuffer[docBufferUpto];
+        docBufferUpto++;
+        return doc = accum;
+      } else {
+        if (DEBUG) {
+          System.out.println("  now do nextDoc()");
+        }
+        docBufferUpto++;
+        return nextDoc();
+      }
     }
   }
 
@@ -549,16 +539,10 @@ public final class BlockPackedPostingsRe
   final class BlockDocsAndPositionsEnum extends DocsAndPositionsEnum {
     
     private final byte[] encoded;
-    private final LongBuffer encodedBuffer;
-
-    private final long[] docDeltaBuffer = new long[blockSize];
-    private final long[] freqBuffer = new long[blockSize];
-    private final long[] posDeltaBuffer = new long[blockSize];
-
 
-    private final LongBuffer docDeltaLBuffer = LongBuffer.wrap(docDeltaBuffer);
-    private final LongBuffer freqLBuffer = LongBuffer.wrap(freqBuffer);
-    private final LongBuffer posDeltaLBuffer = LongBuffer.wrap(posDeltaBuffer);
+    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 int docBufferUpto;
     private int posBufferUpto;
@@ -615,8 +599,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[blockSize*4];
-      encodedBuffer = ByteBuffer.wrap(encoded).asLongBuffer();
+      encoded = new byte[BLOCK_SIZE*4];
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       indexHasPayloads = fieldInfo.hasPayloads();
     }
@@ -640,9 +623,9 @@ public final class BlockPackedPostingsRe
       skipOffset = termState.skipOffset;
       posPendingFP = posTermStartFP;
       posPendingCount = 0;
-      if (termState.totalTermFreq < blockSize) {
+      if (termState.totalTermFreq < BLOCK_SIZE) {
         lastPosBlockFP = posTermStartFP;
-      } else if (termState.totalTermFreq == blockSize) {
+      } else if (termState.totalTermFreq == BLOCK_SIZE) {
         lastPosBlockFP = -1;
       } else {
         lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset;
@@ -651,7 +634,7 @@ public final class BlockPackedPostingsRe
       doc = -1;
       accum = 0;
       docUpto = 0;
-      docBufferUpto = blockSize;
+      docBufferUpto = BLOCK_SIZE;
       skipped = false;
       return this;
     }
@@ -667,35 +650,24 @@ public final class BlockPackedPostingsRe
     }
 
     private void refillDocs() throws IOException {
+    //System.out.println("["+docFreq+"]"+" refillDoc");
       final int left = docFreq - docUpto;
       assert left > 0;
-
-      if (left >= blockSize) {
+      if (left >= BLOCK_SIZE) {
         if (DEBUG) {
           System.out.println("    fill doc block from fp=" + docIn.getFilePointer());
         }
-
-        readBlock(docIn, encoded, encodedBuffer, docDeltaLBuffer);
-
+        ForUtil.readBlock(docIn, encoded, docDeltaBuffer);
         if (DEBUG) {
           System.out.println("    fill freq block from fp=" + docIn.getFilePointer());
         }
-
-        readBlock(docIn, encoded, encodedBuffer, freqLBuffer);
+        ForUtil.readBlock(docIn, encoded, freqBuffer);
       } else {
         // Read vInts:
         if (DEBUG) {
           System.out.println("    fill last vInt doc block from fp=" + docIn.getFilePointer());
         }
-        for(int i=0;i<left;i++) {
-          final int code = docIn.readVInt();
-          docDeltaBuffer[i] = code >>> 1;
-          if ((code & 1) != 0) {
-            freqBuffer[i] = 1;
-          } else {
-            freqBuffer[i] = docIn.readVInt();
-          }
-        }
+        ForUtil.readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, true);
       }
       docBufferUpto = 0;
     }
@@ -723,7 +695,6 @@ public final class BlockPackedPostingsRe
           } else {
             posDeltaBuffer[i] = code;
           }
-
           if (indexHasOffsets) {
             posIn.readVInt();
             posIn.readVInt();
@@ -733,48 +704,42 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("        bulk pos block @ fp=" + posIn.getFilePointer());
         }
-        readBlock(posIn, encoded, encodedBuffer, posDeltaLBuffer);
+        ForUtil.readBlock(posIn, encoded, posDeltaBuffer);
       }
     }
 
     @Override
     public int nextDoc() throws IOException {
-
       if (DEBUG) {
         System.out.println("  FPR.nextDoc");
       }
-
       while (true) {
         if (DEBUG) {
           System.out.println("    docUpto=" + docUpto + " (of df=" + docFreq + ") docBufferUpto=" + docBufferUpto);
         }
-
         if (docUpto == docFreq) {
           return doc = NO_MORE_DOCS;
         }
-
-        if (docBufferUpto == blockSize) {
+        if (docBufferUpto == BLOCK_SIZE) {
           refillDocs();
         }
-
         if (DEBUG) {
           System.out.println("    accum=" + accum + " docDeltaBuffer[" + docBufferUpto + "]=" + docDeltaBuffer[docBufferUpto]);
         }
-        accum += (int)docDeltaBuffer[docBufferUpto];
-        freq = (int)freqBuffer[docBufferUpto];
+        accum += (int) docDeltaBuffer[docBufferUpto];
+        freq = (int) freqBuffer[docBufferUpto];
         posPendingCount += freq;
         docBufferUpto++;
         docUpto++;
 
         if (liveDocs == null || liveDocs.get(accum)) {
           doc = accum;
+          position = 0;
           if (DEBUG) {
             System.out.println("    return doc=" + doc + " freq=" + freq + " posPendingCount=" + posPendingCount);
           }
-          position = 0;
           return doc;
         }
-
         if (DEBUG) {
           System.out.println("    doc=" + accum + " is deleted; try next doc");
         }
@@ -792,12 +757,10 @@ public final class BlockPackedPostingsRe
       // nocommit put cheating back!  does it help?
       // nocommit use skipper!!!  it has next last doc id!!
       //if (docFreq > blockSize && target - (blockSize - docBufferUpto) - 2*blockSize > accum) {
-      if (docFreq > blockSize && target - accum > blockSize) {
-
+      if (docFreq > BLOCK_SIZE && target - accum > BLOCK_SIZE) {
         if (DEBUG) {
           System.out.println("    try skipper");
         }
-
         if (skipper == null) {
           // Lazy init: first time this enum has ever been used for skipping
           if (DEBUG) {
@@ -805,7 +768,7 @@ public final class BlockPackedPostingsRe
           }
           skipper = new BlockPackedSkipReader((IndexInput) docIn.clone(),
                                         BlockPackedPostingsWriter.maxSkipLevels,
-                                        blockSize,
+                                        BLOCK_SIZE,
                                         true,
                                         indexHasOffsets,
                                         indexHasPayloads);
@@ -826,16 +789,15 @@ public final class BlockPackedPostingsRe
 
         if (newDocUpto > docUpto) {
           // Skipper moved
-
           if (DEBUG) {
             System.out.println("    skipper moved to docUpto=" + newDocUpto + " vs current=" + docUpto + "; docID=" + skipper.getDoc() + " fp=" + skipper.getDocPointer() + " pos.fp=" + skipper.getPosPointer() + " pos.bufferUpto=" + skipper.getPosBufferUpto());
           }
 
-          assert newDocUpto % blockSize == (blockSize-1): "got " + newDocUpto;
+          assert newDocUpto % BLOCK_SIZE == (BLOCK_SIZE - 1): "got " + newDocUpto;
           docUpto = newDocUpto+1;
 
-          // Force block read next:
-          docBufferUpto = blockSize;
+          // Force to read next block
+          docBufferUpto = BLOCK_SIZE;
           accum = skipper.getDoc();
           docIn.seek(skipper.getDocPointer());
           posPendingFP = skipper.getPosPointer();
@@ -843,21 +805,48 @@ public final class BlockPackedPostingsRe
         }
       }
 
-      // Now scan:
-      while (nextDoc() != NO_MORE_DOCS) {
-        if (doc >= target) {
-          if (DEBUG) {
-            System.out.println("  advance return doc=" + doc);
-          }
-          return doc;
+      // Now scan... this is an inlined/pared down version
+      // of nextDoc():
+      while (true) {
+        if (DEBUG) {
+          System.out.println("  scan doc=" + accum + " docBufferUpto=" + docBufferUpto);
         }
-      }
+        if (docUpto == docFreq) {
+          return doc = NO_MORE_DOCS;
+        }
+        // nocommit: in theory we should not hit this?  ie
+        // skipper should already have moved us to the block
+        // containing the doc?  yet assert false trips ... i
+        // think because if you advance w/o having done a
+        // nextDoc yet()... can we assert/remove this?
+        if (docBufferUpto == BLOCK_SIZE) {
+          // nocommit hmm skip freq?  but: we don't ever
+          // scan over more than one block?
+          refillDocs();
+        }
+        accum += docDeltaBuffer[docBufferUpto];
+        freq = (int) freqBuffer[docBufferUpto];
+        posPendingCount += freq;
+        docBufferUpto++;
+        docUpto++;
 
-      if (DEBUG) {
-        System.out.println("  advance return doc=END");
+        if (accum >= target) {
+          break;
+        }
       }
 
-      return NO_MORE_DOCS;
+      if (liveDocs == null || liveDocs.get(accum)) {
+        if (DEBUG) {
+          System.out.println("  return doc=" + accum);
+        }
+        position = 0;
+        return doc = accum;
+      } else {
+        if (DEBUG) {
+          System.out.println("  now do nextDoc()");
+        }
+        return nextDoc();
+      }
     }
 
     // nocommit in theory we could avoid loading frq block
@@ -871,7 +860,7 @@ public final class BlockPackedPostingsRe
         System.out.println("      FPR.skipPositions: toSkip=" + toSkip);
       }
 
-      final int leftInBlock = blockSize - posBufferUpto;
+      final int leftInBlock = BLOCK_SIZE - posBufferUpto;
       if (toSkip < leftInBlock) {
         posBufferUpto += toSkip;
         if (DEBUG) {
@@ -879,13 +868,13 @@ public final class BlockPackedPostingsRe
         }
       } else {
         toSkip -= leftInBlock;
-        while(toSkip >= blockSize) {
+        while(toSkip >= BLOCK_SIZE) {
           if (DEBUG) {
             System.out.println("        skip whole block @ fp=" + posIn.getFilePointer());
           }
           assert posIn.getFilePointer() != lastPosBlockFP;
-          skipBlock(posIn);
-          toSkip -= blockSize;
+          ForUtil.skipBlock(posIn);
+          toSkip -= BLOCK_SIZE;
         }
         refillPositions();
         posBufferUpto = toSkip;
@@ -910,7 +899,7 @@ public final class BlockPackedPostingsRe
         posPendingFP = -1;
 
         // Force buffer refill:
-        posBufferUpto = blockSize;
+        posBufferUpto = BLOCK_SIZE;
       }
 
       if (posPendingCount > freq) {
@@ -918,11 +907,11 @@ public final class BlockPackedPostingsRe
         posPendingCount = freq;
       }
 
-      if (posBufferUpto == blockSize) {
+      if (posBufferUpto == BLOCK_SIZE) {
         refillPositions();
         posBufferUpto = 0;
       }
-      position += (int)posDeltaBuffer[posBufferUpto++];
+      position += (int) posDeltaBuffer[posBufferUpto++];
       posPendingCount--;
       if (DEBUG) {
         System.out.println("      return pos=" + position);
@@ -955,25 +944,15 @@ public final class BlockPackedPostingsRe
   final class EverythingEnum extends DocsAndPositionsEnum {
     
     private final byte[] encoded;
-    private final LongBuffer encodedBuffer;
 
-    private final long[] docDeltaBuffer = new long[blockSize];
-    private final long[] freqBuffer = new long[blockSize];
-    private final long[] posDeltaBuffer = new long[blockSize];
+    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[] payloadLengthBuffer;
     private final long[] offsetStartDeltaBuffer;
     private final long[] offsetLengthBuffer;
 
-
-    private final LongBuffer docDeltaLBuffer = LongBuffer.wrap(docDeltaBuffer);
-    private final LongBuffer freqLBuffer = LongBuffer.wrap(freqBuffer);
-    private final LongBuffer posDeltaLBuffer = LongBuffer.wrap(posDeltaBuffer);
-
-    private final LongBuffer payloadLengthLBuffer;
-    private final LongBuffer offsetStartDeltaLBuffer;
-    private final LongBuffer offsetLengthLBuffer;
-
     private byte[] payloadBytes;
     private int payloadByteUpto;
     private int payloadLength;
@@ -1044,32 +1023,25 @@ 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[blockSize*4];
-      encodedBuffer = ByteBuffer.wrap(encoded).asLongBuffer();
+      encoded = new byte[BLOCK_SIZE*4];
       indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
       if (indexHasOffsets) {
-        offsetStartDeltaBuffer = new long[blockSize];
-        offsetLengthBuffer = new long[blockSize];
-        offsetStartDeltaLBuffer = LongBuffer.wrap(offsetStartDeltaBuffer); 
-        offsetLengthLBuffer = LongBuffer.wrap(offsetLengthBuffer); 
+        offsetStartDeltaBuffer = new long[BLOCK_SIZE];
+        offsetLengthBuffer = new long[BLOCK_SIZE];
       } else {
         offsetStartDeltaBuffer = null;
-        offsetStartDeltaLBuffer = null;
         offsetLengthBuffer = null;
-        offsetLengthLBuffer = null;
         startOffset = -1;
         endOffset = -1;
       }
 
       indexHasPayloads = fieldInfo.hasPayloads();
       if (indexHasPayloads) {
-        payloadLengthBuffer = new long[blockSize];
-        payloadLengthLBuffer = LongBuffer.wrap(payloadLengthBuffer); 
+        payloadLengthBuffer = new long[BLOCK_SIZE];
         payloadBytes = new byte[128];
         payload = new BytesRef();
       } else {
         payloadLengthBuffer = null;
-        payloadLengthLBuffer = null;
         payloadBytes = null;
         payload = null;
       }
@@ -1095,9 +1067,9 @@ public final class BlockPackedPostingsRe
       posPendingFP = posTermStartFP;
       payPendingFP = payTermStartFP;
       posPendingCount = 0;
-      if (termState.totalTermFreq < blockSize) {
+      if (termState.totalTermFreq < BLOCK_SIZE) {
         lastPosBlockFP = posTermStartFP;
-      } else if (termState.totalTermFreq == blockSize) {
+      } else if (termState.totalTermFreq == BLOCK_SIZE) {
         lastPosBlockFP = -1;
       } else {
         lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset;
@@ -1106,7 +1078,7 @@ public final class BlockPackedPostingsRe
       doc = -1;
       accum = 0;
       docUpto = 0;
-      docBufferUpto = blockSize;
+      docBufferUpto = BLOCK_SIZE;
       skipped = false;
       return this;
     }
@@ -1122,35 +1094,24 @@ public final class BlockPackedPostingsRe
     }
 
     private void refillDocs() throws IOException {
+      //System.out.println("["+docFreq+"]"+" refillDoc");
       final int left = docFreq - docUpto;
       assert left > 0;
 
-      if (left >= blockSize) {
+      if (left >= BLOCK_SIZE) {
         if (DEBUG) {
           System.out.println("    fill doc block from fp=" + docIn.getFilePointer());
         }
-
-        readBlock(docIn, encoded, encodedBuffer, docDeltaLBuffer);
-
+        ForUtil.readBlock(docIn, encoded, docDeltaBuffer);
         if (DEBUG) {
           System.out.println("    fill freq block from fp=" + docIn.getFilePointer());
         }
-
-        readBlock(docIn, encoded, encodedBuffer, freqLBuffer);
+        ForUtil.readBlock(docIn, encoded, freqBuffer);
       } else {
-        // Read vInts:
         if (DEBUG) {
           System.out.println("    fill last vInt doc block from fp=" + docIn.getFilePointer());
         }
-        for(int i=0;i<left;i++) {
-          final int code = docIn.readVInt();
-          docDeltaBuffer[i] = code >>> 1;
-          if ((code & 1) != 0) {
-            freqBuffer[i] = 1;
-          } else {
-            freqBuffer[i] = docIn.readVInt();
-          }
-        }
+        ForUtil.readVIntBlock(docIn, docDeltaBuffer, freqBuffer, left, true);
       }
       docBufferUpto = 0;
     }
@@ -1205,13 +1166,13 @@ public final class BlockPackedPostingsRe
         if (DEBUG) {
           System.out.println("        bulk pos block @ fp=" + posIn.getFilePointer());
         }
-        readBlock(posIn, encoded, encodedBuffer, posDeltaLBuffer);
+        ForUtil.readBlock(posIn, encoded, posDeltaBuffer);
 
         if (indexHasPayloads) {
           if (DEBUG) {
             System.out.println("        bulk payload block @ pay.fp=" + payIn.getFilePointer());
           }
-          readBlock(payIn, encoded, encodedBuffer, payloadLengthLBuffer);
+          ForUtil.readBlock(payIn, encoded, payloadLengthBuffer);
           int numBytes = payIn.readVInt();
           if (DEBUG) {
             System.out.println("        " + numBytes + " payload bytes @ pay.fp=" + payIn.getFilePointer());
@@ -1227,42 +1188,37 @@ public final class BlockPackedPostingsRe
           if (DEBUG) {
             System.out.println("        bulk offset block @ pay.fp=" + payIn.getFilePointer());
           }
-          readBlock(payIn, encoded, encodedBuffer, offsetStartDeltaLBuffer);
-          readBlock(payIn, encoded, encodedBuffer, offsetLengthLBuffer);
+          ForUtil.readBlock(payIn, encoded, offsetStartDeltaBuffer);
+          ForUtil.readBlock(payIn, encoded, offsetLengthBuffer);
         }
       }
     }
 
     @Override
     public int nextDoc() throws IOException {
-
       if (DEBUG) {
         System.out.println("  FPR.nextDoc");
       }
-
       if (indexHasPayloads) {
         payloadByteUpto += payloadLength;
         payloadLength = 0;
       }
-
       while (true) {
         if (DEBUG) {
           System.out.println("    docUpto=" + docUpto + " (of df=" + docFreq + ") docBufferUpto=" + docBufferUpto);
         }
-
         if (docUpto == docFreq) {
           return doc = NO_MORE_DOCS;
         }
-
-        if (docBufferUpto == blockSize) {
+        //System.out.println("["+docFreq+"]"+" nextDoc");
+        if (docBufferUpto == BLOCK_SIZE) {
           refillDocs();
         }
-
         if (DEBUG) {
           System.out.println("    accum=" + accum + " docDeltaBuffer[" + docBufferUpto + "]=" + docDeltaBuffer[docBufferUpto]);
         }
-        accum += (int)docDeltaBuffer[docBufferUpto];
-        freq = (int)freqBuffer[docBufferUpto];
+        accum += docDeltaBuffer[docBufferUpto];
+        freq = (int) freqBuffer[docBufferUpto];
         posPendingCount += freq;
         docBufferUpto++;
         docUpto++;
@@ -1295,7 +1251,7 @@ public final class BlockPackedPostingsRe
       // nocommit put cheating back!  does it help?
       // nocommit use skipper!!!  it has next last doc id!!
       //if (docFreq > blockSize && target - (blockSize - docBufferUpto) - 2*blockSize > accum) {
-      if (docFreq > blockSize && target - accum > blockSize) {
+      if (docFreq > BLOCK_SIZE && target - accum > BLOCK_SIZE) {
 
         if (DEBUG) {
           System.out.println("    try skipper");
@@ -1308,7 +1264,7 @@ public final class BlockPackedPostingsRe
           }
           skipper = new BlockPackedSkipReader((IndexInput) docIn.clone(),
                                         BlockPackedPostingsWriter.maxSkipLevels,
-                                        blockSize,
+                                        BLOCK_SIZE,
                                         true,
                                         indexHasOffsets,
                                         indexHasPayloads);
@@ -1329,16 +1285,14 @@ public final class BlockPackedPostingsRe
 
         if (newDocUpto > docUpto) {
           // Skipper moved
-
           if (DEBUG) {
             System.out.println("    skipper moved to docUpto=" + newDocUpto + " vs current=" + docUpto + "; docID=" + skipper.getDoc() + " fp=" + skipper.getDocPointer() + " pos.fp=" + skipper.getPosPointer() + " pos.bufferUpto=" + skipper.getPosBufferUpto() + " pay.fp=" + skipper.getPayPointer() + " lastStartOffset=" + lastStartOffset);
           }
-
-          assert newDocUpto % blockSize == (blockSize-1): "got " + newDocUpto;
+          assert newDocUpto % BLOCK_SIZE == (BLOCK_SIZE - 1): "got " + newDocUpto;
           docUpto = newDocUpto+1;
 
-          // Force block read next:
-          docBufferUpto = blockSize;
+          // Force to read next block
+          docBufferUpto = BLOCK_SIZE;
           accum = skipper.getDoc();
           docIn.seek(skipper.getDocPointer());
           posPendingFP = skipper.getPosPointer();
@@ -1349,6 +1303,8 @@ public final class BlockPackedPostingsRe
         }
       }
 
+      // nocommit inline nextDoc here
+
       // Now scan:
       while (nextDoc() != NO_MORE_DOCS) {
         if (doc >= target) {
@@ -1377,7 +1333,7 @@ public final class BlockPackedPostingsRe
         System.out.println("      FPR.skipPositions: toSkip=" + toSkip);
       }
 
-      final int leftInBlock = blockSize - posBufferUpto;
+      final int leftInBlock = BLOCK_SIZE - posBufferUpto;
       if (toSkip < leftInBlock) {
         int end = posBufferUpto + toSkip;
         while(posBufferUpto < end) {
@@ -1394,16 +1350,16 @@ public final class BlockPackedPostingsRe
         }
       } else {
         toSkip -= leftInBlock;
-        while(toSkip >= blockSize) {
+        while(toSkip >= BLOCK_SIZE) {
           if (DEBUG) {
             System.out.println("        skip whole block @ fp=" + posIn.getFilePointer());
           }
           assert posIn.getFilePointer() != lastPosBlockFP;
-          skipBlock(posIn);
+          ForUtil.skipBlock(posIn);
 
           if (indexHasPayloads) {
             // Skip payloadLength block:
-            skipBlock(payIn);
+            ForUtil.skipBlock(payIn);
 
             // Skip payloadBytes block:
             int numBytes = payIn.readVInt();
@@ -1413,13 +1369,13 @@ public final class BlockPackedPostingsRe
           if (indexHasOffsets) {
             // Must load offset blocks merely to sum
             // up into lastStartOffset:
-            readBlock(payIn, encoded, encodedBuffer, offsetStartDeltaLBuffer);
-            readBlock(payIn, encoded, encodedBuffer, offsetLengthLBuffer);
-            for(int i=0;i<blockSize;i++) {
+            ForUtil.readBlock(payIn, encoded, offsetStartDeltaBuffer);
+            ForUtil.readBlock(payIn, encoded, offsetLengthBuffer);
+            for(int i=0;i<BLOCK_SIZE;i++) {
               lastStartOffset += offsetStartDeltaBuffer[i] + offsetLengthBuffer[i];
             }
           }
-          toSkip -= blockSize;
+          toSkip -= BLOCK_SIZE;
         }
         refillPositions();
         payloadByteUpto = 0;
@@ -1440,6 +1396,8 @@ public final class BlockPackedPostingsRe
 
       position = 0;
       payloadLength = 0;
+      // nocommit why carefully sum up lastStartOffset above
+      // only to set it to 0 now?
       lastStartOffset = 0;
     }
 
@@ -1464,7 +1422,7 @@ public final class BlockPackedPostingsRe
         }
 
         // Force buffer refill:
-        posBufferUpto = blockSize;
+        posBufferUpto = BLOCK_SIZE;
       }
 
       if (indexHasPayloads) {
@@ -1482,19 +1440,19 @@ public final class BlockPackedPostingsRe
         posPendingCount = freq;
       }
 
-      if (posBufferUpto == blockSize) {
+      if (posBufferUpto == BLOCK_SIZE) {
         refillPositions();
         posBufferUpto = 0;
       }
-      position += (int)posDeltaBuffer[posBufferUpto];
+      position += posDeltaBuffer[posBufferUpto];
 
       if (indexHasPayloads) {
-        payloadLength = (int)payloadLengthBuffer[posBufferUpto];
+        payloadLength = (int) payloadLengthBuffer[posBufferUpto];
       }
 
       if (indexHasOffsets) {
-        startOffset = lastStartOffset + (int)offsetStartDeltaBuffer[posBufferUpto];
-        endOffset = startOffset + (int)offsetLengthBuffer[posBufferUpto];
+        startOffset = lastStartOffset + (int) offsetStartDeltaBuffer[posBufferUpto];
+        endOffset = startOffset + (int) offsetLengthBuffer[posBufferUpto];
         lastStartOffset = startOffset;
       }
 

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=1370267&r1=1370266&r2=1370267&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 Tue Aug  7 13:56:30 2012
@@ -17,10 +17,9 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+
 import java.io.IOException;
-import java.nio.ByteBuffer;      
-import java.nio.IntBuffer;
-import java.nio.LongBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -28,8 +27,8 @@ import org.apache.lucene.codecs.CodecUti
 import org.apache.lucene.codecs.PostingsWriterBase;
 import org.apache.lucene.codecs.TermStats;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
@@ -38,8 +37,16 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
-// nocommit javadocs
 
+/**
+ * Concrete class that writes docId(maybe frq,pos,offset,payloads) list
+ * with postings format.
+ *
+ * Postings list for each term will be stored separately. 
+ *
+ * @see BlockPackedSkipWriter for details about skipping setting and postings layout.
+ *
+ */
 public final class BlockPackedPostingsWriter extends PostingsWriterBase {
 
   private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
@@ -61,10 +68,6 @@ public final class BlockPackedPostingsWr
   final IndexOutput posOut;
   final IndexOutput payOut;
 
-  static final int DEFAULT_BLOCK_SIZE = 128;
-
-  final int blockSize;
-
   private IndexOutput termsOut;
 
   // How current field indexes postings:
@@ -80,43 +83,35 @@ public final class BlockPackedPostingsWr
 
   final long[] docDeltaBuffer;
   final long[] freqBuffer;
-  final LongBuffer docDeltaLBuffer;
-  final LongBuffer freqLBuffer;
   private int docBufferUpto;
 
   final long[] posDeltaBuffer;
   final long[] payloadLengthBuffer;
   final long[] offsetStartDeltaBuffer;
   final long[] offsetLengthBuffer;
-  final LongBuffer posDeltaLBuffer;
-  final LongBuffer payloadLengthLBuffer;
-  final LongBuffer offsetStartDeltaLBuffer;
-  final LongBuffer offsetLengthLBuffer;
   private int posBufferUpto;
 
   private byte[] payloadBytes;
   private int payloadByteUpto;
 
   private int lastBlockDocID;
-  private boolean saveNextPosBlock;
   private long lastBlockPosFP;
   private long lastBlockPayFP;
   private int lastBlockPosBufferUpto;
   private int lastBlockStartOffset;
   private int lastBlockPayloadByteUpto;
+
   private int lastDocID;
   private int lastPosition;
   private int lastStartOffset;
   private int docCount;
 
   final byte[] encoded;
-  final LongBuffer encodedBuffer;
 
   private final BlockPackedSkipWriter skipWriter;
   
-  public BlockPackedPostingsWriter(SegmentWriteState state, int blockSize) throws IOException {
+  public BlockPackedPostingsWriter(SegmentWriteState state) throws IOException {
     super();
-    this.blockSize = blockSize;
 
     docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockPackedPostingsFormat.DOC_EXTENSION),
                                           state.context);
@@ -126,32 +121,25 @@ public final class BlockPackedPostingsWr
     try {
       CodecUtil.writeHeader(docOut, DOC_CODEC, VERSION_CURRENT);
       if (state.fieldInfos.hasProx()) {
-        posDeltaBuffer = new long[blockSize];
-        posDeltaLBuffer = LongBuffer.wrap(posDeltaBuffer);
+        posDeltaBuffer = new long[BLOCK_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[blockSize];
-          payloadLengthLBuffer = LongBuffer.wrap(payloadLengthBuffer);
+          payloadLengthBuffer = new long[BLOCK_SIZE];
         } else {
           payloadBytes = null;
           payloadLengthBuffer = null;
-          payloadLengthLBuffer = null;
         }
 
         if (state.fieldInfos.hasOffsets()) {
-          offsetStartDeltaBuffer = new long[blockSize];
-          offsetLengthBuffer = new long[blockSize];
-          offsetStartDeltaLBuffer = LongBuffer.wrap(offsetStartDeltaBuffer);
-          offsetLengthLBuffer = LongBuffer.wrap(offsetLengthBuffer);
+          offsetStartDeltaBuffer = new long[BLOCK_SIZE];
+          offsetLengthBuffer = new long[BLOCK_SIZE];
         } else {
           offsetStartDeltaBuffer = null;
           offsetLengthBuffer = null;
-          offsetStartDeltaLBuffer = null;
-          offsetLengthLBuffer = null;
         }
 
         if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
@@ -165,10 +153,6 @@ public final class BlockPackedPostingsWr
         offsetStartDeltaBuffer = null;
         offsetLengthBuffer = null;
         payloadBytes = null;
-        posDeltaLBuffer = null;
-        payloadLengthLBuffer = null;
-        offsetStartDeltaLBuffer = null;
-        offsetLengthLBuffer = null;
       }
       this.payOut = payOut;
       this.posOut = posOut;
@@ -179,27 +163,24 @@ public final class BlockPackedPostingsWr
       }
     }
 
-    docDeltaBuffer = new long[blockSize];
-    freqBuffer = new long[blockSize];
-    docDeltaLBuffer = LongBuffer.wrap(docDeltaBuffer);
-    freqLBuffer = LongBuffer.wrap(freqBuffer);
+    docDeltaBuffer = new long[BLOCK_SIZE];
+    freqBuffer = new long[BLOCK_SIZE];
 
-    skipWriter = new BlockPackedSkipWriter(blockSize,
-                                     maxSkipLevels, 
+    skipWriter = new BlockPackedSkipWriter(maxSkipLevels,
+                                     BlockPackedPostingsFormat.BLOCK_SIZE, 
                                      state.segmentInfo.getDocCount(),
                                      docOut,
                                      posOut,
                                      payOut);
 
-    encoded = new byte[blockSize*4];
-    encodedBuffer = ByteBuffer.wrap(encoded).asLongBuffer();
+    encoded = new byte[BLOCK_SIZE*4];
   }
 
   @Override
   public void start(IndexOutput termsOut) throws IOException {
     this.termsOut = termsOut;
     CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
-    termsOut.writeVInt(blockSize);
+    termsOut.writeVInt(BLOCK_SIZE);
   }
 
   @Override
@@ -221,91 +202,53 @@ public final class BlockPackedPostingsWr
         payTermStartFP = payOut.getFilePointer();
       }
     }
-    lastBlockDocID = -1;
     lastDocID = 0;
+    lastBlockDocID = -1;
     if (DEBUG) {
       System.out.println("FPW.startTerm startFP=" + docTermStartFP);
     }
     skipWriter.resetSkip();
   }
 
-  private void writeBlock(LongBuffer buffer, IndexOutput out) throws IOException {
-    final int header = ForUtil.compress(buffer, encodedBuffer);
-    out.writeVInt(header);
-    out.writeBytes(encoded, ForUtil.getEncodedSize(header));
-  }
-
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
     if (DEBUG) {
-      System.out.println("FPW.startDoc docID=" + docID);
-    }
-
-    // nocommit do this in finishDoc... but does it fail...?
-    // is it not always called...?
-    if (posOut != null && saveNextPosBlock) {
-      lastBlockPosFP = posOut.getFilePointer();
-      if (payOut != null) {
-        lastBlockPayFP = payOut.getFilePointer();
-      }
-      lastBlockPosBufferUpto = posBufferUpto;
-      lastBlockStartOffset = lastStartOffset;
-      lastBlockPayloadByteUpto = payloadByteUpto;
-      saveNextPosBlock = false;
-      if (DEBUG) {
-        System.out.println("  now save lastBlockPosFP=" + lastBlockPosFP + " lastBlockPosBufferUpto=" + lastBlockPosBufferUpto + " lastBlockPayloadByteUpto=" + lastBlockPayloadByteUpto);
-      }
+      System.out.println("FPW.startDoc docID["+docBufferUpto+"]=" + docID);
     }
 
     final int docDelta = docID - lastDocID;
+
     if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
       throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")");
     }
-    lastDocID = docID;
 
     docDeltaBuffer[docBufferUpto] = docDelta;
-    if (DEBUG) {
-      System.out.println("  docDeltaBuffer[" + docBufferUpto + "]=" + docDelta);
-    }
+//    if (DEBUG) {
+//      System.out.println("  docDeltaBuffer[" + docBufferUpto + "]=" + docDelta);
+//    }
     if (fieldHasFreqs) {
       freqBuffer[docBufferUpto] = termDocFreq;
     }
-
     docBufferUpto++;
     docCount++;
 
-    if (docBufferUpto == blockSize) {
-      // nocommit maybe instead of buffering skip before
-      // writing a block based on last block's end data
-      // ... we could buffer after writing the block?  only
-      // iffiness with that approach is it could be a
-      // pointlness skip?  like we may stop adding docs
-      // right after that, then we have skip point AFTER
-      // last doc.  the thing is, in finishTerm we are
-      // already sometimes adding a skip point AFTER the
-      // last doc?
-      if (lastBlockDocID != -1) {
-        if (DEBUG) {
-          System.out.println("  bufferSkip at writeBlock: lastDocID=" + lastBlockDocID + " docCount=" + (docCount-blockSize));
-        }
-        skipWriter.bufferSkip(lastBlockDocID, docCount-blockSize, lastBlockPosFP, lastBlockPayFP, lastBlockPosBufferUpto, lastBlockStartOffset, lastBlockPayloadByteUpto);
-      }
-      lastBlockDocID = docID;
-      saveNextPosBlock = true;
-
+    if (docBufferUpto == BLOCK_SIZE) {
       if (DEBUG) {
         System.out.println("  write docDelta block @ fp=" + docOut.getFilePointer());
       }
-      writeBlock(docDeltaLBuffer, docOut);
+      ForUtil.writeBlock(docDeltaBuffer, encoded, docOut);
       if (fieldHasFreqs) {
         if (DEBUG) {
           System.out.println("  write freq block @ fp=" + docOut.getFilePointer());
         }
-        writeBlock(freqLBuffer, docOut);
+        ForUtil.writeBlock(freqBuffer, encoded, docOut);
       }
-      docBufferUpto = 0;
+      // NOTE: don't set docBufferUpto back to 0 here;
+      // finishDoc will do so (because it needs to see that
+      // the block was filled so it can save skip data)
     }
 
+    lastDocID = docID;
     lastPosition = 0;
     lastStartOffset = 0;
   }
@@ -313,9 +256,9 @@ public final class BlockPackedPostingsWr
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
-    if (DEBUG) {
-      System.out.println("FPW.addPosition pos=" + position + " posBufferUpto=" + posBufferUpto + (fieldHasPayloads ? " payloadByteUpto=" + payloadByteUpto: ""));
-    }
+//    if (DEBUG) {
+//      System.out.println("FPW.addPosition pos=" + position + " posBufferUpto=" + posBufferUpto + (fieldHasPayloads ? " payloadByteUpto=" + payloadByteUpto: ""));
+//    }
     posDeltaBuffer[posBufferUpto] = position - lastPosition;
     if (fieldHasPayloads) {
       if (payload == null || payload.length == 0) {
@@ -341,28 +284,60 @@ public final class BlockPackedPostingsWr
     
     posBufferUpto++;
     lastPosition = position;
-    if (posBufferUpto == blockSize) {
+    if (posBufferUpto == BLOCK_SIZE) {
       if (DEBUG) {
         System.out.println("  write pos bulk block @ fp=" + posOut.getFilePointer());
       }
-      writeBlock(posDeltaLBuffer, posOut);
+      ForUtil.writeBlock(posDeltaBuffer, encoded, posOut);
 
       if (fieldHasPayloads) {
-        writeBlock(payloadLengthLBuffer, payOut);
+        ForUtil.writeBlock(payloadLengthBuffer, encoded, payOut);
         payOut.writeVInt(payloadByteUpto);
         payOut.writeBytes(payloadBytes, 0, payloadByteUpto);
         payloadByteUpto = 0;
       }
       if (fieldHasOffsets) {
-        writeBlock(offsetStartDeltaLBuffer, payOut);
-        writeBlock(offsetLengthLBuffer, payOut);
+        ForUtil.writeBlock(offsetStartDeltaBuffer, encoded, payOut);
+        ForUtil.writeBlock(offsetLengthBuffer, encoded, payOut);
       }
       posBufferUpto = 0;
     }
   }
 
   @Override
-  public void finishDoc() {
+  public void finishDoc() throws IOException {
+    // Have collected a block of docs, and get a new doc. 
+    // Should write skip data as well as postings list for
+    // current block
+
+    if (lastBlockDocID != -1 && docBufferUpto == 1) {
+      // nocomit move to startDoc?  ie we can write skip
+      // data as soon as the next doc starts...
+      if (DEBUG) {
+        System.out.println("  bufferSkip at writeBlock: lastDocID=" + lastBlockDocID + " docCount=" + (docCount-1));
+      }
+      skipWriter.bufferSkip(lastBlockDocID, docCount-1, lastBlockPosFP, lastBlockPayFP, lastBlockPosBufferUpto, lastBlockStartOffset, lastBlockPayloadByteUpto);
+    }
+
+    // Since we don't know df for current term, we had to buffer
+    // those skip data for each block, and when a new doc comes, 
+    // write them to skip file.
+    if (docBufferUpto == BLOCK_SIZE) {
+      lastBlockDocID = lastDocID;
+      if (posOut != null) {
+        if (payOut != null) {
+          lastBlockPayFP = payOut.getFilePointer();
+        }
+        lastBlockPosFP = posOut.getFilePointer();
+        lastBlockPosBufferUpto = posBufferUpto;
+        lastBlockStartOffset = lastStartOffset;
+        lastBlockPayloadByteUpto = payloadByteUpto;
+      }
+      if (DEBUG) {
+        System.out.println("  docBufferUpto="+docBufferUpto+" now get lastBlockDocID="+lastBlockDocID+" lastBlockPosFP=" + lastBlockPosFP + " lastBlockPosBufferUpto=" + lastBlockPosBufferUpto + " lastBlockPayloadByteUpto=" + lastBlockPayloadByteUpto);
+      }
+      docBufferUpto = 0;
+    }
   }
 
   private static class PendingTerm {
@@ -386,7 +361,6 @@ public final class BlockPackedPostingsWr
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(TermStats stats) throws IOException {
-
     assert stats.docFreq > 0;
 
     // TODO: wasteful we are counting this (counting # docs
@@ -397,19 +371,6 @@ public final class BlockPackedPostingsWr
       System.out.println("FPW.finishTerm docFreq=" + stats.docFreq);
     }
 
-    // nocommit silly that skipper must write skip when we no
-    // postings come after it, but if we don't do this, skip
-    // reader incorrectly thinks it can read another level 0
-    // skip entry here!:
-    //if (docCount > blockSize && docBufferUpto > 0) {
-    if (docCount > blockSize) {
-      final int lastDocCount = blockSize*(docCount/blockSize);
-      if (DEBUG) {
-        System.out.println("  bufferSkip at finishTerm: lastDocID=" + lastBlockDocID + " docCount=" + lastDocCount);
-      }
-      skipWriter.bufferSkip(lastBlockDocID, lastDocCount, lastBlockPosFP, lastBlockPayFP, lastBlockPosBufferUpto, lastBlockStartOffset, lastBlockPayloadByteUpto);
-    }
-
     if (DEBUG) {
       if (docBufferUpto > 0) {
         System.out.println("  write doc/freq vInt block (count=" + docBufferUpto + ") at fp=" + docOut.getFilePointer() + " docTermStartFP=" + docTermStartFP);
@@ -418,8 +379,8 @@ public final class BlockPackedPostingsWr
 
     // vInt encode the remaining doc deltas and freqs:
     for(int i=0;i<docBufferUpto;i++) {
-      final int docDelta = (int)docDeltaBuffer[i];
-      final int freq = (int)freqBuffer[i];
+      final int docDelta = (int) docDeltaBuffer[i];
+      final int freq = (int) freqBuffer[i];
       if (!fieldHasFreqs) {
         docOut.writeVInt(docDelta);
       } else if (freqBuffer[i] == 1) {
@@ -440,7 +401,7 @@ public final class BlockPackedPostingsWr
       }
 
       assert stats.totalTermFreq != -1;
-      if (stats.totalTermFreq > blockSize) {
+      if (stats.totalTermFreq > BLOCK_SIZE) {
         lastPosBlockOffset = (int) (posOut.getFilePointer() - posTermStartFP);
       } else {
         lastPosBlockOffset = -1;
@@ -457,9 +418,9 @@ public final class BlockPackedPostingsWr
         int lastPayloadLength = -1;
         int payloadBytesReadUpto = 0;
         for(int i=0;i<posBufferUpto;i++) {
-          final int posDelta = (int)posDeltaBuffer[i];
+          final int posDelta = (int) posDeltaBuffer[i];
           if (fieldHasPayloads) {
-            final int payloadLength = (int)payloadLengthBuffer[i];
+            final int payloadLength = (int) payloadLengthBuffer[i];
             if (payloadLength != lastPayloadLength) {
               lastPayloadLength = payloadLength;
               posOut.writeVInt((posDelta<<1)|1);
@@ -487,8 +448,8 @@ public final class BlockPackedPostingsWr
             if (DEBUG) {
               System.out.println("          write offset @ pos.fp=" + posOut.getFilePointer());
             }
-            posOut.writeVInt((int)offsetStartDeltaBuffer[i]);
-            posOut.writeVInt((int)offsetLengthBuffer[i]);
+            posOut.writeVInt((int) offsetStartDeltaBuffer[i]);
+            posOut.writeVInt((int) offsetLengthBuffer[i]);
           }
         }
 
@@ -505,7 +466,7 @@ public final class BlockPackedPostingsWr
     }
 
     int skipOffset;
-    if (docCount > blockSize) {
+    if (docCount > BLOCK_SIZE) {
       skipOffset = (int) (skipWriter.writeSkip(docOut)-docTermStartFP);
       
       if (DEBUG) {
@@ -519,7 +480,7 @@ public final class BlockPackedPostingsWr
     }
 
     long payStartFP;
-    if (stats.totalTermFreq >= blockSize) {
+    if (stats.totalTermFreq >= BLOCK_SIZE) {
       payStartFP = payTermStartFP;
     } else {
       payStartFP = -1;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java Tue Aug  7 13:56:30 2012
@@ -24,14 +24,35 @@ import org.apache.lucene.codecs.MultiLev
 import org.apache.lucene.store.IndexInput;
 
 /**
- * Implements the skip list reader for the 4.0 posting list format
+ * Implements the skip list reader for block postings format
  * that stores positions and payloads.
  * 
- * @see Lucene40PostingsFormat
- * @lucene.experimental
+ * Although this skipper uses MultiLevelSkipListReader as an interface, 
+ * its definition of skip position will be a little different. 
+ *
+ * For example, when skipInterval = blockSize = 3, df = 2*skipInterval = 6, 
+ * 
+ * 0 1 2 3 4 5
+ * d d d d d d    (posting list)
+ *     ^     ^    (skip point in MultiLeveSkipWriter)
+ *       ^        (skip point in BlockSkipWriter)
+ *
+ * In this case, MultiLevelSkipListReader will use the last document as a skip point, 
+ * while BlockSkipReader should assume no skip point will comes. 
+ *
+ * If we use the interface directly in BlockSkipReader, it may silly try to read 
+ * another skip data after the only skip point is loaded. 
+ *
+ * To illustrate this, we can call skipTo(d[5]), since skip point d[3] has smaller docId,
+ * and numSkipped+blockSize== df, the MultiLevelSkipListReader will assume the skip list
+ * isn't exhausted yet, and try to load a non-existed skip point
+ *
+ * Therefore, we'll trim df before passing it to the interface. see trim(int)
+ *
  */
 final class BlockPackedSkipReader extends MultiLevelSkipListReader {
   private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
+  private final int blockSize;
 
   private long docPointer[];
   private long posPointer[];
@@ -47,8 +68,9 @@ final class BlockPackedSkipReader extend
   private long lastDocPointer;
   private int lastPosBufferUpto;
 
-  public BlockPackedSkipReader(IndexInput skipStream, int maxSkipLevels, int skipInterval, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
-    super(skipStream, maxSkipLevels, skipInterval);
+  public BlockPackedSkipReader(IndexInput skipStream, int maxSkipLevels, int blockSize, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
+    super(skipStream, maxSkipLevels, blockSize, 8);
+    this.blockSize = blockSize;
     docPointer = new long[maxSkipLevels];
     if (hasPos) {
       posPointer = new long[maxSkipLevels];
@@ -73,8 +95,21 @@ final class BlockPackedSkipReader extend
     }
   }
 
+  /**
+   * Trim original docFreq to tell skipReader read proper number of skip points.
+   *
+   * Since our definition in BlockSkip* is a little different from MultiLevelSkip*
+   * This trimed docFreq will prevent skipReader from:
+   * 1. silly reading a non-existed skip point after the last block boundary
+   * 2. moving into the vInt block
+   *
+   */
+  protected int trim(int df) {
+    return df % blockSize == 0? df - 1: df;
+  }
+
   public void init(long skipPointer, long docBasePointer, long posBasePointer, long payBasePointer, int df) {
-    super.init(skipPointer, df);
+    super.init(skipPointer, trim(df));
     lastDocPointer = docBasePointer;
     lastPosPointer = posBasePointer;
     lastPayPointer = payBasePointer;

Modified: lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java?rev=1370267&r1=1370266&r2=1370267&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java Tue Aug  7 13:56:30 2012
@@ -23,10 +23,26 @@ import java.util.Arrays;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.codecs.MultiLevelSkipListWriter;
 
-// nocommit do we need more frequent skips at level > 0?
-// 128*128 is immense?  may need to decouple
-// baseSkipInterval & theRestSkipInterval?
-
+/**
+* Write skip lists with multiple levels, and support skip within block ints.
+*
+* Assume that docFreq = 28, skipInterval = blockSize = 12
+*
+*  |       block#0       | |      block#1        | |vInts|
+*  d d d d d d d d d d d d d d d d d d d d d d d d d d d d (posting list)
+*                          ^                       ^       (level 0 skip point)
+*
+* Note that skipWriter will ignore first document in block#0, since 
+* it is useless as a skip point.  Also, we'll never skip into the vInts
+* block, only record skip data at the start its start point(if it exist).
+*
+* For each skip point, we will record: 
+* 1. lastDocID, 
+* 2. its related file points(position, payload), 
+* 3. related numbers or uptos(position, payload).
+* 4. start offset.
+*
+*/
 final class BlockPackedSkipWriter extends MultiLevelSkipListWriter {
   private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
   
@@ -52,8 +68,10 @@ final class BlockPackedSkipWriter extend
   private boolean fieldHasOffsets;
   private boolean fieldHasPayloads;
 
-  public BlockPackedSkipWriter(int skipInterval, int maxSkipLevels, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
-    super(skipInterval, maxSkipLevels, docCount);
+  public BlockPackedSkipWriter(int maxSkipLevels, int blockSize, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
+    // nocommit figure out what skipMultiplier is best (4 is
+    // total guess):
+    super(blockSize, 8, maxSkipLevels, docCount);
     this.docOut = docOut;
     this.posOut = posOut;
     this.payOut = payOut;

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=1370267&r1=1370266&r2=1370267&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 Tue Aug  7 13:56:30 2012
@@ -16,115 +16,127 @@ package org.apache.lucene.codecs.blockpa
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.blockpacked.BlockPackedPostingsFormat.BLOCK_SIZE;
+
 import java.io.IOException;
-import java.nio.LongBuffer;
-import java.nio.IntBuffer;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
 
+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.Reader;
-import org.apache.lucene.util.packed.PackedInts.Writer;
-import org.apache.lucene.util.packed.PackedInts.Mutable;
-import org.apache.lucene.util.packed.PackedInts.Encoder;
-import org.apache.lucene.util.packed.PackedInts.Decoder;
 
 /**
  * Encode all values in normal area with fixed bit width, 
  * which is determined by the max value in this block.
  */
 public class ForUtil {
-  protected 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};
-
-  /** Compress given int[] into output stream, with For format
-   */
-  public static int compress(final LongBuffer data, LongBuffer packed) throws IOException {
-    int numBits=getNumBits(data.array());
-
-    if (numBits == 0) { // when block is equal, save the value once
-      packed.put(0, data.get(0)<<32); // java uses big endian for LongBuffer impl 
-      return (getHeader(1,numBits));
+  
+  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];
+  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();
     }
-
-    PackedInts.Format format = PackedInts.fastestFormatAndBits(128, numBits, PackedInts.FASTEST).format;
-    PackedInts.Encoder encoder = PackedInts.getEncoder(format, PackedInts.VERSION_CURRENT, numBits);
-    int perIter = encoder.values();
-    int iters = 128/perIter;
-    int nblocks = encoder.blocks()*iters;
-    assert 128 % perIter == 0;
-
-    packed.rewind();
-    data.rewind();
-
-    encoder.encode(data, packed, iters);
-
-    int encodedSize = nblocks*2;
-    return getHeader(encodedSize,numBits);
   }
 
-  /** Decompress given ouput stream into int array.
+  /**
+   * Write a block of data (<code>For</code> format).
+   *
+   * @param data     the data to write
+   * @param encoded  a buffer to use to encode data
+   * @param out      the destination output
+   * @throws IOException
    */
-  public static void decompress(LongBuffer data, LongBuffer packed, int header) throws IOException {
-    // nocommit assert header isn't "malformed", ie besides
-    // numBytes / bit-width there is nothing else!
-    
-    packed.rewind();
-    data.rewind();
-    int numBits = ((header >> 8) & MASK[6]);
-
-    if (numBits == 0) {
-      Arrays.fill(data.array(), (int)(packed.get(0)>>>32));
-      return;
-    }
-
-    PackedInts.Format format = PackedInts.fastestFormatAndBits(128, numBits, PackedInts.FASTEST).format;
-    PackedInts.Decoder decoder = PackedInts.getDecoder(format, PackedInts.VERSION_CURRENT, numBits);
-    int perIter = decoder.values();
-    int iters = 128/perIter;
-    int nblocks = decoder.blocks()*iters;
-    assert 128 % perIter == 0;
-
-    decoder.decode(packed, data, iters);
+  static void writeBlock(long[] data, byte[] encoded, IndexOutput out) throws IOException {
+    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 int encodedSize = encoder.blockCount() * iters; // number of 64-bits blocks
+    assert encodedSize > 0 && encodedSize <= BLOCK_SIZE / 2 : encodedSize;
+
+    out.writeByte((byte) numBits);
+    out.writeByte((byte) encodedSize);
+
+    encoder.encode(data, 0, encoded, 0, iters);
+    out.writeBytes(encoded, encodedSize << 3);
+  }
+
+  /**
+   * Read the next block of data (<code>For</code> format).
+   *
+   * @param in        the input to use to read data
+   * @param encoded   a buffer that can be used to store encoded data
+   * @param decoded   where to write decoded data
+   * @throws IOException
+   */
+  static void readBlock(IndexInput in, byte[] encoded, long[] decoded) throws IOException {
+    final int numBits = in.readByte(); // no mask because should be <= 32
+    final int encodedSize = in.readByte(); // no mask because should be <= 64
+    assert numBits > 0 && numBits <= 32 : numBits;
+    assert encodedSize > 0 && encodedSize <= BLOCK_SIZE / 2 : encodedSize; // because blocks are 64-bits and decoded values are 32-bits at most
+
+    in.readBytes(encoded, 0, encodedSize << 3);
+
+    final PackedInts.Decoder decoder = DECODERS[numBits];
+    final int iters = ITERATIONS[numBits];
+    assert iters * decoder.valueCount() == BLOCK_SIZE;
+    assert iters * decoder.blockCount() == encodedSize;
+
+    decoder.decode(encoded, 0, decoded, 0, iters);
+  }
+
+  /**
+   * Skip the next block of data.
+   *
+   * @param in      the input where to read data
+   * @throws IOException
+   */
+  static void skipBlock(IndexInput in) throws IOException {
+    // see readBlock for comments
+    final int numBits = in.readByte();
+    final int encodedSize = in.readByte();
+    assert numBits > 0 && numBits <= 32 : numBits;
+    assert encodedSize > 0 && encodedSize <= BLOCK_SIZE / 2 : encodedSize;
+    in.seek(in.getFilePointer() + (encodedSize << 3));
   }
 
-  static int getNumBits(final long[] data) {
-    if (isAllEqual(data)) {
-      return 0;
-    }
-    int size=data.length;
-    int optBits=1;
-    for (int i=0; i<size; ++i) {
-      while ((data[i] & ~MASK[optBits]) != 0) {
-        optBits++;
+  /**
+   * Read values that have been written using variable-length encoding instead of bit-packing.
+   */
+  static void readVIntBlock(IndexInput docIn, long[] docBuffer, long[] freqBuffer, int num, boolean indexHasFreq) throws IOException {
+    if (indexHasFreq) {
+      for(int i=0;i<num;i++) {
+        final int code = docIn.readVInt();
+        docBuffer[i] = code >>> 1;
+        if ((code & 1) != 0) {
+          freqBuffer[i] = 1;
+        } else {
+          freqBuffer[i] = docIn.readVInt();
+        }
+      }
+    } else {
+      for(int i=0;i<num;i++) {
+        docBuffer[i] = docIn.readVInt();
       }
     }
-    return optBits;
   }
 
-  protected static boolean isAllEqual(final long[] data) {
-    int len = data.length;
-    long v = data[0];
-    for (int i=1; i<len; i++) {
-      if (data[i] != v) {
-        return false;
-      }
+  /**
+   * Compute the number of bits required to serialize any of the longs in <code>data</code>.
+   */
+  private static int bitsRequired(final long[] data) {
+    long or = 0;
+    for (int i = 0; i < data.length; ++i) {
+      or |= data[i];
     }
-    return true;
-  }
-  static int getHeader(int encodedSize, int numBits) {
-    return  (encodedSize)
-          | ((numBits) << 8);
-  }
-  public static int getEncodedSize(int header) {
-    return ((header & MASK[8]))*4;
-  }
-  public static int getNumBits(int header) {
-    return ((header >> 8) & MASK[6]);
+    return PackedInts.bitsRequired(or);
   }
+
 }