You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2010/11/15 23:18:54 UTC

svn commit: r1035473 - in /lucene/dev/trunk/lucene: ./ src/java/org/apache/lucene/index/codecs/standard/ src/java/org/apache/lucene/store/ src/test/org/apache/lucene/store/

Author: rmuir
Date: Mon Nov 15 22:18:54 2010
New Revision: 1035473

URL: http://svn.apache.org/viewvc?rev=1035473&view=rev
Log:
LUCENE-2761: specialize payload processing in DandPEnum, speed up scanning through positions

Modified:
    lucene/dev/trunk/lucene/MIGRATE.txt
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataInput.java
    lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java
    lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/MockIndexInputWrapper.java

Modified: lucene/dev/trunk/lucene/MIGRATE.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/MIGRATE.txt?rev=1035473&r1=1035472&r2=1035473&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/MIGRATE.txt (original)
+++ lucene/dev/trunk/lucene/MIGRATE.txt Mon Nov 15 22:18:54 2010
@@ -324,3 +324,7 @@ LUCENE-1458, LUCENE-2111: Flexible Index
   The other way round MTQ.TopTermsBooleanQueryRewrite supplys a
   global AttributeSource to each segments TermsEnum. The TermsEnum is consumer
   and gets the current minimum competitive boosts (MTQ.MaxNonCompetitiveBoostAttribute).
+
+* LUCENE-2761: DataInput.readVInt/readVLong and DataOutput.writeVInt/writeVLong
+  are final. If you subclassed this code before to encode variable-length
+  integers in some specialized way, use the Codec API instead.

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1035473&r1=1035472&r2=1035473&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Mon Nov 15 22:18:54 2010
@@ -175,19 +175,37 @@ public class StandardPostingsReader exte
     if (fieldInfo.omitTermFreqAndPositions) {
       return null;
     }
-    SegmentDocsAndPositionsEnum docsEnum;
-    if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
-      docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
+    
+    // TODO: refactor
+    if (fieldInfo.storePayloads) {
+      SegmentDocsAndPositionsAndPayloadsEnum docsEnum;
+      if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsAndPayloadsEnum)) {
+        docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
+      } else {
+        docsEnum = (SegmentDocsAndPositionsAndPayloadsEnum) reuse;
+        if (docsEnum.startFreqIn != freqIn) {
+          // If you are using ParellelReader, and pass in a
+          // reused DocsEnum, it could have come from another
+          // reader also using standard codec
+          docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
+        }
+      }
+      return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
     } else {
-      docsEnum = (SegmentDocsAndPositionsEnum) reuse;
-      if (docsEnum.startFreqIn != freqIn) {
-        // If you are using ParellelReader, and pass in a
-        // reused DocsEnum, it could have come from another
-        // reader also using standard codec
+      SegmentDocsAndPositionsEnum docsEnum;
+      if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
         docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
+      } else {
+        docsEnum = (SegmentDocsAndPositionsEnum) reuse;
+        if (docsEnum.startFreqIn != freqIn) {
+          // If you are using ParellelReader, and pass in a
+          // reused DocsEnum, it could have come from another
+          // reader also using standard codec
+          docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
+        }
       }
+      return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
     }
-    return docsEnum.reset(fieldInfo, (DocTermState) termState, skipDocs);
   }
 
   // Decodes only docs
@@ -360,13 +378,195 @@ public class StandardPostingsReader exte
     }
   }
 
-  // Decodes docs & positions
+  // Decodes docs & positions. payloads are not present.
   private class SegmentDocsAndPositionsEnum extends DocsAndPositionsEnum {
     final IndexInput startFreqIn;
     private final IndexInput freqIn;
     private final IndexInput proxIn;
 
-    boolean storePayloads;                        // does current field store payloads?
+    int limit;                                    // number of docs in this posting
+    int ord;                                      // how many docs we've read
+    int doc;                                      // doc we last read
+    int freq;                                     // freq we last read
+    int position;
+
+    Bits skipDocs;
+
+    long freqOffset;
+    int skipOffset;
+    long proxOffset;
+
+    int posPendingCount;
+
+    boolean skipped;
+    DefaultSkipListReader skipper;
+    private long lazyProxPointer;
+
+    public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
+      startFreqIn = freqIn;
+      this.freqIn = (IndexInput) freqIn.clone();
+      this.proxIn = (IndexInput) proxIn.clone();
+    }
+
+    public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
+      assert !fieldInfo.omitTermFreqAndPositions;
+      assert !fieldInfo.storePayloads;
+
+      this.skipDocs = skipDocs;
+
+      // TODO: for full enum case (eg segment merging) this
+      // seek is unnecessary; maybe we can avoid in such
+      // cases
+      freqIn.seek(termState.freqOffset);
+      lazyProxPointer = termState.proxOffset;
+
+      limit = termState.docFreq;
+      ord = 0;
+      doc = 0;
+      position = 0;
+
+      skipped = false;
+      posPendingCount = 0;
+
+      freqOffset = termState.freqOffset;
+      proxOffset = termState.proxOffset;
+      skipOffset = termState.skipOffset;
+
+      return this;
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while(true) {
+        if (ord == limit) {
+          return doc = NO_MORE_DOCS;
+        }
+
+        ord++;
+
+        // Decode next doc/freq pair
+        final int code = freqIn.readVInt();
+
+        doc += code >>> 1;              // shift off low bit
+        if ((code & 1) != 0) {          // if low bit is set
+          freq = 1;                     // freq is one
+        } else {
+          freq = freqIn.readVInt();     // else read freq
+        }
+        posPendingCount += freq;
+
+        if (skipDocs == null || !skipDocs.get(doc)) {
+          break;
+        }
+      }
+
+      position = 0;
+
+      return doc;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public int freq() {
+      return freq;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+
+      // TODO: jump right to next() if target is < X away
+      // from where we are now?
+
+      if (skipOffset > 0) {
+
+        // There are enough docs in the posting to have
+        // skip data
+
+        if (skipper == null) {
+          // This is the first time this enum has ever been used for skipping -- do lazy init
+          skipper = new DefaultSkipListReader((IndexInput) freqIn.clone(), maxSkipLevels, skipInterval);
+        }
+
+        if (!skipped) {
+
+          // This is the first time this posting has
+          // skipped, since reset() was called, so now we
+          // load the skip data for this posting
+
+          skipper.init(freqOffset+skipOffset,
+                       freqOffset, proxOffset,
+                       limit, false);
+
+          skipped = true;
+        }
+
+        final int newOrd = skipper.skipTo(target); 
+
+        if (newOrd > ord) {
+          // Skipper moved
+          ord = newOrd;
+          doc = skipper.getDoc();
+          freqIn.seek(skipper.getFreqPointer());
+          lazyProxPointer = skipper.getProxPointer();
+          posPendingCount = 0;
+          position = 0;
+        }
+      }
+        
+      // Now, linear scan for the rest:
+      do {
+        nextDoc();
+      } while (target > doc);
+
+      return doc;
+    }
+
+    public int nextPosition() throws IOException {
+
+      if (lazyProxPointer != -1) {
+        proxIn.seek(lazyProxPointer);
+        lazyProxPointer = -1;
+      }
+
+      // scan over any docs that were iterated without their positions
+      if (posPendingCount > freq) {
+        position = 0;
+        while(posPendingCount != freq) {
+          if ((proxIn.readByte() & 0x80) == 0) {
+            posPendingCount--;
+          }
+        }
+      }
+
+      position += proxIn.readVInt();
+
+      posPendingCount--;
+
+      assert posPendingCount >= 0: "nextPosition() was called too many times (more than freq() times) posPendingCount=" + posPendingCount;
+
+      return position;
+    }
+
+    /** Returns the payload at this position, or null if no
+     *  payload was indexed. */
+    public BytesRef getPayload() throws IOException {
+      throw new IOException("No payloads exist for this field!");
+    }
+
+    public boolean hasPayload() {
+      return false;
+    }
+  }
+  
+  // Decodes docs & positions & payloads
+  private class SegmentDocsAndPositionsAndPayloadsEnum extends DocsAndPositionsEnum {
+    final IndexInput startFreqIn;
+    private final IndexInput freqIn;
+    private final IndexInput proxIn;
 
     int limit;                                    // number of docs in this posting
     int ord;                                      // how many docs we've read
@@ -389,16 +589,16 @@ public class StandardPostingsReader exte
     private BytesRef payload;
     private long lazyProxPointer;
 
-    public SegmentDocsAndPositionsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
+    public SegmentDocsAndPositionsAndPayloadsEnum(IndexInput freqIn, IndexInput proxIn) throws IOException {
       startFreqIn = freqIn;
       this.freqIn = (IndexInput) freqIn.clone();
       this.proxIn = (IndexInput) proxIn.clone();
     }
 
-    public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
+    public SegmentDocsAndPositionsAndPayloadsEnum reset(FieldInfo fieldInfo, DocTermState termState, Bits skipDocs) throws IOException {
       assert !fieldInfo.omitTermFreqAndPositions;
-      storePayloads = fieldInfo.storePayloads;
-      if (storePayloads && payload == null) {
+      assert fieldInfo.storePayloads;
+      if (payload == null) {
         payload = new BytesRef();
         payload.bytes = new byte[1];
       }
@@ -491,7 +691,7 @@ public class StandardPostingsReader exte
 
           skipper.init(freqOffset+skipOffset,
                        freqOffset, proxOffset,
-                       limit, storePayloads);
+                       limit, true);
 
           skipped = true;
         }
@@ -537,15 +737,14 @@ public class StandardPostingsReader exte
 
         final int code = proxIn.readVInt();
 
-        if (storePayloads) {
-          if ((code & 1) != 0) {
-            // new payload length
-            payloadLength = proxIn.readVInt();
-            assert payloadLength >= 0;
-          }
-          assert payloadLength != -1;
-          proxIn.seek(proxIn.getFilePointer() + payloadLength);
+        if ((code & 1) != 0) {
+          // new payload length
+          payloadLength = proxIn.readVInt();
+          assert payloadLength >= 0;
         }
+        
+        assert payloadLength != -1;
+        proxIn.seek(proxIn.getFilePointer() + payloadLength);
 
         posPendingCount--;
         position = 0;
@@ -553,26 +752,21 @@ public class StandardPostingsReader exte
       }
 
       // read next position
-      if (storePayloads) {
-
-        if (payloadPending && payloadLength > 0) {
-          // payload wasn't retrieved for last position
-          proxIn.seek(proxIn.getFilePointer()+payloadLength);
-        }
+      if (payloadPending && payloadLength > 0) {
+        // payload wasn't retrieved for last position
+        proxIn.seek(proxIn.getFilePointer()+payloadLength);
+      }
 
-        final int code = proxIn.readVInt();
-        if ((code & 1) != 0) {
-          // new payload length
-          payloadLength = proxIn.readVInt();
-          assert payloadLength >= 0;
-        }
-        assert payloadLength != -1;
-          
-        payloadPending = true;
-        position += code >>> 1;
-      } else {
-        position += proxIn.readVInt();
+      final int code = proxIn.readVInt();
+      if ((code & 1) != 0) {
+        // new payload length
+        payloadLength = proxIn.readVInt();
+        assert payloadLength >= 0;
       }
+      assert payloadLength != -1;
+          
+      payloadPending = true;
+      position += code >>> 1;
 
       posPendingCount--;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataInput.java?rev=1035473&r1=1035472&r2=1035473&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataInput.java Mon Nov 15 22:18:54 2010
@@ -82,7 +82,7 @@ public abstract class DataInput implemen
    * supported.
    * @see DataOutput#writeVInt(int)
    */
-  public int readVInt() throws IOException {
+  public final int readVInt() throws IOException {
     byte b = readByte();
     int i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -102,7 +102,7 @@ public abstract class DataInput implemen
   /** Reads a long stored in variable-length format.  Reads between one and
    * nine bytes.  Smaller values take fewer bytes.  Negative numbers are not
    * supported. */
-  public long readVLong() throws IOException {
+  public final long readVLong() throws IOException {
     byte b = readByte();
     long i = b & 0x7F;
     for (int shift = 7; (b & 0x80) != 0; shift += 7) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java?rev=1035473&r1=1035472&r2=1035473&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/store/DataOutput.java Mon Nov 15 22:18:54 2010
@@ -66,7 +66,7 @@ public abstract class DataOutput {
    * supported.
    * @see DataInput#readVInt()
    */
-  public void writeVInt(int i) throws IOException {
+  public final void writeVInt(int i) throws IOException {
     while ((i & ~0x7F) != 0) {
       writeByte((byte)((i & 0x7f) | 0x80));
       i >>>= 7;
@@ -87,7 +87,7 @@ public abstract class DataOutput {
    * supported.
    * @see DataInput#readVLong()
    */
-  public void writeVLong(long i) throws IOException {
+  public final void writeVLong(long i) throws IOException {
     while ((i & ~0x7F) != 0) {
       writeByte((byte)((i & 0x7f) | 0x80));
       i >>>= 7;

Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/MockIndexInputWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/MockIndexInputWrapper.java?rev=1035473&r1=1035472&r2=1035473&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/MockIndexInputWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/store/MockIndexInputWrapper.java Mon Nov 15 22:18:54 2010
@@ -132,21 +132,11 @@ public class MockIndexInputWrapper exten
   }
 
   @Override
-  public int readVInt() throws IOException {
-    return delegate.readVInt();
-  }
-
-  @Override
   public long readLong() throws IOException {
     return delegate.readLong();
   }
 
   @Override
-  public long readVLong() throws IOException {
-    return delegate.readVLong();
-  }
-
-  @Override
   public String readString() throws IOException {
     return delegate.readString();
   }