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

svn commit: r1367338 [2/2] - in /lucene/dev/branches/pforcodec_3892/lucene: core/src/java/org/apache/lucene/codecs/block/ core/src/java/org/apache/lucene/codecs/blockpacked/ core/src/java/org/apache/lucene/codecs/pfor/ core/src/resources/META-INF/servi...

Added: 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=1367338&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipReader.java Mon Jul 30 22:15:06 2012
@@ -0,0 +1,205 @@
+package org.apache.lucene.codecs.blockpacked;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.codecs.MultiLevelSkipListReader;
+import org.apache.lucene.store.IndexInput;
+
+/**
+ * Implements the skip list reader for the 4.0 posting list format
+ * that stores positions and payloads.
+ * 
+ * @see Lucene40PostingsFormat
+ * @lucene.experimental
+ */
+final class BlockPackedSkipReader extends MultiLevelSkipListReader {
+  private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
+
+  private long docPointer[];
+  private long posPointer[];
+  private long payPointer[];
+  private int posBufferUpto[];
+  private int endOffset[];
+  private int payloadByteUpto[];
+
+  private long lastPosPointer;
+  private long lastPayPointer;
+  private int lastEndOffset;
+  private int lastPayloadByteUpto;
+  private long lastDocPointer;
+  private int lastPosBufferUpto;
+
+  public BlockPackedSkipReader(IndexInput skipStream, int maxSkipLevels, int skipInterval, boolean hasPos, boolean hasOffsets, boolean hasPayloads) {
+    super(skipStream, maxSkipLevels, skipInterval);
+    docPointer = new long[maxSkipLevels];
+    if (hasPos) {
+      posPointer = new long[maxSkipLevels];
+      posBufferUpto = new int[maxSkipLevels];
+      if (hasPayloads) {
+        payloadByteUpto = new int[maxSkipLevels];
+      } else {
+        payloadByteUpto = null;
+      }
+      if (hasOffsets) {
+        endOffset = new int[maxSkipLevels];
+      } else {
+        endOffset = null;
+      }
+      if (hasOffsets || hasPayloads) {
+        payPointer = new long[maxSkipLevels];
+      } else {
+        payPointer = null;
+      }
+    } else {
+      posPointer = null;
+    }
+  }
+
+  public void init(long skipPointer, long docBasePointer, long posBasePointer, long payBasePointer, int df) {
+    super.init(skipPointer, df);
+    lastDocPointer = docBasePointer;
+    lastPosPointer = posBasePointer;
+    lastPayPointer = payBasePointer;
+
+    Arrays.fill(docPointer, docBasePointer);
+    if (posPointer != null) {
+      Arrays.fill(posPointer, posBasePointer);
+      if (payPointer != null) {
+        Arrays.fill(payPointer, payBasePointer);
+      }
+    } else {
+      assert posBasePointer == 0;
+    }
+  }
+
+  /** Returns the doc pointer of the doc to which the last call of 
+   * {@link MultiLevelSkipListReader#skipTo(int)} has skipped.  */
+  public long getDocPointer() {
+    return lastDocPointer;
+  }
+
+  public long getPosPointer() {
+    return lastPosPointer;
+  }
+
+  public int getPosBufferUpto() {
+    return lastPosBufferUpto;
+  }
+
+  public long getPayPointer() {
+    return lastPayPointer;
+  }
+
+  public int getEndOffset() {
+    return lastEndOffset;
+  }
+
+  public int getPayloadByteUpto() {
+    return lastPayloadByteUpto;
+  }
+
+  @Override
+  protected void seekChild(int level) throws IOException {
+    super.seekChild(level);
+    if (DEBUG) {
+      System.out.println("seekChild level=" + level);
+    }
+    docPointer[level] = lastDocPointer;
+    if (posPointer != null) {
+      posPointer[level] = lastPosPointer;
+      posBufferUpto[level] = lastPosBufferUpto;
+      if (endOffset != null) {
+        endOffset[level] = lastEndOffset;
+      }
+      if (payloadByteUpto != null) {
+        payloadByteUpto[level] = lastPayloadByteUpto;
+      }
+      if (payPointer != null) {
+        payPointer[level] = lastPayPointer;
+      }
+    }
+  }
+  
+  @Override
+  protected void setLastSkipData(int level) {
+    super.setLastSkipData(level);
+    lastDocPointer = docPointer[level];
+    if (DEBUG) {
+      System.out.println("setLastSkipData level=" + level);
+      System.out.println("  lastDocPointer=" + lastDocPointer);
+    }
+    if (posPointer != null) {
+      lastPosPointer = posPointer[level];
+      lastPosBufferUpto = posBufferUpto[level];
+      if (DEBUG) {
+        System.out.println("  lastPosPointer=" + lastPosPointer + " lastPosBUfferUpto=" + lastPosBufferUpto);
+      }
+      if (payPointer != null) {
+        lastPayPointer = payPointer[level];
+      }
+      if (endOffset != null) {
+        lastEndOffset = endOffset[level];
+      }
+      if (payloadByteUpto != null) {
+        lastPayloadByteUpto = payloadByteUpto[level];
+      }
+    }
+  }
+
+  @Override
+  protected int readSkipData(int level, IndexInput skipStream) throws IOException {
+    if (DEBUG) {
+      System.out.println("readSkipData level=" + level);
+    }
+    int delta = skipStream.readVInt();
+    if (DEBUG) {
+      System.out.println("  delta=" + delta);
+    }
+    docPointer[level] += skipStream.readVInt();
+    if (DEBUG) {
+      System.out.println("  docFP=" + docPointer[level]);
+    }
+
+    if (posPointer != null) {
+      posPointer[level] += skipStream.readVInt();
+      if (DEBUG) {
+        System.out.println("  posFP=" + posPointer[level]);
+      }
+      posBufferUpto[level] = skipStream.readVInt();
+      if (DEBUG) {
+        System.out.println("  posBufferUpto=" + posBufferUpto[level]);
+      }
+
+      if (payloadByteUpto != null) {
+        payloadByteUpto[level] = skipStream.readVInt();
+      }
+
+      if (endOffset != null) {
+        endOffset[level] += skipStream.readVInt();
+      }
+
+      if (payPointer != null) {
+        payPointer[level] += skipStream.readVInt();
+      }
+    }
+    return delta;
+  }
+}

Added: 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=1367338&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/BlockPackedSkipWriter.java Mon Jul 30 22:15:06 2012
@@ -0,0 +1,147 @@
+package org.apache.lucene.codecs.blockpacked;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+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?
+
+final class BlockPackedSkipWriter extends MultiLevelSkipListWriter {
+  private boolean DEBUG = BlockPackedPostingsReader.DEBUG;
+  
+  private int[] lastSkipDoc;
+  private long[] lastSkipDocPointer;
+  private long[] lastSkipPosPointer;
+  private long[] lastSkipPayPointer;
+  private int[] lastEndOffset;
+  private int[] lastPayloadByteUpto;
+
+  private final IndexOutput docOut;
+  private final IndexOutput posOut;
+  private final IndexOutput payOut;
+
+  private int curDoc;
+  private long curDocPointer;
+  private long curPosPointer;
+  private long curPayPointer;
+  private int curPosBufferUpto;
+  private int curEndOffset;
+  private int curPayloadByteUpto;
+  private boolean fieldHasPositions;
+  private boolean fieldHasOffsets;
+  private boolean fieldHasPayloads;
+
+  public BlockPackedSkipWriter(int skipInterval, int maxSkipLevels, int docCount, IndexOutput docOut, IndexOutput posOut, IndexOutput payOut) {
+    super(skipInterval, maxSkipLevels, docCount);
+    this.docOut = docOut;
+    this.posOut = posOut;
+    this.payOut = payOut;
+    
+    lastSkipDoc = new int[maxSkipLevels];
+    lastSkipDocPointer = new long[maxSkipLevels];
+    if (posOut != null) {
+      lastSkipPosPointer = new long[maxSkipLevels];
+      if (payOut != null) {
+        lastSkipPayPointer = new long[maxSkipLevels];
+      }
+      lastEndOffset = new int[maxSkipLevels];
+      lastPayloadByteUpto = new int[maxSkipLevels];
+    }
+  }
+
+  public void setField(boolean fieldHasPositions, boolean fieldHasOffsets, boolean fieldHasPayloads) {
+    this.fieldHasPositions = fieldHasPositions;
+    this.fieldHasOffsets = fieldHasOffsets;
+    this.fieldHasPayloads = fieldHasPayloads;
+  }
+
+  @Override
+  public void resetSkip() {
+    super.resetSkip();
+    Arrays.fill(lastSkipDoc, 0);
+    Arrays.fill(lastSkipDocPointer, docOut.getFilePointer());
+    if (fieldHasPositions) {
+      Arrays.fill(lastSkipPosPointer, posOut.getFilePointer());
+      if (fieldHasOffsets) {
+        Arrays.fill(lastEndOffset, 0);
+      }
+      if (fieldHasPayloads) {
+        Arrays.fill(lastPayloadByteUpto, 0);
+      }
+      if (fieldHasOffsets || fieldHasPayloads) {
+        Arrays.fill(lastSkipPayPointer, payOut.getFilePointer());
+      }
+    }
+  }
+
+  /**
+   * Sets the values for the current skip data. 
+   */
+  public void bufferSkip(int doc, int numDocs, long posFP, long payFP, int posBufferUpto, int endOffset, int payloadByteUpto) throws IOException {
+    this.curDoc = doc;
+    this.curDocPointer = docOut.getFilePointer();
+    this.curPosPointer = posFP;
+    this.curPayPointer = payFP;
+    this.curPosBufferUpto = posBufferUpto;
+    this.curPayloadByteUpto = payloadByteUpto;
+    this.curEndOffset = endOffset;
+    bufferSkip(numDocs);
+  }
+  
+  @Override
+  protected void writeSkipData(int level, IndexOutput skipBuffer) throws IOException {
+    int delta = curDoc - lastSkipDoc[level];
+    if (DEBUG) {
+      System.out.println("writeSkipData level=" + level + " lastDoc=" + curDoc + " delta=" + delta + " curDocPointer=" + curDocPointer);
+    }
+    skipBuffer.writeVInt(delta);
+    lastSkipDoc[level] = curDoc;
+
+    skipBuffer.writeVInt((int) (curDocPointer - lastSkipDocPointer[level]));
+    lastSkipDocPointer[level] = curDocPointer;
+
+    if (fieldHasPositions) {
+      if (DEBUG) {
+        System.out.println("  curPosPointer=" + curPosPointer + " curPosBufferUpto=" + curPosBufferUpto);
+      }
+      skipBuffer.writeVInt((int) (curPosPointer - lastSkipPosPointer[level]));
+      lastSkipPosPointer[level] = curPosPointer;
+      skipBuffer.writeVInt(curPosBufferUpto);
+
+      if (fieldHasPayloads) {
+        skipBuffer.writeVInt(curPayloadByteUpto);
+      }
+
+      if (fieldHasOffsets) {
+        skipBuffer.writeVInt(curEndOffset - lastEndOffset[level]);
+        lastEndOffset[level] = curEndOffset;
+      }
+
+      if (fieldHasOffsets || fieldHasPayloads) {
+        skipBuffer.writeVInt((int) (curPayPointer - lastSkipPayPointer[level]));
+        lastSkipPayPointer[level] = curPayPointer;
+      }
+    }
+  }
+}

Added: 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=1367338&view=auto
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java (added)
+++ lucene/dev/branches/pforcodec_3892/lucene/core/src/java/org/apache/lucene/codecs/blockpacked/ForUtil.java Mon Jul 30 22:15:06 2012
@@ -0,0 +1,130 @@
+package org.apache.lucene.codecs.blockpacked;
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.nio.LongBuffer;
+import java.nio.IntBuffer;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+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));
+    }
+
+    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.
+   */
+  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 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++;
+      }
+    }
+    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;
+      }
+    }
+    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]);
+  }
+}

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=1367338&r1=1367337&r2=1367338&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 Mon Jul 30 22:15:06 2012
@@ -17,8 +17,6 @@ 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
-org.apache.lucene.codecs.pfor.PForPostingsFormat
 org.apache.lucene.codecs.bulkvint.BulkVIntPostingsFormat
 org.apache.lucene.codecs.block.BlockPostingsFormat
 org.apache.lucene.codecs.memory.DirectPostingsFormat

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=1367338&r1=1367337&r2=1367338&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 Mon Jul 30 22:15:06 2012
@@ -61,7 +61,6 @@ 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.
@@ -103,8 +102,6 @@ public class MockRandomPostingsFormat ex
       final int baseBlockSize = _TestUtil.nextInt(random, 1, 127);
       delegates.add(new MockVariableIntBlockPostingsFormat.MockIntFactory(baseBlockSize));
       // TODO: others
-      delegates.add(new ForFactory());
-      delegates.add(new PForFactory());
     }
 
     private static String getExtension(String fileName) {

Modified: lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1367338&r1=1367337&r2=1367338&view=diff
==============================================================================
--- lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/pforcodec_3892/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Mon Jul 30 22:15:06 2012
@@ -282,9 +282,7 @@ public abstract class LuceneTestCase ext
     "MockFixedIntBlock",
     "MockVariableIntBlock",
     "MockSep",
-    "MockRandom",
-    "For",
-    "PFor"
+    "MockRandom"
   ));
   
   // -----------------------------------------------------------------