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 2010/12/14 18:18:02 UTC

svn commit: r1049178 [1/2] - in /lucene/dev/branches/bulkpostings/lucene: contrib/instantiated/src/java/org/apache/lucene/store/instantiated/ contrib/memory/src/java/org/apache/lucene/index/memory/ src/java/org/apache/lucene/index/ src/java/org/apache/...

Author: mikemccand
Date: Tue Dec 14 17:18:00 2010
New Revision: 1049178

URL: http://svn.apache.org/viewvc?rev=1049178&view=rev
Log:
LUCENE-2723: initial patch (Robert's last patch)

Modified:
    lucene/dev/branches/bulkpostings/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermsEnum.java
    lucene/dev/branches/bulkpostings/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/CheckIndex.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/DocsEnum.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexReader.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/Terms.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsEnum.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsReader.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/SepSkipListReader.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/BooleanScorer.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/ConstantScoreQuery.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermQuery.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/TermScorer.java
    lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/BitUtil.java
    lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/TestExternalCodecs.java
    lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestIndexWriter.java
    lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/codecs/intblock/TestIntBlockCodec.java
    lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/codecs/mocksep/MockSingleIntIndexInput.java

Modified: lucene/dev/branches/bulkpostings/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermsEnum.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/contrib/instantiated/src/java/org/apache/lucene/store/instantiated/InstantiatedTermsEnum.java Tue Dec 14 17:18:00 2010
@@ -22,6 +22,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.BulkPostingsEnum;
 import java.util.Arrays;
 import java.util.Comparator;
 
@@ -126,6 +127,17 @@ public class InstantiatedTermsEnum exten
   }
 
   @Override
+  public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) {
+    InstantiatedBulkPostingsEnum postingsEnum;
+    if (reuse == null || !(reuse instanceof InstantiatedBulkPostingsEnum) || !((InstantiatedBulkPostingsEnum) reuse).canReuse(field, doFreqs, doPositions)) {
+      postingsEnum = new InstantiatedBulkPostingsEnum(field, doFreqs, doPositions);
+    } else {
+      postingsEnum = (InstantiatedBulkPostingsEnum) reuse;
+    }
+    return postingsEnum.reset(terms[upto]);
+  }
+
+  @Override
   public Comparator<BytesRef> getComparator() {
     return BytesRef.getUTF8SortedAsUnicodeComparator();
   }

Modified: lucene/dev/branches/bulkpostings/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/contrib/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Tue Dec 14 17:18:00 2010
@@ -43,6 +43,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermFreqVector;
 import org.apache.lucene.index.TermPositionVector;
@@ -907,6 +908,17 @@ public class MemoryIndex implements Seri
       }
 
       @Override
+      public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) {
+        MemoryBulkPostingsEnum postingsEnum;
+        if (reuse == null || !(reuse instanceof MemoryBulkPostingsEnum) || !((MemoryBulkPostingsEnum) reuse).canReuse(info, doFreqs, doPositions)) {
+          postingsEnum = new MemoryBulkPostingsEnum(info, doFreqs, doPositions);
+        } else {
+          postingsEnum = (MemoryBulkPostingsEnum) reuse;
+        }
+        return postingsEnum.reset(info.sortedTerms[termUpto].getValue());
+      }
+
+      @Override
       public Comparator<BytesRef> getComparator() {
         return BytesRef.getUTF8SortedAsUnicodeComparator();
       }
@@ -1004,6 +1016,173 @@ public class MemoryIndex implements Seri
         return null;
       }
     }
+
+    private class MemoryBulkPostingsEnum extends BulkPostingsEnum {
+
+      private final DocDeltasReader docDeltasReader;
+      private final FreqsReader freqsReader;
+      private final PositionDeltasReader positionDeltasReader;
+      private final Info info;
+      
+      public MemoryBulkPostingsEnum(Info info, boolean doFreqs, boolean doPositions) {
+        this.info = info;
+        docDeltasReader = new DocDeltasReader();
+        if (doFreqs) {
+          freqsReader = new FreqsReader();
+        } else {
+          freqsReader = null;
+        }
+
+        if (doPositions) {
+          positionDeltasReader = new PositionDeltasReader();
+        } else {
+          positionDeltasReader = null;
+        }
+      }
+
+      public boolean canReuse(Info info, boolean doFreq, boolean doPositions) {
+        return this.info == info && (doFreq == (freqsReader != null)) && (doPositions == (positionDeltasReader != null));
+      }
+
+      private class DocDeltasReader extends BlockReader {
+        private final int[] buffer = new int[1];
+
+        public void reset() {
+        }
+
+        @Override
+        public int[] getBuffer() {
+          return buffer;
+        }
+
+        @Override
+        public int offset() {
+          return 0;
+        }
+
+        @Override
+        public void setOffset(int offset) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int end() {
+          return 1;
+        }
+
+        @Override
+        public int fill() {
+          return 1;
+        }
+      }
+
+      private class FreqsReader extends BlockReader {
+        private final int[] buffer = new int[1];
+
+        public void reset(int freq) {
+          buffer[0] = freq;
+        }
+
+        @Override
+        public int[] getBuffer() {
+          return buffer;
+        }
+
+        @Override
+        public int offset() {
+          return 0;
+        }
+
+        @Override
+        public void setOffset(int offset) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int end() {
+          return 1;
+        }
+
+        @Override
+        public int fill() {
+          return 1;
+        }
+      }
+
+      private class PositionDeltasReader extends BlockReader {
+        private final int[] buffer = new int[64];
+        private ArrayIntList positions;
+        private int posUpto;
+        private int limit;
+
+        public void reset(ArrayIntList positions) {
+          posUpto = 0;
+          this.positions = positions;
+          fill();
+        }
+
+        @Override
+        public int[] getBuffer() {
+          return buffer;
+        }
+
+        @Override
+        public int offset() {
+          return 0;
+        }
+
+        @Override
+        public void setOffset(int offset) {
+          throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public int end() {
+          return limit;
+        }
+
+        @Override
+        public int fill() {
+          final int chunk = Math.min(buffer.length, positions.size() - posUpto);
+          for(int i=0;i<chunk;i++) {
+            buffer[i] = positions.get(posUpto++);
+          }
+          return limit = chunk;
+        }
+      }
+
+      public BulkPostingsEnum reset(ArrayIntList positions) {
+        docDeltasReader.reset();
+    
+        if (freqsReader != null) {
+          freqsReader.reset(positions.size());
+        }
+        if (positionDeltasReader != null) {
+          positionDeltasReader.reset(positions);
+        }
+        return this;
+      }
+
+      @Override
+      public BlockReader getDocDeltasReader() {
+        return docDeltasReader;
+      }
+
+      @Override
+      public BlockReader getPositionDeltasReader() {
+        return positionDeltasReader;
+      }
+
+      @Override
+      public BlockReader getFreqsReader() {
+        return freqsReader;
+      }
+
+      @Override
+      public JumpResult jump(int target, int curCount) {
+        return null;
+      }
+    }
     
     @Override
     public TermFreqVector[] getTermFreqVectors(int docNumber) {

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/CheckIndex.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/CheckIndex.java Tue Dec 14 17:18:00 2010
@@ -770,7 +770,7 @@ public class CheckIndex {
             }
 
             if (totDocCount != totDocCount2) {
-              throw new RuntimeException("search to seek terms produced wrong number of hits: " + totDocCount + " vs " + totDocCount2);
+              throw new RuntimeException("search by seek term produced wrong number of hits: " + totDocCount + " vs " + totDocCount2 + " field=" + field);
             }
           }
         }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/DocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/DocsEnum.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/DocsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/DocsEnum.java Tue Dec 14 17:18:00 2010
@@ -44,6 +44,8 @@ public abstract class DocsEnum extends D
     return atts;
   }
 
+  // nocommit -- delete all after here:
+
   // TODO: maybe add bulk read only docIDs (for eventual
   // match-only scoring)
 
@@ -83,6 +85,7 @@ public abstract class DocsEnum extends D
    *  <p>NOTE: the default impl simply delegates to {@link
    *  #nextDoc}, but subclasses may do this more
    *  efficiently. */
+  // nocommit -- remove this
   public int read() throws IOException {
     int count = 0;
     final int[] docs = bulkResult.docs.ints;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/FilterIndexReader.java Tue Dec 14 17:18:00 2010
@@ -170,11 +170,18 @@ public class FilterIndexReader extends I
     }
 
     @Override
+    public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+      return in.bulkPostings(reuse, doFreqs, doPositions);
+    }
+
+    @Override
     public Comparator<BytesRef> getComparator() throws IOException {
       return in.getComparator();
     }
   }
 
+  // nocommit need FilteredBulkDocsEnum
+
   /** Base class for filtering {@link DocsEnum} implementations. */
   public static class FilterDocsEnum extends DocsEnum {
     protected DocsEnum in;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexReader.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/IndexReader.java Tue Dec 14 17:18:00 2010
@@ -1052,6 +1052,22 @@ public abstract class IndexReader implem
     }
   }
 
+  // nocommit jdocs
+  public BulkPostingsEnum bulkTermPostingsEnum(String field, BytesRef term, boolean doFreqs, boolean doPositions) throws IOException {
+    assert field != null;
+    assert term != null;
+    final Fields fields = fields();
+    if (fields == null) {
+      return null;
+    }
+    final Terms terms = fields.terms(field);
+    if (terms != null) {
+      return terms.bulkPostings(term, null, doFreqs, doPositions);
+    } else {
+      return null;
+    }
+  }
+
   /** Returns {@link DocsAndPositionsEnum} for the specified
    *  field & term.  This may return null, if either the
    *  field or term does not exist, or, positions were not

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiFields.java Tue Dec 14 17:18:00 2010
@@ -215,6 +215,19 @@ public final class MultiFields extends F
     }
   }
 
+  /** Returns {@link BulkPostingsEnum} for the specified
+   *  field & term.  This may return null if the term does
+   *  not exist or positions were not indexed. */
+  public static BulkPostingsEnum getBulkPostingsEnum(IndexReader r, String field, BytesRef term, boolean doFreqs, boolean doPositions) throws IOException {
+    assert field != null;
+    assert term != null;
+    final Terms terms = getTerms(r, field);
+    if (terms != null) {
+      return terms.bulkPostings(term, null, doFreqs, doPositions);
+    } else {
+      return null;
+    }
+  }
   public MultiFields(Fields[] subs, ReaderUtil.Slice[] subSlices) {
     this.subs = subs;
     this.subSlices = subSlices;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/MultiTermsEnum.java Tue Dec 14 17:18:00 2010
@@ -41,6 +41,7 @@ public final class MultiTermsEnum extend
   private final TermsEnumWithSlice[] top;
   private final MultiDocsEnum.EnumWithSlice[] subDocs;
   private final MultiDocsAndPositionsEnum.EnumWithSlice[] subDocsAndPositions;
+  private final MultiBulkPostingsEnum.EnumWithSlice[] subBulkPostings;
 
   private BytesRef lastSeek;
   private final BytesRef lastSeekScratch = new BytesRef();
@@ -75,12 +76,15 @@ public final class MultiTermsEnum extend
     subs = new TermsEnumWithSlice[slices.length];
     subDocs = new MultiDocsEnum.EnumWithSlice[slices.length];
     subDocsAndPositions = new MultiDocsAndPositionsEnum.EnumWithSlice[slices.length];
+    subBulkPostings = new MultiBulkPostingsEnum.EnumWithSlice[slices.length];
     for(int i=0;i<slices.length;i++) {
       subs[i] = new TermsEnumWithSlice(slices[i]);
       subDocs[i] = new MultiDocsEnum.EnumWithSlice();
       subDocs[i].slice = slices[i];
       subDocsAndPositions[i] = new MultiDocsAndPositionsEnum.EnumWithSlice();
       subDocsAndPositions[i].slice = slices[i];
+      subBulkPostings[i] = new MultiBulkPostingsEnum.EnumWithSlice();
+      subBulkPostings[i].slice = slices[i];
     }
     currentSubs = new TermsEnumWithSlice[slices.length];
   }
@@ -402,6 +406,40 @@ public final class MultiTermsEnum extend
     }
   }
 
+  @Override
+  public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+    final MultiBulkPostingsEnum postingsEnum = new MultiBulkPostingsEnum();
+    
+    int upto = 0;
+
+    for(int i=0;i<numTop;i++) {
+
+      final TermsEnumWithSlice entry = top[i];
+
+      final BulkPostingsEnum subPostings = entry.terms.bulkPostings(null, doFreqs, doPositions);
+
+      if (subPostings != null) {
+        subBulkPostings[upto].postingsEnum = subPostings;
+        subBulkPostings[upto].slice = entry.subSlice;
+        subBulkPostings[upto].docFreq = entry.terms.docFreq();
+        upto++;
+      } else {
+        if (entry.terms.docs(null, null) != null) {
+          // At least one of our subs does not store
+          // positions -- we can't correctly produce a
+          // MultiDocsAndPositions enum
+          return null;
+        }
+      }
+    }
+
+    if (upto == 0) {
+      return null;
+    } else {
+      return postingsEnum.reset(subBulkPostings, upto, doFreqs, doPositions);
+    }
+  }
+
   private final static class TermsEnumWithSlice {
     private final ReaderUtil.Slice subSlice;
     private TermsEnum terms;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SlowMultiReaderWrapper.java Tue Dec 14 17:18:00 2010
@@ -66,5 +66,9 @@ public final class SlowMultiReaderWrappe
   public IndexReader[] getSequentialSubReaders() {
     return null;
   }
-  
+
+  @Override
+  public String toString() {
+    return "SlowMultiReaderWrapper(" + in + ")";
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/Terms.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/Terms.java Tue Dec 14 17:18:00 2010
@@ -68,6 +68,15 @@ public abstract class Terms {
     }
   }
 
+  public BulkPostingsEnum bulkPostings(BytesRef text, BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+    final TermsEnum termsEnum = getThreadTermsEnum();
+    if (termsEnum.seek(text) == TermsEnum.SeekStatus.FOUND) {
+      return termsEnum.bulkPostings(reuse, doFreqs, doPositions);
+    } else {
+      return null;
+    }
+  }
+
   /** Get {@link DocsEnum} for the specified term.  This
    *  method will may return null if the term does not
    *  exists, or positions were not indexed. */ 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsEnum.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsEnum.java Tue Dec 14 17:18:00 2010
@@ -109,6 +109,11 @@ public abstract class TermsEnum {
    * @param reuse pass a prior DocsEnum for possible reuse */
   public abstract DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException;
 
+  // nocommit -- make abstract
+  public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
   /** Get {@link DocsAndPositionsEnum} for the current term.
    *  Do not call this before calling {@link #next} or
    *  {@link #seek} for the first time.  This method will

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java Tue Dec 14 17:18:00 2010
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.Closeable;
 
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.store.IndexInput;
@@ -50,6 +51,12 @@ public abstract class PostingsReaderBase
    *  TermState may be reused. */
   public abstract DocsEnum docs(FieldInfo fieldInfo, TermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
 
+  // nocommit jdocs
+  // nocommit make abstract
+  public BulkPostingsEnum bulkPostings(FieldInfo fieldInfo, TermState state, BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
   public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsReader.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/PrefixCodedTermsReader.java Tue Dec 14 17:18:00 2010
@@ -25,6 +25,7 @@ import java.util.TreeMap;
 import java.util.Comparator;
 
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
@@ -319,9 +320,9 @@ public class PrefixCodedTermsReader exte
       @Override
       public SeekStatus seek(BytesRef term, boolean useCache) throws IOException {
         // Check cache
-        fieldTerm.term = term;
         TermState cachedState;
         if (useCache) {
+          fieldTerm.term = term;
           cachedState = termsCache.get(fieldTerm);
           if (cachedState != null) {
             state.copy(cachedState);
@@ -387,7 +388,6 @@ public class PrefixCodedTermsReader exte
         while(next() != null) {
           final int cmp = termComp.compare(bytesReader.term, term);
           if (cmp == 0) {
-
             if (doSeek && useCache) {
               // Store in cache
               FieldAndTerm entryKey = new FieldAndTerm(fieldTerm);
@@ -396,7 +396,6 @@ public class PrefixCodedTermsReader exte
               cachedState.filePointer = in.getFilePointer();
               termsCache.put(entryKey, cachedState);
             }
-              
             return SeekStatus.FOUND;
           } else if (cmp > 0) {
             return SeekStatus.NOT_FOUND;
@@ -500,6 +499,12 @@ public class PrefixCodedTermsReader exte
       }
 
       @Override
+      public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+        BulkPostingsEnum postingsEnum = postingsReader.bulkPostings(fieldInfo, state, reuse, doFreqs, doPositions);
+        return postingsEnum;
+      }
+
+      @Override
       public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
         if (fieldInfo.omitTermFreqAndPositions) {
           return null;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexInput.java Tue Dec 14 17:18:00 2010
@@ -24,8 +24,8 @@ package org.apache.lucene.index.codecs.i
 import java.io.IOException;
 
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.IntsRef;
 
 /** Abstract base class that reads fixed-size blocks of ints
  *  from an IndexInput.  While this is a simple approach, a
@@ -42,7 +42,9 @@ public abstract class FixedIntBlockIndex
   
   public FixedIntBlockIndexInput(final IndexInput in) throws IOException {
     this.in = in;
-    blockSize = in.readVInt();
+    //blockSize = in.readVInt();
+    blockSize = in.readInt();
+    //System.out.println("BLOCK size " + blockSize);
   }
 
   @Override
@@ -67,80 +69,72 @@ public abstract class FixedIntBlockIndex
 
   public interface BlockReader {
     public void readBlock() throws IOException;
+    // nocommit -- need seek here so mmapdir "knows"
   }
 
-  private static class Reader extends IntIndexInput.Reader {
+  private static class Reader extends BulkPostingsEnum.BlockReader {
     private final IndexInput in;
 
     protected final int[] pending;
-    int upto;
+    private int offset;
 
-    private boolean seekPending;
-    private long pendingFP;
-    private int pendingUpto;
     private long lastBlockFP;
     private final BlockReader blockReader;
     private final int blockSize;
-    private final IntsRef bulkResult = new IntsRef();
 
     public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader)
-    throws IOException {
+      throws IOException {
       this.in = in;
       this.pending = pending;
       this.blockSize = pending.length;
-      bulkResult.ints = pending;
       this.blockReader = blockReader;
-      upto = blockSize;
     }
 
-    void seek(final long fp, final int upto) {
-      pendingFP = fp;
-      pendingUpto = upto;
-      seekPending = true;
-    }
-
-    private void maybeSeek() throws IOException {
-      if (seekPending) {
-        if (pendingFP != lastBlockFP) {
-          // need new block
-          in.seek(pendingFP);
-          lastBlockFP = pendingFP;
-          blockReader.readBlock();
-        }
-        upto = pendingUpto;
-        seekPending = false;
+    void seek(final long fp, final int upto) throws IOException {
+      offset = upto;
+      if (fp != lastBlockFP) {
+        // Seek to new block; this may in fact be the next
+        // block ie when caller is doing sequential scan (eg
+        // PrefixQuery)
+        //System.out.println("  seek block fp=" + fp + " vs last=" + lastBlockFP + " upto=" + upto);
+        in.seek(fp);
+        fill();
+      } else {
+        // Seek within current block
+        //System.out.println("  seek in-block fp=" + fp + " upto=" + offset);
       }
     }
 
     @Override
-    public int next() throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
-        lastBlockFP = in.getFilePointer();
-        blockReader.readBlock();
-        upto = 0;
-      }
+    public int[] getBuffer() {
+      return pending;
+    }
 
-      return pending[upto++];
+    @Override
+    public int end() {
+      return blockSize;
     }
 
     @Override
-    public IntsRef read(final int count) throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
-        blockReader.readBlock();
-        upto = 0;
-      }
-      bulkResult.offset = upto;
-      if (upto + count < blockSize) {
-        bulkResult.length = count;
-        upto += count;
-      } else {
-        bulkResult.length = blockSize - upto;
-        upto = blockSize;
-      }
+    public int offset() {
+      return offset;
+    }
+
+    @Override
+    public void setOffset(int offset) {
+      this.offset = offset;
+    }
 
-      return bulkResult;
+    @Override
+    public int fill() throws IOException {
+      //System.out.println("fii.fill seekPending=" + seekPending + " set lastFP=" + pendingFP + " this=" + this);
+      // nocommit -- not great that we do this on each
+      // fill -- but we need it to detect seek w/in block
+      // case:
+      // nocommit: can't we += blockNumBytes instead?
+      lastBlockFP = in.getFilePointer();
+      blockReader.readBlock();
+      return blockSize;
     }
   }
 
@@ -150,10 +144,14 @@ public abstract class FixedIntBlockIndex
 
     @Override
     public void read(final IndexInput indexIn, final boolean absolute) throws IOException {
+      // nocommit -- somehow we should share the "upto" for
+      // doc & freq since they will always be "in sync"
       if (absolute) {
         fp = indexIn.readVLong();
         upto = indexIn.readVInt();
       } else {
+        // nocommit -- can't this be more efficient?  read a
+        // single byte and check a bit?  block size is 128...
         final long delta = indexIn.readVLong();
         if (delta == 0) {
           // same block
@@ -168,7 +166,7 @@ public abstract class FixedIntBlockIndex
     }
 
     @Override
-    public void read(final IntIndexInput.Reader indexIn, final boolean absolute) throws IOException {
+    public void read(final BulkPostingsEnum.BlockReader indexIn, final boolean absolute) throws IOException {
       if (absolute) {
         fp = indexIn.readVLong();
         upto = indexIn.next();
@@ -187,7 +185,7 @@ public abstract class FixedIntBlockIndex
     }
 
     @Override
-    public void seek(final IntIndexInput.Reader other) throws IOException {
+    public void seek(final BulkPostingsEnum.BlockReader other) throws IOException {
       ((Reader) other).seek(fp, upto);
     }
 
@@ -205,5 +203,10 @@ public abstract class FixedIntBlockIndex
       other.upto = upto;
       return other;
     }
+
+    @Override
+    public String toString() {
+      return "FixedBlockIndex(fp=" + fp + " offset=" + upto + ")";
+    }
   }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/FixedIntBlockIndexOutput.java Tue Dec 14 17:18:00 2010
@@ -44,7 +44,7 @@ public abstract class FixedIntBlockIndex
   protected FixedIntBlockIndexOutput(IndexOutput out, int fixedBlockSize) throws IOException {
     blockSize = fixedBlockSize;
     this.out = out;
-    out.writeVInt(blockSize);
+    out.writeInt(blockSize);
     buffer = new int[blockSize];
   }
 
@@ -111,6 +111,11 @@ public abstract class FixedIntBlockIndex
       lastUpto = upto;
       lastFP = fp;
     }
+
+    @Override
+    public String toString() {
+      return "fp=" + fp + " idx=" + upto;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/intblock/VariableIntBlockIndexInput.java Tue Dec 14 17:18:00 2010
@@ -23,9 +23,9 @@ package org.apache.lucene.index.codecs.i
 
 import java.io.IOException;
 
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.codecs.sep.IntIndexInput;
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.IntsRef;
 
 // TODO: much of this can be shared code w/ the fixed case
 
@@ -72,94 +72,97 @@ public abstract class VariableIntBlockIn
     public void seek(long pos) throws IOException;
   }
 
-  public static class Reader extends IntIndexInput.Reader {
+  public static class Reader extends BulkPostingsEnum.BlockReader {
     private final IndexInput in;
 
     public final int[] pending;
-    int upto;
 
     private boolean seekPending;
     private long pendingFP;
-    private int pendingUpto;
+    private int offset;
     private long lastBlockFP;
     private int blockSize;
     private final BlockReader blockReader;
-    private final IntsRef bulkResult = new IntsRef();
+    private int limit;
 
     public Reader(final IndexInput in, final int[] pending, final BlockReader blockReader)
       throws IOException {
       this.in = in;
       this.pending = pending;
-      bulkResult.ints = pending;
       this.blockReader = blockReader;
     }
 
     void seek(final long fp, final int upto) throws IOException {
+      //System.out.println("vintb seek fp=" + fp + " upto=" + upto);
       // TODO: should we do this in real-time, not lazy?
       pendingFP = fp;
-      pendingUpto = upto;
-      assert pendingUpto >= 0: "pendingUpto=" + pendingUpto;
-      seekPending = true;
-    }
+      offset = upto;
+      assert offset >= 0: "pendingUpto=" + offset;
+      if (pendingFP != lastBlockFP) {
+        // Clear current block
+        seekPending = true;
+        //        System.out.println("  seekPending=true now fill");
+        fill();
+      } else {
+        //System.out.println("  no seekPending");
+      }
+      //System.out.println("  now offset=" + offset + " limit=" + limit);
 
-    private final void maybeSeek() throws IOException {
-      if (seekPending) {
-        if (pendingFP != lastBlockFP) {
-          // need new block
-          in.seek(pendingFP);
-          blockReader.seek(pendingFP);
-          lastBlockFP = pendingFP;
-          blockSize = blockReader.readBlock();
-        }
-        upto = pendingUpto;
+      // This is necessary for int encoders that are
+      // non-causal, ie must see future int values to
+      // encode the current ones.
+      while(offset >= limit) {
+        offset -= limit;
+        //System.out.println("  non-causal fill");
+        fill();
+      }
+      //System.out.println("  after skip bock offset=" + offset);
+    }
 
-        // TODO: if we were more clever when writing the
-        // index, such that a seek point wouldn't be written
-        // until the int encoder "committed", we could avoid
-        // this (likely minor) inefficiency:
+    @Override
+    public int[] getBuffer() {
+      return pending;
+    }
 
-        // This is necessary for int encoders that are
-        // non-causal, ie must see future int values to
-        // encode the current ones.
-        while(upto >= blockSize) {
-          upto -= blockSize;
-          lastBlockFP = in.getFilePointer();
-          blockSize = blockReader.readBlock();
-        }
-        seekPending = false;
-      }
+    @Override
+    public int end() {
+      return limit;
     }
 
     @Override
-    public int next() throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
-        lastBlockFP = in.getFilePointer();
-        blockSize = blockReader.readBlock();
-        upto = 0;
-      }
+    public int offset() {
+      return offset;
+    }
 
-      return pending[upto++];
+    @Override
+    public void setOffset(int offset) {
+      this.offset = offset;
     }
 
     @Override
-    public IntsRef read(final int count) throws IOException {
-      this.maybeSeek();
-      if (upto == blockSize) {
-        lastBlockFP = in.getFilePointer();
+    public int fill() throws IOException {
+      if (seekPending) {
+        seekPending = false;
+        in.seek(pendingFP);
+        blockReader.seek(pendingFP);
+        lastBlockFP = pendingFP;
         blockSize = blockReader.readBlock();
-        upto = 0;
-      }
-      bulkResult.offset = upto;
-      if (upto + count < blockSize) {
-        bulkResult.length = count;
-        upto += count;
+
+        // TODO: if we were more clever when writing the
+        // index, such that a seek point wouldn't be written
+        // until the int encoder "committed", we could avoid
+        // this (likely minor) inefficiency:
+
+        //System.out.println("varintblock.fill offset=" + offset + " vs blockSize=" + blockSize);
+
       } else {
-        bulkResult.length = blockSize - upto;
-        upto = blockSize;
+        // nocommit -- not great that we do this on each
+        // fill -- but we need it to detect seek w/in block
+        // case:
+        lastBlockFP = in.getFilePointer();
+        blockSize = blockReader.readBlock();
       }
-
-      return bulkResult;
+      return limit = blockSize;
     }
   }
 
@@ -189,7 +192,7 @@ public abstract class VariableIntBlockIn
     }
 
     @Override
-    public void read(final IntIndexInput.Reader indexIn, final boolean absolute) throws IOException {
+    public void read(final BulkPostingsEnum.BlockReader indexIn, final boolean absolute) throws IOException {
       if (absolute) {
         fp = indexIn.readVLong();
         upto = indexIn.next()&0xFF;
@@ -212,7 +215,7 @@ public abstract class VariableIntBlockIn
     }
 
     @Override
-    public void seek(final IntIndexInput.Reader other) throws IOException {
+    public void seek(final BulkPostingsEnum.BlockReader other) throws IOException {
       ((Reader) other).seek(fp, upto);
     }
 

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Tue Dec 14 17:18:00 2010
@@ -27,6 +27,7 @@ import java.util.Comparator;
 
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldsEnum;
@@ -42,6 +43,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.index.codecs.standard.DefaultSkipListReader;
 
 /** Exposes flex API on a pre-flex index, as a codec. 
  * @lucene.experimental
@@ -971,6 +973,17 @@ public class PreFlexFields extends Field
       }
       return docsPosEnum.reset(termEnum, skipDocs);        
     }
+
+    @Override
+    public BulkPostingsEnum bulkPostings(BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+      PreBulkPostingsEnum postingsEnum;
+      if (reuse == null || !(reuse instanceof PreBulkPostingsEnum) || !((PreBulkPostingsEnum) reuse).canReuse(fieldInfo, freqStream, doFreqs, doPositions)) {
+        postingsEnum = new PreBulkPostingsEnum(fieldInfo.omitTermFreqAndPositions, doFreqs, doPositions);
+      } else {
+        postingsEnum = (PreBulkPostingsEnum) reuse;
+      }
+      return postingsEnum.reset(fieldInfo, termEnum);
+    }
   }
 
   private final class PreDocsEnum extends DocsEnum {
@@ -1103,4 +1116,324 @@ public class PreFlexFields extends Field
       return payload;
     }
   }
+
+  static final int BULK_BUFFER_SIZE = 64;
+  
+  // Bulk postings API
+  private final class PreBulkPostingsEnum extends BulkPostingsEnum {
+    private final IndexInput freqIn;
+    private final IndexInput proxIn;
+
+    final IndexInput startFreqIn;
+    private final boolean omitTF;
+
+    boolean storePayloads;                        // does current field store payloads?
+
+    int ord;                                      // how many docs we've read
+    int docFreq;
+
+    long freqOffset;
+    long proxOffset;
+    int skipOffset;
+
+    boolean skipped;
+    DefaultSkipListReader skipper;
+    private int payloadLength;
+
+    private final DocDeltasReader docDeltasReader;
+    private final FreqsReader freqsReader;
+    private final PositionsReader positionDeltasReader;
+
+    private boolean docsPending, freqsPending;
+
+    public PreBulkPostingsEnum(boolean omitTF, boolean doFreqs, boolean doPositions) throws IOException {
+      startFreqIn = PreFlexFields.this.freqStream;
+      this.freqIn = (IndexInput) PreFlexFields.this.freqStream.clone();
+      this.omitTF = omitTF;
+
+      docDeltasReader = new DocDeltasReader();
+      if (doFreqs && !omitTF) {
+        freqsReader = new FreqsReader();
+      } else {
+        freqsReader = null;
+      }
+
+      if (doPositions && !omitTF) {
+        this.proxIn = (IndexInput) PreFlexFields.this.proxStream.clone();
+        positionDeltasReader = new PositionsReader();
+      } else {
+        this.proxIn = null;
+        positionDeltasReader = null;
+      }
+    }
+
+    public boolean canReuse(FieldInfo fieldInfo, IndexInput freqin, boolean doFreqs, boolean doPositions) {
+      return freqIn == startFreqIn &&
+        (!doFreqs || freqsReader == null) &&
+        (!doPositions || positionDeltasReader == null) && 
+        (omitTF == fieldInfo.omitTermFreqAndPositions);
+    }
+
+    final void read() throws IOException {
+      try {
+        if (freqsReader == null) {
+          // Consumer only wants doc deltas
+          assert !docsPending;
+          if (omitTF) {
+            // Index only stores doc deltas
+            for(int i=0;i<BULK_BUFFER_SIZE;i++) {
+              docDeltasReader.buffer[i] = freqIn.readVInt();
+            }
+          } else {
+            // Index stores doc deltas & freq
+            for(int i=0;i<BULK_BUFFER_SIZE;i++) {
+              final int code = freqIn.readVInt();
+              docDeltasReader.buffer[i] = code >>> 1;
+              if ((code & 1) == 0) {
+                freqIn.readVInt();
+              }
+            }
+          }
+          docsPending = true;
+        } else {
+          // Consumer wants both
+          assert !docsPending;
+          assert !freqsPending;
+          for(int i=0;i<BULK_BUFFER_SIZE;i++) {
+            final int code = freqIn.readVInt();
+            docDeltasReader.buffer[i] = code >>> 1;
+            if ((code & 1) == 0) {
+              freqsReader.buffer[i] = freqIn.readVInt();
+            } else {
+              freqsReader.buffer[i] = 1;
+            }
+          }
+          docsPending = true;
+          freqsPending = true;
+        }
+        ord += BULK_BUFFER_SIZE;
+      } catch (IOException ioe) {
+        if (freqIn.getFilePointer() != freqIn.length()) {
+          throw ioe;
+        }
+      }
+    }
+
+    class DocDeltasReader extends BulkPostingsEnum.BlockReader {
+      private final int[] buffer = new int[BULK_BUFFER_SIZE];
+      private int limit;
+
+      @Override
+      public int[] getBuffer() {
+        return buffer;
+      }
+
+      @Override
+      public int end() {
+        return limit;
+      }
+
+      @Override
+      public int fill() throws IOException {
+        if (!docsPending) {
+          read();
+        }
+        docsPending = false;
+        limit = BULK_BUFFER_SIZE;
+        return BULK_BUFFER_SIZE;
+      }
+
+      @Override
+      public int offset() {
+        return 0;
+      }
+
+      @Override
+      public void setOffset(int offset) {
+        assert offset == 0;
+      }
+    }
+
+    class FreqsReader extends BulkPostingsEnum.BlockReader {
+      private final int[] buffer = new int[BULK_BUFFER_SIZE];
+      private int limit;
+
+      @Override
+      public int[] getBuffer() {
+        return buffer;
+      }
+
+      @Override
+      public int end() {
+        return limit;
+      }
+
+      @Override
+      public int fill() throws IOException {
+        if (!freqsPending) {
+          read();
+        }
+        freqsPending = false;
+        limit = BULK_BUFFER_SIZE;
+        return BULK_BUFFER_SIZE;
+      }
+
+      @Override
+      public int offset() {
+        return 0;
+      }
+
+      @Override
+      public void setOffset(int offset) {
+        assert offset == 0;
+      }
+    }
+
+    class PositionsReader extends BulkPostingsEnum.BlockReader {
+      final int[] buffer = new int[BULK_BUFFER_SIZE];
+      int limit;
+
+      @Override
+      public int[] getBuffer() {
+        return buffer;
+      }
+
+      @Override
+      public int end() {
+        return limit;
+      }
+
+      @Override
+      public int fill() throws IOException {
+        // nocommit -- must "handle" EOF here -- cannot
+        // change old index format!
+        if (storePayloads) {
+          for(int i=0;i<BULK_BUFFER_SIZE;i++) {
+            final int code = proxIn.readVInt();
+            buffer[i] = code >>> 1;
+            if ((code & 1) != 0) {
+              payloadLength = proxIn.readVInt();
+            }
+            if (payloadLength != 0) {
+              // skip payload
+              proxIn.seek(proxIn.getFilePointer()+payloadLength);
+            }
+          }
+        } else {
+          for(int i=0;i<BULK_BUFFER_SIZE;i++) {
+            buffer[i] = proxIn.readVInt();
+          }
+        }
+        limit = BULK_BUFFER_SIZE;
+        return BULK_BUFFER_SIZE;
+      }
+
+      @Override
+      public int offset() {
+        return 0;
+      }
+
+      @Override
+      public void setOffset(int offset) {
+        assert offset == 0;
+      }
+    }
+    
+    @Override
+    public BlockReader getDocDeltasReader() {
+      return docDeltasReader;
+    }
+      
+    @Override
+    public BlockReader getFreqsReader() {
+      return freqsReader;
+    }
+
+    @Override
+    public BlockReader getPositionDeltasReader() {
+      return positionDeltasReader;
+    }
+
+    public PreBulkPostingsEnum reset(FieldInfo fieldInfo, SegmentTermEnum termEnum) throws IOException {
+      storePayloads = fieldInfo.storePayloads;
+      freqOffset = termEnum.termInfo.freqPointer;
+      freqIn.seek(freqOffset);
+
+      // TODO: for full enum case (eg segment merging) this
+      // seek is unnecessary; maybe we can avoid in such
+      // cases
+      if (positionDeltasReader != null) {
+        proxOffset = termEnum.termInfo.proxPointer;
+        proxIn.seek(proxOffset);
+      }
+
+      skipOffset = termEnum.termInfo.skipOffset;
+      docFreq = termEnum.docFreq();
+
+      ord = 0;
+      skipped = false;
+
+      return this;
+    }
+
+    private final JumpResult jumpResult = new JumpResult();
+
+    @Override
+    public JumpResult jump(int target, int curCount) 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(), tis.getMaxSkipLevels(), tis.getSkipInterval());
+        }
+
+        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,
+                       docFreq, storePayloads);
+
+          skipped = true;
+        }
+
+        final int newOrd = skipper.skipTo(target); 
+
+        // nocommit rename ord -> count
+        assert curCount == ord: "ord=" + ord + " curCount=" + curCount;
+
+        if (newOrd > ord) {
+
+          // Skipper moved
+          freqIn.seek(skipper.getFreqPointer());
+          if (freqsReader != null) {
+            freqsReader.limit = 0;
+          }
+          docDeltasReader.limit = 0;
+
+          if (positionDeltasReader != null) {
+            positionDeltasReader.limit = 0;
+            proxIn.seek(skipper.getProxPointer());
+          }
+
+          jumpResult.count = ord = newOrd;
+          jumpResult.docID = skipper.getDoc();
+
+          return jumpResult;
+        }
+      }
+
+      // no jump occurred
+      return null;
+    }
+  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Tue Dec 14 17:18:00 2010
@@ -20,6 +20,7 @@ package org.apache.lucene.index.codecs.p
 import java.io.IOException;
 
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.codecs.TermState;
@@ -30,6 +31,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.ArrayUtil;
 
 /** Concrete class that reads the current doc/freq/skip
  *  postings format 
@@ -200,6 +202,27 @@ public class PulsingPostingsReaderImpl e
     }
   }
 
+  // TODO: we could actually reuse, by having TL that
+  // holds the last wrapped reuse, and vice-versa
+  @Override
+  public BulkPostingsEnum bulkPostings(FieldInfo field, TermState _termState, BulkPostingsEnum reuse, boolean doFreqs, boolean doPositions) throws IOException {
+    PulsingTermState termState = (PulsingTermState) _termState;
+    if (termState.docFreq <= maxPulsingDocFreq) {
+      if (reuse instanceof PulsingBulkPostingsEnum && ((PulsingBulkPostingsEnum) reuse).docDeltas.length == maxPulsingDocFreq) {
+        return ((PulsingBulkPostingsEnum) reuse).reset(termState, doFreqs, doPositions);
+      } else {
+        PulsingBulkPostingsEnum postingsEnum = new PulsingBulkPostingsEnum(maxPulsingDocFreq);
+        return postingsEnum.reset(termState, doFreqs, doPositions);
+      }
+    } else {
+      if (reuse instanceof PulsingBulkPostingsEnum) {
+        return wrappedPostingsReader.bulkPostings(field, termState.wrappedTermState, null, doFreqs, doPositions);
+      } else {
+        return wrappedPostingsReader.bulkPostings(field, termState.wrappedTermState, reuse, doFreqs, doPositions);
+      }
+    }
+  }
+
   // TODO: -- not great that we can't always reuse
   @Override
   public DocsAndPositionsEnum docsAndPositions(FieldInfo field, TermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
@@ -226,8 +249,6 @@ public class PulsingPostingsReaderImpl e
     private Document doc;
     private PulsingTermState state;
 
-    public void close() {}
-
     PulsingDocsEnum reset(Bits skipDocs, PulsingTermState termState) {
       // TODO: -- not great we have to clone here --
       // merging is wasteful; TermRangeQuery too
@@ -290,6 +311,159 @@ public class PulsingPostingsReaderImpl e
     }
   }
 
+  static class PulsingBulkPostingsEnum extends BulkPostingsEnum {
+    private Document doc;
+    private PulsingTermState state;
+    private int numDocs;
+    private final int[] docDeltas;
+    private final int[] freqs;
+    private int[] positionDeltas;
+    private int numPositions;
+    private boolean doFreqs;
+    private boolean doPositions;
+
+    public PulsingBulkPostingsEnum(int maxFreq) {
+      docDeltas = new int[maxFreq];
+      freqs = new int[maxFreq];
+      positionDeltas = new int[maxFreq];
+    }
+
+    PulsingBulkPostingsEnum reset(PulsingTermState termState, boolean doFreqs, boolean doPositions) {
+      numDocs = termState.docFreq;
+      this.doFreqs = doFreqs;
+      this.doPositions = doPositions;
+      assert numDocs <= docDeltas.length;
+      int lastDocID = 0;
+      numPositions = 0;
+      for(int i=0;i<numDocs;i++) {
+        final int docID = termState.docs[i].docID;
+        docDeltas[i] = docID - lastDocID;
+        if (doFreqs) {
+          freqs[i] = termState.docs[i].numPositions;
+          assert freqs[i] > 0;
+          if (doPositions) {
+            final Position[] positions = termState.docs[i].positions;
+            int lastPos = 0;
+            for(int posIndex=0;posIndex<freqs[i];posIndex++) {
+              if (positionDeltas.length == numPositions) {
+                positionDeltas = ArrayUtil.grow(positionDeltas, 1+numPositions);
+              }
+              final int pos = positions[i].pos;
+              positionDeltas[numPositions++] = pos - lastPos;
+              lastPos = pos;
+            }
+          }
+        }
+        lastDocID = docID;
+      }
+      
+      return this;
+    }
+
+    private final BulkPostingsEnum.BlockReader docDeltasReader = new BulkPostingsEnum.BlockReader() {
+      @Override
+      public int[] getBuffer() {
+        return docDeltas;
+      }
+
+      @Override
+      public int fill() {
+        return numDocs;
+      }
+
+      @Override
+      public int offset() {
+        return 0;
+      }
+
+      @Override
+      public void setOffset(int offset) {
+        assert offset == 0;
+      }
+
+      @Override
+      public int end() {
+        return docDeltas.length;
+      }
+    };
+
+    @Override
+    public BulkPostingsEnum.BlockReader getDocDeltasReader() {
+      return docDeltasReader;
+    }
+
+    private final BulkPostingsEnum.BlockReader freqsReader = new BulkPostingsEnum.BlockReader() {
+      @Override
+      public int[] getBuffer() {
+        return freqs;
+      }
+
+      @Override
+      public int fill() {
+        return numDocs;
+      }
+
+      @Override
+      public int offset() {
+        return 0;
+      }
+
+      @Override
+      public void setOffset(int offset) {
+        assert offset == 0;
+      }
+
+      @Override
+      public int end() {
+        return numDocs;
+      }
+    };
+
+    @Override
+    public BulkPostingsEnum.BlockReader getFreqsReader() {
+      return doFreqs ? freqsReader: null;
+    }
+
+    private final BulkPostingsEnum.BlockReader positionDeltasReader = new BulkPostingsEnum.BlockReader() {
+      @Override
+      public int[] getBuffer() {
+        return positionDeltas;
+      }
+
+      @Override
+      public int fill() {
+        return numPositions;
+      }
+
+      @Override
+      public int offset() {
+        return 0;
+      }
+
+      @Override
+      public void setOffset(int offset) {
+        assert offset == 0;
+      }
+
+      @Override
+      public int end() {
+        return positionDeltas.length;
+      }
+    };
+
+    @Override
+    public BulkPostingsEnum.BlockReader getPositionDeltasReader() {
+      return doPositions ? positionDeltasReader : null;
+    }
+
+    @Override
+    public JumpResult jump(int target, int curCount) throws IOException {
+      // TODO: advance is likely unhelpful since apps
+      // "usually" set a lowish docFreq cutoff
+      return null;
+    }
+  }
+
   static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum {
     private int nextRead;
     private int nextPosRead;

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java Tue Dec 14 17:18:00 2010
@@ -67,6 +67,8 @@ public final class PulsingPostingsWriter
     public Object clone() {
       Document doc = new Document();
       doc.docID = docID;
+      // nocommit -- aren't termDocFreq and numPositions the
+      // same thing???
       doc.termDocFreq = termDocFreq;
       doc.numPositions = numPositions;
       doc.positions = new Position[positions.length];

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java Tue Dec 14 17:18:00 2010
@@ -18,72 +18,43 @@ package org.apache.lucene.index.codecs.s
  */
 
 import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.index.BulkPostingsEnum;
 
 import java.io.IOException;
 import java.io.Closeable;
 
-/** Defines basic API for writing ints to an IndexOutput.
- *  IntBlockCodec interacts with this API. @see
- *  IntBlockReader
+// nocommit -- move to oal.store?
+
+/** Defines basic API for reading blocks of ints.  SepCodec
+ *  interacts with this API.
  *
  * @lucene.experimental */
 public abstract class IntIndexInput implements Closeable {
 
-  public abstract Reader reader() throws IOException;
+  public abstract BulkPostingsEnum.BlockReader reader() throws IOException;
 
   public abstract void close() throws IOException;
 
   public abstract Index index() throws IOException;
   
-  // TODO: -- can we simplify this?
   public abstract static class Index {
 
     public abstract void read(IndexInput indexIn, boolean absolute) throws IOException;
 
-    public abstract void read(IntIndexInput.Reader indexIn, boolean absolute) throws IOException;
+    public abstract void read(BulkPostingsEnum.BlockReader indexIn, boolean absolute) throws IOException;
 
-    /** Seeks primary stream to the last read offset */
-    public abstract void seek(IntIndexInput.Reader stream) throws IOException;
+    /** Seeks primary stream to the last read offset.
+     *  Returns true if the seek was "within block", ie
+     *  within the last read block, at which point you
+     *  should call {@link
+     *  BulkPostingsEnum.BlockReader#offset} to know where
+     *  to start from.  If this returns false, you must call
+     *  {@link BulkPostingsEnum.BlockReader#fill} to read
+     *  the buffer. */ 
+    public abstract void seek(BulkPostingsEnum.BlockReader stream) throws IOException;
 
     public abstract void set(Index other);
     
     public abstract Object clone();
   }
-
-  public abstract static class Reader {
-
-    /** Reads next single int */
-    public abstract int next() throws IOException;
-
-    /** Encodes as 1 or 2 ints, and can only use 61 of the 64
-     *  long bits. */
-    public long readVLong() throws IOException {
-      final int v = next();
-      if ((v & 1) == 0) {
-        return v >> 1;
-      } else {
-        final long v2 = next();
-        return (v2 << 30) | (v >> 1);
-      }
-    }
-
-    /** Reads next chunk of ints */
-    private IntsRef bulkResult;
-
-    /** Read up to count ints. */
-    public IntsRef read(int count) throws IOException {
-      if (bulkResult == null) {
-        bulkResult = new IntsRef();
-        bulkResult.ints = new int[count];
-      } else {
-        bulkResult.grow(count);
-      }
-      for(int i=0;i<count;i++) {
-        bulkResult.ints[i] = next();
-      }
-      bulkResult.length = count;
-      return bulkResult;
-    }
-  }
 }

Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java?rev=1049178&r1=1049177&r2=1049178&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexOutput.java Tue Dec 14 17:18:00 2010
@@ -25,6 +25,8 @@ import org.apache.lucene.store.IndexOutp
 import java.io.IOException;
 import java.io.Closeable;
 
+// nocommit -- move to oal.store?
+
 /** Defines basic API for writing ints to an IndexOutput.
  *  IntBlockCodec interacts with this API. @see
  *  IntBlockReader.