You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2011/02/09 10:36:03 UTC

svn commit: r1068809 [8/36] - in /lucene/dev/branches/docvalues: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/.idea/copyright/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ant/ dev-tools/idea/lucene/contrib/queryparser/ dev-tools/...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java Wed Feb  9 09:35:27 2011
@@ -58,6 +58,7 @@ extends SegmentTermDocs  {
     this.proxStreamOrig = proxStream;  // the proxStream will be cloned lazily when nextPosition() is called for the first time
   }
 
+  @Override
   final void seek(TermInfo ti, Term term) throws IOException {
     super.seek(ti, term);
     if (ti != null)
@@ -69,6 +70,7 @@ extends SegmentTermDocs  {
     needToLoadPayload = false;
   }
 
+  @Override
   public final void close() throws IOException {
     super.close();
     if (proxStream != null) proxStream.close();
@@ -100,11 +102,13 @@ extends SegmentTermDocs  {
     return delta;
   }
   
+  @Override
   protected final void skippingDoc() throws IOException {
     // we remember to skip a document lazily
     lazySkipProxCount += freq;
   }
 
+  @Override
   public final boolean next() throws IOException {
     // we remember to skip the remaining positions of the current
     // document lazily
@@ -118,12 +122,14 @@ extends SegmentTermDocs  {
     return false;
   }
 
+  @Override
   public final int read(final int[] docs, final int[] freqs) {
     throw new UnsupportedOperationException("TermPositions does not support processing multiple documents in one call. Use TermDocs instead.");
   }
 
 
   /** Called by super.skipTo(). */
+  @Override
   protected void skipProx(long proxPointer, int payloadLength) throws IOException {
     // we save the pointer, we might have to skip there lazily
     lazySkipPointer = proxPointer;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Wed Feb  9 09:35:27 2011
@@ -67,15 +67,18 @@ public final class TermInfosReader {
       this.term = t;
     }
 
+    @Override
     public boolean equals(Object other) {
       CloneableTerm t = (CloneableTerm) other;
       return this.term.equals(t.term);
     }
 
+    @Override
     public int hashCode() {
       return term.hashCode();
     }
 
+    @Override
     public Object clone() {
       return new CloneableTerm(term);
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Wed Feb  9 09:35:27 2011
@@ -32,8 +32,8 @@ import org.apache.lucene.index.codecs.Fi
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexReader;
 import org.apache.lucene.index.codecs.FixedGapTermsIndexWriter;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
+import org.apache.lucene.index.codecs.BlockTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.TermsIndexWriterBase;
 import org.apache.lucene.index.codecs.standard.StandardCodec;
@@ -89,7 +89,7 @@ public class PulsingCodec extends Codec 
     // Terms dict
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, pulsingWriter, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -132,13 +132,13 @@ public class PulsingCodec extends Codec 
     // Terms dict reader
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                       state.dir, state.fieldInfos, state.segmentInfo.name,
-                                                       pulsingReader,
-                                                       state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       StandardCodec.TERMS_CACHE_SIZE,
-                                                       state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir, state.fieldInfos, state.segmentInfo.name,
+                                                pulsingReader,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                StandardCodec.TERMS_CACHE_SIZE,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -155,7 +155,7 @@ public class PulsingCodec extends Codec 
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
     StandardPostingsReader.files(dir, segmentInfo, id, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, id, files);
+    BlockTermsReader.files(dir, segmentInfo, id, files);
     FixedGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Wed Feb  9 09:35:27 2011
@@ -19,14 +19,15 @@ package org.apache.lucene.index.codecs.p
 
 import java.io.IOException;
 
+import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.codecs.TermState;
+import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Document;
-import org.apache.lucene.index.codecs.pulsing.PulsingPostingsWriterImpl.Position;
+import org.apache.lucene.index.codecs.BlockTermState;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
@@ -43,7 +44,7 @@ public class PulsingPostingsReaderImpl e
 
   // Fallback reader for non-pulsed terms:
   final PostingsReaderBase wrappedPostingsReader;
-  int maxPulsingDocFreq;
+  int maxPositions;
 
   public PulsingPostingsReaderImpl(PostingsReaderBase wrappedPostingsReader) throws IOException {
     this.wrappedPostingsReader = wrappedPostingsReader;
@@ -53,145 +54,139 @@ public class PulsingPostingsReaderImpl e
   public void init(IndexInput termsIn) throws IOException {
     CodecUtil.checkHeader(termsIn, PulsingPostingsWriterImpl.CODEC,
       PulsingPostingsWriterImpl.VERSION_START, PulsingPostingsWriterImpl.VERSION_START);
-    maxPulsingDocFreq = termsIn.readVInt();
+    maxPositions = termsIn.readVInt();
     wrappedPostingsReader.init(termsIn);
   }
 
-  private static class PulsingTermState extends TermState {
-    private Document docs[];
-    private TermState wrappedTermState;
-    private boolean pendingIndexTerm;
+  private static class PulsingTermState extends BlockTermState {
+    private byte[] postings;
+    private int postingsSize;                     // -1 if this term was not inlined
+    private BlockTermState wrappedTermState;
+
+    ByteArrayDataInput inlinedBytesReader;
+    private byte[] inlinedBytes;
 
+    @Override
     public Object clone() {
       PulsingTermState clone;
       clone = (PulsingTermState) super.clone();
-      clone.docs = docs.clone();
-      for(int i=0;i<clone.docs.length;i++) {
-        final Document doc = clone.docs[i];
-        if (doc != null) {
-          clone.docs[i] = (Document) doc.clone();
-        }
+      if (postingsSize != -1) {
+        clone.postings = new byte[postingsSize];
+        System.arraycopy(postings, 0, clone.postings, 0, postingsSize);
+      } else {
+        assert wrappedTermState != null;
+        clone.wrappedTermState = (BlockTermState) wrappedTermState.clone();
       }
-      clone.wrappedTermState = (TermState) wrappedTermState.clone();
       return clone;
     }
 
-    public void copy(TermState _other) {
-      super.copy(_other);
+    @Override
+    public void copyFrom(TermState _other) {
+      super.copyFrom(_other);
       PulsingTermState other = (PulsingTermState) _other;
-      pendingIndexTerm = other.pendingIndexTerm;
-      wrappedTermState.copy(other.wrappedTermState);
-      for(int i=0;i<docs.length;i++) {
-        if (other.docs[i] != null) {
-          docs[i] = (Document) other.docs[i].clone();
+      postingsSize = other.postingsSize;
+      if (other.postingsSize != -1) {
+        if (postings == null || postings.length < other.postingsSize) {
+          postings = new byte[ArrayUtil.oversize(other.postingsSize, 1)];
         }
+        System.arraycopy(other.postings, 0, postings, 0, other.postingsSize);
+      } else {
+        wrappedTermState.copyFrom(other.wrappedTermState);
+      }
+
+      // NOTE: we do not copy the
+      // inlinedBytes/inlinedBytesReader; these are only
+      // stored on the "primary" TermState.  They are
+      // "transient" to cloned term states.
+    }
+
+    @Override
+    public String toString() {
+      if (postingsSize == -1) {
+        return "PulsingTermState: not inlined: wrapped=" + wrappedTermState;
+      } else {
+        return "PulsingTermState: inlined size=" + postingsSize + " " + super.toString();
       }
     }
   }
 
   @Override
-  public TermState newTermState() throws IOException {
+  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    final PulsingTermState termState = (PulsingTermState) _termState;
+    if (termState.inlinedBytes == null) {
+      termState.inlinedBytes = new byte[128];
+      termState.inlinedBytesReader = new ByteArrayDataInput(null);
+    }
+    int len = termsIn.readVInt();
+    if (termState.inlinedBytes.length < len) {
+      termState.inlinedBytes = new byte[ArrayUtil.oversize(len, 1)];
+    }
+    termsIn.readBytes(termState.inlinedBytes, 0, len);
+    termState.inlinedBytesReader.reset(termState.inlinedBytes);
+    termState.wrappedTermState.termCount = 0;
+    wrappedPostingsReader.readTermsBlock(termsIn, fieldInfo, termState.wrappedTermState);
+  }
+
+  @Override
+  public BlockTermState newTermState() throws IOException {
     PulsingTermState state = new PulsingTermState();
     state.wrappedTermState = wrappedPostingsReader.newTermState();
-    state.docs = new Document[maxPulsingDocFreq];
     return state;
   }
 
   @Override
-  public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, TermState _termState, boolean isIndexTerm) throws IOException {
-
+  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    //System.out.println("PR nextTerm");
     PulsingTermState termState = (PulsingTermState) _termState;
 
-    termState.pendingIndexTerm |= isIndexTerm;
-
-    if (termState.docFreq <= maxPulsingDocFreq) {
-
-      // Inlined into terms dict -- read everything in
-
-      // TODO: maybe only read everything in lazily?  But
-      // then we'd need to store length so we could seek
-      // over it when docs/pos enum was not requested
-
-      // TODO: it'd be better to share this encoding logic
-      // in some inner codec that knows how to write a
-      // single doc / single position, etc.  This way if a
-      // given codec wants to store other interesting
-      // stuff, it could use this pulsing codec to do so
-
-      int docID = 0;
-      for(int i=0;i<termState.docFreq;i++) {
-        Document doc = termState.docs[i];
-        if (doc == null) {
-          doc = termState.docs[i] = new Document();
-        }
-        final int code = termsIn.readVInt();
-        if (fieldInfo.omitTermFreqAndPositions) {
-          docID += code;
-          doc.numPositions = 1;
-        } else {
-          docID += code>>>1;
-          if ((code & 1) != 0) {
-            doc.numPositions = 1;
-          } else {
-            doc.numPositions = termsIn.readVInt();
-          }
-            
-          if (doc.numPositions > doc.positions.length) {
-            doc.reallocPositions(doc.numPositions);
-          }
-
-          int position = 0;
-          int payloadLength = -1;
-
-          for(int j=0;j<doc.numPositions;j++) {
-            final Position pos = doc.positions[j];
-            final int code2 = termsIn.readVInt();
-            if (fieldInfo.storePayloads) {
-              position += code2 >>> 1;
-              if ((code2 & 1) != 0) {
-                payloadLength = termsIn.readVInt();
-              }
-
-              if (payloadLength > 0) {
-                if (pos.payload == null) {
-                  pos.payload = new BytesRef();
-                  pos.payload.bytes = new byte[payloadLength];
-                } else if (payloadLength > pos.payload.bytes.length) {
-                  pos.payload.grow(payloadLength);
-                }
-                pos.payload.length = payloadLength;
-                termsIn.readBytes(pos.payload.bytes, 0, payloadLength);
-              } else if (pos.payload != null) {
-                pos.payload.length = 0;
-              }
-            } else {
-              position += code2;
-            }
-            pos.pos = position;
-          }
-        }
-        doc.docID = docID;
-      }
+    // total TF, but in the omitTFAP case its computed based on docFreq.
+    long count = fieldInfo.omitTermFreqAndPositions ? termState.docFreq : termState.totalTermFreq;
+    //System.out.println("  count=" + count + " threshold=" + maxPositions);
+
+    if (count <= maxPositions) {
+      //System.out.println("  inlined");
+
+      // Inlined into terms dict -- just read the byte[] blob in,
+      // but don't decode it now (we only decode when a DocsEnum
+      // or D&PEnum is pulled):
+      termState.postingsSize = termState.inlinedBytesReader.readVInt();
+      if (termState.postings == null || termState.postings.length < termState.postingsSize) {
+        termState.postings = new byte[ArrayUtil.oversize(termState.postingsSize, 1)];
+      }
+      // TODO: sort of silly to copy from one big byte[]
+      // (the blob holding all inlined terms' blobs for
+      // current term block) into another byte[] (just the
+      // blob for this term)...
+      termState.inlinedBytesReader.readBytes(termState.postings, 0, termState.postingsSize);
     } else {
+      //System.out.println("  not inlined");
+      termState.postingsSize = -1;
+      // TODO: should we do full copyFrom?  much heavier...?
       termState.wrappedTermState.docFreq = termState.docFreq;
-      wrappedPostingsReader.readTerm(termsIn, fieldInfo, termState.wrappedTermState, termState.pendingIndexTerm);
-      termState.pendingIndexTerm = false;
+      termState.wrappedTermState.totalTermFreq = termState.totalTermFreq;
+      wrappedPostingsReader.nextTerm(fieldInfo, termState.wrappedTermState);
+      termState.wrappedTermState.termCount++;
     }
   }
 
   // TODO: we could actually reuse, by having TL that
   // holds the last wrapped reuse, and vice-versa
   @Override
-  public DocsEnum docs(FieldInfo field, TermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
     PulsingTermState termState = (PulsingTermState) _termState;
-    if (termState.docFreq <= maxPulsingDocFreq) {
+    if (termState.postingsSize != -1) {
+      PulsingDocsEnum postings;
       if (reuse instanceof PulsingDocsEnum) {
-        return ((PulsingDocsEnum) reuse).reset(skipDocs, termState);
+        postings = (PulsingDocsEnum) reuse;
+        if (!postings.canReuse(field)) {
+          postings = new PulsingDocsEnum(field);
+        }
       } else {
-        PulsingDocsEnum docsEnum = new PulsingDocsEnum();
-        return docsEnum.reset(skipDocs, termState);
+        postings = new PulsingDocsEnum(field);
       }
+      return postings.reset(skipDocs, termState);
     } else {
+      // TODO: not great that we lose reuse of PulsingDocsEnum in this case:
       if (reuse instanceof PulsingDocsEnum) {
         return wrappedPostingsReader.docs(field, termState.wrappedTermState, skipDocs, null);
       } else {
@@ -202,15 +197,26 @@ public class PulsingPostingsReaderImpl e
 
   // TODO: -- not great that we can't always reuse
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo field, TermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
-    PulsingTermState termState = (PulsingTermState) _termState;
-    if (termState.docFreq <= maxPulsingDocFreq) {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+    if (field.omitTermFreqAndPositions) {
+      return null;
+    }
+    //System.out.println("D&P: field=" + field.name);
+
+    final PulsingTermState termState = (PulsingTermState) _termState;
+
+    if (termState.postingsSize != -1) {
+      PulsingDocsAndPositionsEnum postings;
       if (reuse instanceof PulsingDocsAndPositionsEnum) {
-        return ((PulsingDocsAndPositionsEnum) reuse).reset(skipDocs, termState);
+        postings = (PulsingDocsAndPositionsEnum) reuse;
+        if (!postings.canReuse(field)) {
+          postings = new PulsingDocsAndPositionsEnum(field);
+        }
       } else {
-        PulsingDocsAndPositionsEnum postingsEnum = new PulsingDocsAndPositionsEnum();
-        return postingsEnum.reset(skipDocs, termState);
+        postings = new PulsingDocsAndPositionsEnum(field);
       }
+
+      return postings.reset(skipDocs, termState);
     } else {
       if (reuse instanceof PulsingDocsAndPositionsEnum) {
         return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, skipDocs, null);
@@ -220,63 +226,90 @@ public class PulsingPostingsReaderImpl e
     }
   }
 
-  static class PulsingDocsEnum extends DocsEnum {
-    private int nextRead;
+  private static class PulsingDocsEnum extends DocsEnum {
+    private final ByteArrayDataInput postings = new ByteArrayDataInput(null);
+    private final boolean omitTF;
+    private final boolean storePayloads;
     private Bits skipDocs;
-    private Document doc;
-    private PulsingTermState state;
-
-    public void close() {}
+    private int docID;
+    private int freq;
 
-    PulsingDocsEnum reset(Bits skipDocs, PulsingTermState termState) {
-      // TODO: -- not great we have to clone here --
-      // merging is wasteful; TermRangeQuery too
-      state = (PulsingTermState) termState.clone();
+    public PulsingDocsEnum(FieldInfo fieldInfo) {
+      omitTF = fieldInfo.omitTermFreqAndPositions;
+      storePayloads = fieldInfo.storePayloads;
+    }
+
+    public PulsingDocsEnum reset(Bits skipDocs, PulsingTermState termState) {
+      //System.out.println("PR docsEnum termState=" + termState + " docFreq=" + termState.docFreq);
+      assert termState.postingsSize != -1;
+      final byte[] bytes = new byte[termState.postingsSize];
+      System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
+      postings.reset(bytes);
+      docID = 0;
+      freq = 1;
       this.skipDocs = skipDocs;
-      nextRead = 0;
       return this;
     }
 
+    boolean canReuse(FieldInfo fieldInfo) {
+      return omitTF == fieldInfo.omitTermFreqAndPositions && storePayloads == fieldInfo.storePayloads;
+    }
+
     @Override
-    public int nextDoc() {
+    public int nextDoc() throws IOException {
+      //System.out.println("PR nextDoc this= "+ this);
       while(true) {
-        if (nextRead >= state.docFreq) {
-          return NO_MORE_DOCS;
+        if (postings.eof()) {
+          //System.out.println("PR   END");
+          return docID = NO_MORE_DOCS;
+        }
+
+        final int code = postings.readVInt();
+        if (omitTF) {
+          docID += code;
         } else {
-          doc = state.docs[nextRead++];
-          if (skipDocs == null || !skipDocs.get(doc.docID)) {
-            return doc.docID;
+          docID += code >>> 1;              // shift off low bit
+          if ((code & 1) != 0) {          // if low bit is set
+            freq = 1;                     // freq is one
+          } else {
+            freq = postings.readVInt();     // else read freq
+          }
+
+          // Skip positions
+          if (storePayloads) {
+            int payloadLength = -1;
+            for(int pos=0;pos<freq;pos++) {
+              final int posCode = postings.readVInt();
+              if ((posCode & 1) != 0) {
+                payloadLength = postings.readVInt();
+              }
+              if (payloadLength != 0) {
+                postings.skipBytes(payloadLength);
+              }
+            }
+          } else {
+            for(int pos=0;pos<freq;pos++) {
+              // TODO: skipVInt
+              postings.readVInt();
+            }
           }
         }
-      }
-    }
 
-    @Override
-    public int read() {
-      int i=0;
-      // TODO: -- ob1?
-      initBulkResult();
-      final int[] docs = bulkResult.docs.ints;
-      final int[] freqs = bulkResult.freqs.ints;
-      while(nextRead < state.docFreq) {
-        doc = state.docs[nextRead++];
-        if (skipDocs == null || !skipDocs.get(doc.docID)) {
-          docs[i] = doc.docID;
-          freqs[i] = doc.numPositions;
-          i++;
+        if (skipDocs == null || !skipDocs.get(docID)) {
+          //System.out.println("  return docID=" + docID + " freq=" + freq);
+          return docID;
         }
       }
-      return i;
     }
 
     @Override
     public int freq() {
-      return doc.numPositions;
+      return freq;
     }
 
     @Override
     public int docID() {
-      return doc.docID;
+      return docID;
     }
 
     @Override
@@ -286,57 +319,83 @@ public class PulsingPostingsReaderImpl e
         if (doc >= target)
           return doc;
       }
-      return NO_MORE_DOCS;
+      return docID = NO_MORE_DOCS;
     }
   }
 
-  static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    private int nextRead;
-    private int nextPosRead;
+  private static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum {
+    private final ByteArrayDataInput postings = new ByteArrayDataInput(null);
+    private final boolean storePayloads;
+
     private Bits skipDocs;
-    private Document doc;
-    private Position pos;
-    private PulsingTermState state;
+    private int docID;
+    private int freq;
+    private int posPending;
+    private int position;
+    private int payloadLength;
+    private BytesRef payload;
 
-    // Only here to emulate limitation of standard codec,
-    // which only allows retrieving payload more than once
     private boolean payloadRetrieved;
 
-    public void close() {}
+    public PulsingDocsAndPositionsEnum(FieldInfo fieldInfo) {
+      storePayloads = fieldInfo.storePayloads;
+    }
+
+    boolean canReuse(FieldInfo fieldInfo) {
+      return storePayloads == fieldInfo.storePayloads;
+    }
 
-    PulsingDocsAndPositionsEnum reset(Bits skipDocs, PulsingTermState termState) {
-      // TODO: -- not great we have to clone here --
-      // merging is wasteful; TermRangeQuery too
-      state = (PulsingTermState) termState.clone();
+    public PulsingDocsAndPositionsEnum reset(Bits skipDocs, PulsingTermState termState) {
+      assert termState.postingsSize != -1;
+      final byte[] bytes = new byte[termState.postingsSize];
+      System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
+      postings.reset(bytes);
       this.skipDocs = skipDocs;
-      nextRead = 0;
-      nextPosRead = 0;
+      payloadLength = 0;
+      docID = 0;
+      //System.out.println("PR d&p reset storesPayloads=" + storePayloads + " bytes=" + bytes.length + " this=" + this);
       return this;
     }
 
     @Override
-    public int nextDoc() {
+    public int nextDoc() throws IOException {
+      //System.out.println("PR d&p nextDoc this=" + this);
+
       while(true) {
-        if (nextRead >= state.docFreq) {
-          return NO_MORE_DOCS;
+        //System.out.println("  cycle skip posPending=" + posPending);
+
+        skipPositions();
+
+        if (postings.eof()) {
+          //System.out.println("PR   END");
+          return docID = NO_MORE_DOCS;
+        }
+
+        final int code = postings.readVInt();
+        docID += code >>> 1;            // shift off low bit
+        if ((code & 1) != 0) {          // if low bit is set
+          freq = 1;                     // freq is one
         } else {
-          doc = state.docs[nextRead++];
-          if (skipDocs == null || !skipDocs.get(doc.docID)) {
-            nextPosRead = 0;
-            return doc.docID;
-          }
+          freq = postings.readVInt();     // else read freq
+        }
+        posPending = freq;
+
+        if (skipDocs == null || !skipDocs.get(docID)) {
+          //System.out.println("  return docID=" + docID + " freq=" + freq);
+          position = 0;
+          return docID;
         }
       }
     }
 
     @Override
     public int freq() {
-      return doc.numPositions;
+      return freq;
     }
 
     @Override
     public int docID() {
-      return doc.docID;
+      return docID;
     }
 
     @Override
@@ -347,26 +406,72 @@ public class PulsingPostingsReaderImpl e
           return doc;
         }
       }
-      return NO_MORE_DOCS;
+      return docID = NO_MORE_DOCS;
     }
 
     @Override
-    public int nextPosition() {
-      assert nextPosRead < doc.numPositions;
-      pos = doc.positions[nextPosRead++];
-      payloadRetrieved = false;
-      return pos.pos;
+    public int nextPosition() throws IOException {
+      //System.out.println("PR d&p nextPosition posPending=" + posPending + " vs freq=" + freq);
+      
+      assert posPending > 0;
+      posPending--;
+
+      if (storePayloads) {
+        if (!payloadRetrieved) {
+          //System.out.println("PR     skip payload=" + payloadLength);
+          postings.skipBytes(payloadLength);
+        }
+        final int code = postings.readVInt();
+        //System.out.println("PR     code=" + code);
+        if ((code & 1) != 0) {
+          payloadLength = postings.readVInt();
+          //System.out.println("PR     new payload len=" + payloadLength);
+        }
+        position += code >> 1;
+        payloadRetrieved = false;
+      } else {
+        position += postings.readVInt();
+      }
+
+      //System.out.println("PR d&p nextPos return pos=" + position + " this=" + this);
+      return position;
+    }
+
+    private void skipPositions() throws IOException {
+      while(posPending != 0) {
+        nextPosition();
+      }
+      if (storePayloads && !payloadRetrieved) {
+        //System.out.println("  skip payload len=" + payloadLength);
+        postings.skipBytes(payloadLength);
+        payloadRetrieved = true;
+      }
     }
 
     @Override
     public boolean hasPayload() {
-      return !payloadRetrieved && pos.payload != null && pos.payload.length > 0;
+      return storePayloads && !payloadRetrieved && payloadLength > 0;
     }
 
     @Override
-    public BytesRef getPayload() {
+    public BytesRef getPayload() throws IOException {
+      //System.out.println("PR  getPayload payloadLength=" + payloadLength + " this=" + this);
+      if (payloadRetrieved) {
+        throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
+      }
       payloadRetrieved = true;
-      return pos.payload;
+      if (payloadLength > 0) {
+        if (payload == null) {
+          payload = new BytesRef(payloadLength);
+        } else {
+          payload.grow(payloadLength);
+        }
+        postings.readBytes(payload.bytes, 0, payloadLength);
+        payload.length = payloadLength;
+        return payload;
+      } else {
+        return null;
+      }
     }
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsWriterImpl.java Wed Feb  9 09:35:27 2011
@@ -20,17 +20,17 @@ package org.apache.lucene.index.codecs.p
 import java.io.IOException;
 
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.CodecUtil;
 
-// TODO: we now pulse entirely according to docFreq of the
-// term; it might be better to eg pulse by "net bytes used"
-// so that a term that has only 1 doc but zillions of
-// positions would not be inlined.  Though this is
+// TODO: we now inline based on total TF of the term,
+// but it might be better to inline by "net bytes used"
+// so that a term that has only 1 posting but a huge
+// payload would not be inlined.  Though this is
 // presumably rare in practice...
 
 /** @lucene.experimental */
@@ -44,86 +44,42 @@ public final class PulsingPostingsWriter
 
   final static int VERSION_CURRENT = VERSION_START;
 
-  IndexOutput termsOut;
-
-  boolean omitTF;
-  boolean storePayloads;
-
-  // Starts a new term
-  FieldInfo fieldInfo;
+  private IndexOutput termsOut;
 
-  /** @lucene.experimental */
-  public static class Document {
-    int docID;
-    int termDocFreq;
-    int numPositions;
-    Position[] positions;
-    Document() {
-      positions = new Position[1];
-      positions[0] = new Position();
-    }
-    
-    @Override
-    public Object clone() {
-      Document doc = new Document();
-      doc.docID = docID;
-      doc.termDocFreq = termDocFreq;
-      doc.numPositions = numPositions;
-      doc.positions = new Position[positions.length];
-      for(int i = 0; i < positions.length; i++) {
-        doc.positions[i] = (Position) positions[i].clone();
-      }
+  private boolean omitTF;
+  private boolean storePayloads;
 
-      return doc;
-    }
+  // one entry per position
+  private final Position[] pending;
+  private int pendingCount = 0;                           // -1 once we've hit too many positions
+  private Position currentDoc;                    // first Position entry of current doc
 
-    void reallocPositions(int minSize) {
-      final Position[] newArray = new Position[ArrayUtil.oversize(minSize, RamUsageEstimator.NUM_BYTES_OBJECT_REF)];
-      System.arraycopy(positions, 0, newArray, 0, positions.length);
-      for(int i=positions.length;i<newArray.length;i++) {
-        newArray[i] = new Position();
-      }
-      positions = newArray;
-    }
-  }
-
-  final Document[] pendingDocs;
-  int pendingDocCount = 0;
-  Document currentDoc;
-  boolean pulsed;                                 // false if we've seen > maxPulsingDocFreq docs
-
-  static class Position {
+  private static final class Position {
     BytesRef payload;
+    int termFreq;                                 // only incremented on first position for a given doc
     int pos;
-    
-    @Override
-    public Object clone() {
-      Position position = new Position();
-      position.pos = pos;
-      if (payload != null) {
-        position.payload = new BytesRef(payload);
-      }
-      return position;
-    }
+    int docID;
   }
 
   // TODO: -- lazy init this?  ie, if every single term
-  // was pulsed then we never need to use this fallback?
-  // Fallback writer for non-pulsed terms:
+  // was inlined (eg for a "primary key" field) then we
+  // never need to use this fallback?  Fallback writer for
+  // non-inlined terms:
   final PostingsWriterBase wrappedPostingsWriter;
 
-  /** If docFreq <= maxPulsingDocFreq, its postings are
+  /** If the total number of positions (summed across all docs
+   *  for this term) is <= maxPositions, then the postings are
    *  inlined into terms dict */
-  public PulsingPostingsWriterImpl(int maxPulsingDocFreq, PostingsWriterBase wrappedPostingsWriter) throws IOException {
+  public PulsingPostingsWriterImpl(int maxPositions, PostingsWriterBase wrappedPostingsWriter) throws IOException {
     super();
 
-    pendingDocs = new Document[maxPulsingDocFreq];
-    for(int i=0;i<maxPulsingDocFreq;i++) {
-      pendingDocs[i] = new Document();
+    pending = new Position[maxPositions];
+    for(int i=0;i<maxPositions;i++) {
+      pending[i] = new Position();
     }
 
     // We simply wrap another postings writer, but only call
-    // on it when doc freq is higher than our cutoff
+    // on it when tot positions is >= the cutoff:
     this.wrappedPostingsWriter = wrappedPostingsWriter;
   }
 
@@ -131,14 +87,14 @@ public final class PulsingPostingsWriter
   public void start(IndexOutput termsOut) throws IOException {
     this.termsOut = termsOut;
     CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
-    termsOut.writeVInt(pendingDocs.length);
+    termsOut.writeVInt(pending.length); // encode maxPositions in header
     wrappedPostingsWriter.start(termsOut);
   }
 
   @Override
   public void startTerm() {
-    assert pendingDocCount == 0;
-    pulsed = false;
+    //System.out.println("PW   startTerm");
+    assert pendingCount == 0;
   }
 
   // TODO: -- should we NOT reuse across fields?  would
@@ -148,73 +104,56 @@ public final class PulsingPostingsWriter
   // our parent calls setField whenever the field changes
   @Override
   public void setField(FieldInfo fieldInfo) {
-    this.fieldInfo = fieldInfo;
     omitTF = fieldInfo.omitTermFreqAndPositions;
+    //System.out.println("PW field=" + fieldInfo.name + " omitTF=" + omitTF);
     storePayloads = fieldInfo.storePayloads;
     wrappedPostingsWriter.setField(fieldInfo);
   }
 
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
-
     assert docID >= 0: "got docID=" + docID;
-        
-    if (!pulsed && pendingDocCount == pendingDocs.length) {
-      
-      // OK we just crossed the threshold, this term should
-      // now be written with our wrapped codec:
-      wrappedPostingsWriter.startTerm();
-      
-      // Flush all buffered docs
-      for(int i=0;i<pendingDocCount;i++) {
-        final Document doc = pendingDocs[i];
-
-        wrappedPostingsWriter.startDoc(doc.docID, doc.termDocFreq);
-
-        if (!omitTF) {
-          assert doc.termDocFreq == doc.numPositions;
-          for(int j=0;j<doc.termDocFreq;j++) {
-            final Position pos = doc.positions[j];
-            if (pos.payload != null && pos.payload.length > 0) {
-              assert storePayloads;
-              wrappedPostingsWriter.addPosition(pos.pos, pos.payload);
-            } else {
-              wrappedPostingsWriter.addPosition(pos.pos, null);
-            }
-          }
-          wrappedPostingsWriter.finishDoc();
-        }
-      }
+    //System.out.println("PW     doc=" + docID);
 
-      pendingDocCount = 0;
-
-      pulsed = true;
+    if (pendingCount == pending.length) {
+      push();
+      //System.out.println("PW: wrapped.finishDoc");
+      wrappedPostingsWriter.finishDoc();
     }
 
-    if (pulsed) {
+    if (pendingCount != -1) {
+      assert pendingCount < pending.length;
+      currentDoc = pending[pendingCount];
+      currentDoc.docID = docID;
+      if (omitTF) {
+        pendingCount++;
+      } else {
+        currentDoc.termFreq = termDocFreq;
+      }
+    } else {
       // We've already seen too many docs for this term --
       // just forward to our fallback writer
       wrappedPostingsWriter.startDoc(docID, termDocFreq);
-    } else {
-      currentDoc = pendingDocs[pendingDocCount++];
-      currentDoc.docID = docID;
-      // TODO: -- need not store in doc?  only used for alloc & assert
-      currentDoc.termDocFreq = termDocFreq;
-      if (termDocFreq > currentDoc.positions.length) {
-        currentDoc.reallocPositions(termDocFreq);
-      }
-      currentDoc.numPositions = 0;
     }
   }
 
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
-    if (pulsed) {
+
+    //System.out.println("PW       pos=" + position + " payload=" + (payload == null ? "null" : payload.length + " bytes"));
+    if (pendingCount == pending.length) {
+      push();
+    }
+
+    if (pendingCount == -1) {
+      // We've already seen too many docs for this term --
+      // just forward to our fallback writer
       wrappedPostingsWriter.addPosition(position, payload);
     } else {
-      // just buffer up
-      Position pos = currentDoc.positions[currentDoc.numPositions++];
+      // buffer up
+      final Position pos = pending[pendingCount++];
       pos.pos = position;
+      pos.docID = currentDoc.docID;
       if (payload != null && payload.length > 0) {
         if (pos.payload == null) {
           pos.payload = new BytesRef(payload);
@@ -229,86 +168,146 @@ public final class PulsingPostingsWriter
 
   @Override
   public void finishDoc() throws IOException {
-    assert omitTF || currentDoc.numPositions == currentDoc.termDocFreq;
-    if (pulsed) {
+    //System.out.println("PW     finishDoc");
+    if (pendingCount == -1) {
       wrappedPostingsWriter.finishDoc();
     }
   }
 
-  boolean pendingIsIndexTerm;
-
-  int pulsedCount;
-  int nonPulsedCount;
+  private final RAMOutputStream buffer = new RAMOutputStream();
+  private final RAMOutputStream buffer2 = new RAMOutputStream();
 
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(int docCount, boolean isIndexTerm) throws IOException {
-
-    assert docCount > 0;
+  public void finishTerm(TermStats stats) throws IOException {
+    //System.out.println("PW   finishTerm docCount=" + stats.docFreq);
 
-    pendingIsIndexTerm |= isIndexTerm;
+    assert pendingCount > 0 || pendingCount == -1;
 
-    if (pulsed) {
-      wrappedPostingsWriter.finishTerm(docCount, pendingIsIndexTerm);
-      pendingIsIndexTerm = false;
-      pulsedCount++;
+    if (pendingCount == -1) {
+      wrappedPostingsWriter.finishTerm(stats);
     } else {
-      nonPulsedCount++;
-      // OK, there were few enough occurrences for this
+
+      // There were few enough total occurrences for this
       // term, so we fully inline our postings data into
       // terms dict, now:
-      int lastDocID = 0;
-      for(int i=0;i<pendingDocCount;i++) {
-        final Document doc = pendingDocs[i];
-        final int delta = doc.docID - lastDocID;
-        lastDocID = doc.docID;
-        if (omitTF) {
-          termsOut.writeVInt(delta);
-        } else {
-          assert doc.numPositions == doc.termDocFreq;
-          if (doc.numPositions == 1)
-            termsOut.writeVInt((delta<<1)|1);
-          else {
-            termsOut.writeVInt(delta<<1);
-            termsOut.writeVInt(doc.numPositions);
+
+      // TODO: it'd be better to share this encoding logic
+      // in some inner codec that knows how to write a
+      // single doc / single position, etc.  This way if a
+      // given codec wants to store other interesting
+      // stuff, it could use this pulsing codec to do so
+
+      if (!omitTF) {
+        int lastDocID = 0;
+        int pendingIDX = 0;
+        while(pendingIDX < pendingCount) {
+          final Position doc = pending[pendingIDX];
+
+          final int delta = doc.docID - lastDocID;
+          lastDocID = doc.docID;
+
+          //System.out.println("  write doc=" + doc.docID + " freq=" + doc.termFreq);
+
+          if (doc.termFreq == 1) {
+            buffer.writeVInt((delta<<1)|1);
+          } else {
+            buffer.writeVInt(delta<<1);
+            buffer.writeVInt(doc.termFreq);
           }
 
-          // TODO: we could do better in encoding
-          // payloadLength, eg, if it's always the same
-          // across all terms
-          int lastPosition = 0;
+          int lastPos = 0;
           int lastPayloadLength = -1;
-
-          for(int j=0;j<doc.numPositions;j++) {
-            final Position pos = doc.positions[j];
-            final int delta2 = pos.pos - lastPosition;
-            lastPosition = pos.pos;
+          for(int posIDX=0;posIDX<doc.termFreq;posIDX++) {
+            final Position pos = pending[pendingIDX++];
+            assert pos.docID == doc.docID;
+            final int posDelta = pos.pos - lastPos;
+            lastPos = pos.pos;
+            //System.out.println("    write pos=" + pos.pos);
             if (storePayloads) {
               final int payloadLength = pos.payload == null ? 0 : pos.payload.length;
               if (payloadLength != lastPayloadLength) {
-                termsOut.writeVInt((delta2 << 1)|1);
-                termsOut.writeVInt(payloadLength);
+                buffer.writeVInt((posDelta << 1)|1);
+                buffer.writeVInt(payloadLength);
                 lastPayloadLength = payloadLength;
               } else {
-                termsOut.writeVInt(delta2 << 1);
+                buffer.writeVInt(posDelta << 1);
               }
-
               if (payloadLength > 0) {
-                termsOut.writeBytes(pos.payload.bytes, 0, pos.payload.length);
+                buffer.writeBytes(pos.payload.bytes, 0, pos.payload.length);
               }
             } else {
-              termsOut.writeVInt(delta2);
+              buffer.writeVInt(posDelta);
             }
           }
         }
+      } else {
+        int lastDocID = 0;
+        for(int posIDX=0;posIDX<pendingCount;posIDX++) {
+          final Position doc = pending[posIDX];
+          buffer.writeVInt(doc.docID - lastDocID);
+          lastDocID = doc.docID;
+        }
       }
+      
+      //System.out.println("  bytes=" + buffer.getFilePointer());
+      buffer2.writeVInt((int) buffer.getFilePointer());
+      buffer.writeTo(buffer2);
+      buffer.reset();
     }
 
-    pendingDocCount = 0;
+    pendingCount = 0;
   }
 
   @Override
   public void close() throws IOException {
     wrappedPostingsWriter.close();
   }
+
+  @Override
+  public void flushTermsBlock() throws IOException {
+    termsOut.writeVInt((int) buffer2.getFilePointer());
+    buffer2.writeTo(termsOut);
+    buffer2.reset();
+
+    // TODO: can we avoid calling this if all terms
+    // were inlined...?  Eg for a "primary key" field, the
+    // wrapped codec is never invoked...
+    wrappedPostingsWriter.flushTermsBlock();
+  }
+
+  // Pushes pending positions to the wrapped codec
+  private void push() throws IOException {
+    //System.out.println("PW now push @ " + pendingCount + " wrapped=" + wrappedPostingsWriter);
+    assert pendingCount == pending.length;
+      
+    wrappedPostingsWriter.startTerm();
+      
+    // Flush all buffered docs
+    if (!omitTF) {
+      Position doc = null;
+      for(Position pos : pending) {
+        if (doc == null) {
+          doc = pos;
+          //System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
+          wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
+        } else if (doc.docID != pos.docID) {
+          assert pos.docID > doc.docID;
+          //System.out.println("PW: wrapped.finishDoc");
+          wrappedPostingsWriter.finishDoc();
+          doc = pos;
+          //System.out.println("PW: wrapped.startDoc docID=" + doc.docID + " tf=" + doc.termFreq);
+          wrappedPostingsWriter.startDoc(doc.docID, doc.termFreq);
+        }
+        //System.out.println("PW:   wrapped.addPos pos=" + pos.pos);
+        wrappedPostingsWriter.addPosition(pos.pos, pos.payload);
+      }
+      //wrappedPostingsWriter.finishDoc();
+    } else {
+      for(Position doc : pending) {
+        wrappedPostingsWriter.startDoc(doc.docID, 0);
+      }
+    }
+    pendingCount = -1;
+  }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/IntIndexInput.java Wed Feb  9 09:35:27 2011
@@ -17,11 +17,11 @@ package org.apache.lucene.index.codecs.s
  * limitations under the License.
  */
 
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.IntsRef;
-
-import java.io.IOException;
 import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.util.IntsRef;
 
 /** Defines basic API for writing ints to an IndexOutput.
  *  IntBlockCodec interacts with this API. @see
@@ -39,7 +39,7 @@ public abstract class IntIndexInput impl
   // TODO: -- can we simplify this?
   public abstract static class Index {
 
-    public abstract void read(IndexInput indexIn, boolean absolute) throws IOException;
+    public abstract void read(DataInput indexIn, boolean absolute) throws IOException;
 
     public abstract void read(IntIndexInput.Reader indexIn, boolean absolute) throws IOException;
 
@@ -48,6 +48,7 @@ public abstract class IntIndexInput impl
 
     public abstract void set(Index other);
     
+    @Override
     public abstract Object clone();
   }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Wed Feb  9 09:35:27 2011
@@ -20,15 +20,18 @@ package org.apache.lucene.index.codecs.s
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.index.codecs.BlockTermState;
 import org.apache.lucene.index.codecs.PostingsReaderBase;
-import org.apache.lucene.index.codecs.TermState;
+import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
@@ -129,44 +132,120 @@ public class SepPostingsReaderImpl exten
     }
   }
 
-  private static class SepTermState extends TermState {
+  private static final class SepTermState extends BlockTermState {
     // We store only the seek point to the docs file because
     // the rest of the info (freqIndex, posIndex, etc.) is
     // stored in the docs file:
     IntIndexInput.Index docIndex;
+    IntIndexInput.Index posIndex;
+    IntIndexInput.Index freqIndex;
+    long payloadFP;
+    long skipFP;
+
+    // Only used for "primary" term state; these are never
+    // copied on clone:
+    byte[] bytes;
+    ByteArrayDataInput bytesReader;
 
+    @Override
     public Object clone() {
       SepTermState other = (SepTermState) super.clone();
       other.docIndex = (IntIndexInput.Index) docIndex.clone();
+      if (freqIndex != null) {
+        other.freqIndex = (IntIndexInput.Index) freqIndex.clone();
+      }
+      if (posIndex != null) {
+        other.posIndex = (IntIndexInput.Index) posIndex.clone();
+      }
       return other;
     }
 
-    public void copy(TermState _other) {
-      super.copy(_other);
+    @Override
+    public void copyFrom(TermState _other) {
+      super.copyFrom(_other);
       SepTermState other = (SepTermState) _other;
       docIndex.set(other.docIndex);
+      if (freqIndex != null && other.freqIndex != null) {
+        freqIndex.set(other.freqIndex);
+      }
+      if (posIndex != null && other.posIndex != null) {
+        posIndex.set(other.posIndex);
+      }
+      payloadFP = other.payloadFP;
+      skipFP = other.skipFP;
     }
 
     @Override
     public String toString() {
-      return "tis.fp=" + filePointer + " docFreq=" + docFreq + " ord=" + ord + " docIndex=" + docIndex;
+      return super.toString() + " docIndex=" + docIndex + " freqIndex=" + freqIndex + " posIndex=" + posIndex + " payloadFP=" + payloadFP + " skipFP=" + skipFP;
     }
   }
 
   @Override
-  public TermState newTermState() throws IOException {
-    final SepTermState state =  new SepTermState();
+  public BlockTermState newTermState() throws IOException {
+    final SepTermState state = new SepTermState();
     state.docIndex = docIn.index();
+    if (freqIn != null) {
+      state.freqIndex = freqIn.index();
+    }
+    if (posIn != null) {
+      state.posIndex = posIn.index();
+    }
     return state;
   }
 
   @Override
-  public void readTerm(IndexInput termsIn, FieldInfo fieldInfo, TermState termState, boolean isIndexTerm) throws IOException {
-    ((SepTermState) termState).docIndex.read(termsIn, isIndexTerm);
+  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    final SepTermState termState = (SepTermState) _termState;
+    final int len = termsIn.readVInt();
+    //System.out.println("SepR.readTermsBlock len=" + len);
+    if (termState.bytes == null) {
+      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
+      termState.bytesReader = new ByteArrayDataInput(termState.bytes);
+    } else if (termState.bytes.length < len) {
+      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
+    }
+    termState.bytesReader.reset(termState.bytes, 0, len);
+    termsIn.readBytes(termState.bytes, 0, len);
+  }
+
+  @Override
+  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
+    final SepTermState termState = (SepTermState) _termState;
+    //System.out.println("SepR.nextTerm termCount=" + termState.termCount);
+    //System.out.println("  docFreq=" + termState.docFreq);
+    final boolean isFirstTerm = termState.termCount == 0;
+    termState.docIndex.read(termState.bytesReader, isFirstTerm);
+    //System.out.println("  docIndex=" + termState.docIndex);
+    if (!fieldInfo.omitTermFreqAndPositions) {
+      termState.freqIndex.read(termState.bytesReader, isFirstTerm);
+      //System.out.println("  freqIndex=" + termState.freqIndex);
+      termState.posIndex.read(termState.bytesReader, isFirstTerm);
+      //System.out.println("  posIndex=" + termState.posIndex);
+      if (fieldInfo.storePayloads) {
+        if (isFirstTerm) {
+          termState.payloadFP = termState.bytesReader.readVLong();
+        } else {
+          termState.payloadFP += termState.bytesReader.readVLong();
+        }
+        //System.out.println("  payloadFP=" + termState.payloadFP);
+      }
+    }
+    if (termState.docFreq >= skipInterval) {
+      //System.out.println("   readSkip @ " + termState.bytesReader.pos);
+      if (isFirstTerm) {
+        termState.skipFP = termState.bytesReader.readVLong();
+      } else {
+        termState.skipFP += termState.bytesReader.readVLong();
+      }
+      //System.out.println("  skipFP=" + termState.skipFP);
+    } else if (isFirstTerm) {
+      termState.skipFP = termState.bytesReader.readVLong();
+    }
   }
 
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, TermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
     SepDocsEnum docsEnum;
     if (reuse == null || !(reuse instanceof SepDocsEnum)) {
@@ -185,7 +264,7 @@ public class SepPostingsReaderImpl exten
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, TermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
     assert !fieldInfo.omitTermFreqAndPositions;
     final SepTermState termState = (SepTermState) _termState;
     SepDocsAndPositionsEnum postingsEnum;
@@ -217,7 +296,7 @@ public class SepPostingsReaderImpl exten
     private Bits skipDocs;
     private final IntIndexInput.Reader docReader;
     private final IntIndexInput.Reader freqReader;
-    private long skipOffset;
+    private long skipFP;
 
     private final IntIndexInput.Index docIndex;
     private final IntIndexInput.Index freqIndex;
@@ -258,18 +337,15 @@ public class SepPostingsReaderImpl exten
       docIndex.seek(docReader);
 
       if (!omitTF) {
-        freqIndex.read(docReader, true);
+        freqIndex.set(termState.freqIndex);
         freqIndex.seek(freqReader);
-        
-        posIndex.read(docReader, true);
-        // skip payload offset
-        docReader.readVLong();
       } else {
         freq = 1;
       }
-      skipOffset = docReader.readVLong();
 
       docFreq = termState.docFreq;
+      // NOTE: unused if docFreq < skipInterval:
+      skipFP = termState.skipFP;
       count = 0;
       doc = 0;
       skipped = false;
@@ -288,9 +364,11 @@ public class SepPostingsReaderImpl exten
         count++;
 
         // Decode next doc
+        //System.out.println("decode docDelta:");
         doc += docReader.next();
           
         if (!omitTF) {
+          //System.out.println("decode freq:");
           freq = freqReader.next();
         }
 
@@ -298,13 +376,13 @@ public class SepPostingsReaderImpl exten
           break;
         }
       }
-
       return doc;
     }
 
     @Override
     public int read() throws IOException {
       // TODO: -- switch to bulk read api in IntIndexInput
+      //System.out.println("sepdocs read");
       final int[] docs = bulkResult.docs.ints;
       final int[] freqs = bulkResult.freqs.ints;
       int i = 0;
@@ -312,14 +390,17 @@ public class SepPostingsReaderImpl exten
       while (i < length && count < docFreq) {
         count++;
         // manually inlined call to next() for speed
+        //System.out.println("decode doc");
         doc += docReader.next();
         if (!omitTF) {
+          //System.out.println("decode freq");
           freq = freqReader.next();
         }
 
         if (skipDocs == null || !skipDocs.get(doc)) {
           docs[i] = doc;
           freqs[i] = freq;
+          //System.out.println("  docs[" + i + "]=" + doc + " count=" + count + " dF=" + docFreq);
           i++;
         }
       }
@@ -359,7 +440,7 @@ public class SepPostingsReaderImpl exten
 
         if (!skipped) {
           // We haven't yet skipped for this posting
-          skipper.init(skipOffset,
+          skipper.init(skipFP,
                        docIndex,
                        freqIndex,
                        posIndex,
@@ -409,14 +490,14 @@ public class SepPostingsReaderImpl exten
     private final IntIndexInput.Reader freqReader;
     private final IntIndexInput.Reader posReader;
     private final IndexInput payloadIn;
-    private long skipOffset;
+    private long skipFP;
 
     private final IntIndexInput.Index docIndex;
     private final IntIndexInput.Index freqIndex;
     private final IntIndexInput.Index posIndex;
     private final IntIndexInput startDocIn;
 
-    private long payloadOffset;
+    private long payloadFP;
 
     private int pendingPosCount;
     private int position;
@@ -442,21 +523,26 @@ public class SepPostingsReaderImpl exten
     SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits skipDocs) throws IOException {
       this.skipDocs = skipDocs;
       storePayloads = fieldInfo.storePayloads;
+      //System.out.println("Sep D&P init");
 
       // TODO: can't we only do this if consumer
       // skipped consuming the previous docs?
       docIndex.set(termState.docIndex);
       docIndex.seek(docReader);
+      //System.out.println("  docIndex=" + docIndex);
 
-      freqIndex.read(docReader, true);
+      freqIndex.set(termState.freqIndex);
       freqIndex.seek(freqReader);
+      //System.out.println("  freqIndex=" + freqIndex);
 
-      posIndex.read(docReader, true);
+      posIndex.set(termState.posIndex);
+      //System.out.println("  posIndex=" + posIndex);
       posSeekPending = true;
       payloadPending = false;
 
-      payloadOffset = docReader.readVLong();
-      skipOffset = docReader.readVLong();
+      payloadFP = termState.payloadFP;
+      skipFP = termState.skipFP;
+      //System.out.println("  skipFP=" + skipFP);
 
       docFreq = termState.docFreq;
       count = 0;
@@ -482,8 +568,10 @@ public class SepPostingsReaderImpl exten
         // freq=1 case?
 
         // Decode next doc
+        //System.out.println("  sep d&p read doc");
         doc += docReader.next();
-          
+
+        //System.out.println("  sep d&p read freq");
         freq = freqReader.next();
 
         pendingPosCount += freq;
@@ -509,6 +597,7 @@ public class SepPostingsReaderImpl exten
 
     @Override
     public int advance(int target) throws IOException {
+      //System.out.println("SepD&P advance target=" + target + " vs current=" + doc + " this=" + this);
 
       // TODO: jump right to next() if target is < X away
       // from where we are now?
@@ -519,6 +608,7 @@ public class SepPostingsReaderImpl exten
         // skip data
 
         if (skipper == null) {
+          //System.out.println("  create skipper");
           // This DocsEnum has never done any skipping
           skipper = new SepSkipListReader((IndexInput) skipIn.clone(),
                                           freqIn,
@@ -528,46 +618,54 @@ public class SepPostingsReaderImpl exten
         }
 
         if (!skipped) {
+          //System.out.println("  init skip data skipFP=" + skipFP);
           // We haven't yet skipped for this posting
-          skipper.init(skipOffset,
+          skipper.init(skipFP,
                        docIndex,
                        freqIndex,
                        posIndex,
-                       payloadOffset,
+                       payloadFP,
                        docFreq,
                        storePayloads);
 
           skipped = true;
         }
-
         final int newCount = skipper.skipTo(target); 
+        //System.out.println("  skip newCount=" + newCount + " vs " + count);
 
         if (newCount > count) {
 
           // Skipper did move
           skipper.getFreqIndex().seek(freqReader);
           skipper.getDocIndex().seek(docReader);
-          //skipper.getPosIndex().seek(posReader);
+          // NOTE: don't seek pos here; do it lazily
+          // instead.  Eg a PhraseQuery may skip to many
+          // docs before finally asking for positions...
           posIndex.set(skipper.getPosIndex());
           posSeekPending = true;
           count = newCount;
           doc = skipper.getDoc();
+          //System.out.println("    moved to doc=" + doc);
           //payloadIn.seek(skipper.getPayloadPointer());
-          payloadOffset = skipper.getPayloadPointer();
+          payloadFP = skipper.getPayloadPointer();
           pendingPosCount = 0;
           pendingPayloadBytes = 0;
           payloadPending = false;
           payloadLength = skipper.getPayloadLength();
+          //System.out.println("    move payloadLen=" + payloadLength);
         }
       }
         
       // Now, linear scan for the rest:
       do {
         if (nextDoc() == NO_MORE_DOCS) {
+          //System.out.println("  advance nextDoc=END");
           return NO_MORE_DOCS;
         }
+        //System.out.println("  advance nextDoc=" + doc);
       } while (target > doc);
 
+      //System.out.println("  return doc=" + doc);
       return doc;
     }
 
@@ -575,7 +673,7 @@ public class SepPostingsReaderImpl exten
     public int nextPosition() throws IOException {
       if (posSeekPending) {
         posIndex.seek(posReader);
-        payloadIn.seek(payloadOffset);
+        payloadIn.seek(payloadFP);
         posSeekPending = false;
       }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java Wed Feb  9 09:35:27 2011
@@ -25,7 +25,9 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.codecs.PostingsWriterBase;
+import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
 
@@ -68,8 +70,7 @@ public final class SepPostingsWriterImpl
   boolean storePayloads;
   boolean omitTF;
 
-  // Starts a new term
-  long lastSkipStart;
+  long lastSkipFP;
 
   FieldInfo fieldInfo;
 
@@ -79,7 +80,10 @@ public final class SepPostingsWriterImpl
   long lastPayloadStart;
   int lastDocID;
   int df;
-  private boolean firstDoc;
+  private int pendingTermCount;
+
+  // Holds pending byte[] blob for the current terms block
+  private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
 
   public SepPostingsWriterImpl(SegmentWriteState state, IntStreamFactory factory) throws IOException {
     super();
@@ -143,13 +147,9 @@ public final class SepPostingsWriterImpl
       payloadStart = payloadOut.getFilePointer();
       lastPayloadLength = -1;
     }
-    firstDoc = true;
     skipListWriter.resetSkip(docIndex, freqIndex, posIndex);
   }
 
-  // TODO: -- should we NOT reuse across fields?  would
-  // be cleaner
-
   // Currently, this instance is re-used across fields, so
   // our parent calls setField whenever the field changes
   @Override
@@ -160,27 +160,13 @@ public final class SepPostingsWriterImpl
     storePayloads = !omitTF && fieldInfo.storePayloads;
   }
 
-
   /** Adds a new doc in this term.  If this returns null
    *  then we just skip consuming positions/payloads. */
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
 
-    if (firstDoc) {
-      // TODO: we are writing absolute file pointers below,
-      // which is wasteful.  It'd be better compression to
-      // write the "baseline" into each indexed term, then
-      // write only the delta here.
-      if (!omitTF) {
-        freqIndex.write(docOut, true);
-        posIndex.write(docOut, true);
-        docOut.writeVLong(payloadStart);
-      }
-      docOut.writeVLong(skipOut.getFilePointer());
-      firstDoc = false;
-    }
-
     final int delta = docID - lastDocID;
+    //System.out.println("SepW startDoc: write doc=" + docID + " delta=" + delta);
 
     if (docID < 0 || (df > 0 && delta <= 0)) {
       throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " )");
@@ -189,6 +175,7 @@ public final class SepPostingsWriterImpl
     if ((++df % skipInterval) == 0) {
       // TODO: -- awkward we have to make these two
       // separate calls to skipper
+      //System.out.println("    buffer skip lastDocID=" + lastDocID);
       skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
       skipListWriter.bufferSkip(df);
     }
@@ -196,10 +183,20 @@ public final class SepPostingsWriterImpl
     lastDocID = docID;
     docOut.write(delta);
     if (!omitTF) {
+      //System.out.println("    sepw startDoc: write freq=" + termDocFreq);
       freqOut.write(termDocFreq);
     }
   }
 
+  @Override
+  public void flushTermsBlock() throws IOException {
+    //System.out.println("SepW.flushTermsBlock: pendingTermCount=" + pendingTermCount + " bytesUsed=" + indexBytesWriter.getFilePointer());
+    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
+    indexBytesWriter.writeTo(termsOut);
+    indexBytesWriter.reset();
+    pendingTermCount = 0;
+  }
+
   /** Add a new position & payload */
   @Override
   public void addPosition(int position, BytesRef payload) throws IOException {
@@ -239,20 +236,57 @@ public final class SepPostingsWriterImpl
 
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(int docCount, boolean isIndexTerm) throws IOException {
-
+  public void finishTerm(TermStats stats) throws IOException {
     // TODO: -- wasteful we are counting this in two places?
-    assert docCount > 0;
-    assert docCount == df;
+    assert stats.docFreq > 0;
+    assert stats.docFreq == df;
 
-    docIndex.write(termsOut, isIndexTerm);
+    final boolean isFirstTerm = pendingTermCount == 0;  
+    //System.out.println("SepW.finishTerm: isFirstTerm=" + isFirstTerm);
+
+    docIndex.write(indexBytesWriter, isFirstTerm);
+    //System.out.println("  docIndex=" + docIndex);
+
+    if (!omitTF) {
+      freqIndex.write(indexBytesWriter, isFirstTerm);
+      //System.out.println("  freqIndex=" + freqIndex);
+
+      posIndex.write(indexBytesWriter, isFirstTerm);
+      //System.out.println("  posIndex=" + posIndex);
+      if (storePayloads) {
+        if (isFirstTerm) {
+          indexBytesWriter.writeVLong(payloadStart);
+        } else {
+          indexBytesWriter.writeVLong(payloadStart - lastPayloadStart);
+        }
+        lastPayloadStart = payloadStart;
+        //System.out.println("  payloadFP=" + payloadStart);
+      }
+    }
 
     if (df >= skipInterval) {
+      //System.out.println("  skipFP=" + skipStart);
+      final long skipFP = skipOut.getFilePointer();
       skipListWriter.writeSkip(skipOut);
+      //System.out.println("   writeSkip @ " + indexBytesWriter.getFilePointer());
+      if (isFirstTerm) {
+        indexBytesWriter.writeVLong(skipFP);
+      } else {
+        indexBytesWriter.writeVLong(skipFP - lastSkipFP);
+      }
+      lastSkipFP = skipFP;
+    } else if (isFirstTerm) {
+      // TODO: this is somewhat wasteful; eg if no terms in
+      // this block will use skip data, we don't need to
+      // write this:
+      final long skipFP = skipOut.getFilePointer();
+      indexBytesWriter.writeVLong(skipFP);
+      lastSkipFP = skipFP;
     }
 
     lastDocID = 0;
     df = 0;
+    pendingTermCount++;
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Wed Feb  9 09:35:27 2011
@@ -125,28 +125,32 @@ class SimpleTextFieldsReader extends Fie
     private final IndexInput in;
     private final boolean omitTF;
     private int docFreq;
+    private long totalTermFreq;
     private long docsStart;
     private boolean ended;
-    private final BytesRefFSTEnum<PairOutputs.Pair<Long,Long>> fstEnum;
+    private final BytesRefFSTEnum<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> fstEnum;
 
-    public SimpleTextTermsEnum(FST<PairOutputs.Pair<Long,Long>> fst, boolean omitTF) throws IOException {
+    public SimpleTextTermsEnum(FST<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> fst, boolean omitTF) throws IOException {
       this.in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
       this.omitTF = omitTF;
-      fstEnum = new BytesRefFSTEnum<PairOutputs.Pair<Long,Long>>(fst);
+      fstEnum = new BytesRefFSTEnum<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>>(fst);
     }
 
+    @Override
     public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
 
       //System.out.println("seek to text=" + text.utf8ToString());
-      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,Long>> result = fstEnum.seekCeil(text);
+      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> result = fstEnum.seekCeil(text);
       if (result == null) {
         //System.out.println("  end");
         return SeekStatus.END;
       } else {
         //System.out.println("  got text=" + term.utf8ToString());
-        PairOutputs.Pair<Long,Long> pair = result.output;
-        docsStart = pair.output1;
-        docFreq = pair.output2.intValue();
+        PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>> pair1 = result.output;
+        PairOutputs.Pair<Long,Long> pair2 = pair1.output2;
+        docsStart = pair1.output1;
+        docFreq = pair2.output1.intValue();
+        totalTermFreq = pair2.output2;
 
         if (result.input.equals(text)) {
           //System.out.println("  match docsStart=" + docsStart);
@@ -159,17 +163,15 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
-    public void cacheCurrentTerm() {
-    }
-
-    @Override
     public BytesRef next() throws IOException {
       assert !ended;
-      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,Long>> result = fstEnum.next();
+      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> result = fstEnum.next();
       if (result != null) {
-        final PairOutputs.Pair<Long,Long> pair = result.output;
-        docsStart = pair.output1;
-        docFreq = pair.output2.intValue();
+        PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>> pair1 = result.output;
+        PairOutputs.Pair<Long,Long> pair2 = pair1.output2;
+        docsStart = pair1.output1;
+        docFreq = pair2.output1.intValue();
+        totalTermFreq = pair2.output2;
         return result.input;
       } else {
         return null;
@@ -197,6 +199,11 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
+    public long totalTermFreq() {
+      return totalTermFreq;
+    }
+ 
+    @Override
     public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
       SimpleTextDocsEnum docsEnum;
       if (reuse != null && reuse instanceof SimpleTextDocsEnum && ((SimpleTextDocsEnum) reuse).canReuse(in)) {
@@ -221,7 +228,7 @@ class SimpleTextFieldsReader extends Fie
       } 
       return docsAndPositionsEnum.reset(docsStart, skipDocs);
     }
-
+    
     @Override
     public Comparator<BytesRef> getComparator() {
       return BytesRef.getUTF8SortedAsUnicodeComparator();
@@ -446,15 +453,14 @@ class SimpleTextFieldsReader extends Fie
   }
 
   private class SimpleTextTerms extends Terms {
-    private final String field;
     private final long termsStart;
     private final boolean omitTF;
-    private FST<PairOutputs.Pair<Long,Long>> fst;
-
+    private long sumTotalTermFreq;
+    private FST<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> fst;
+    private int termCount;
     private final BytesRef scratch = new BytesRef(10);
 
     public SimpleTextTerms(String field, long termsStart) throws IOException {
-      this.field = StringHelper.intern(field);
       this.termsStart = termsStart;
       omitTF = fieldInfos.fieldInfo(field).omitTermFreqAndPositions;
       loadTerms();
@@ -462,24 +468,38 @@ class SimpleTextFieldsReader extends Fie
 
     private void loadTerms() throws IOException {
       PositiveIntOutputs posIntOutputs = PositiveIntOutputs.getSingleton(false);
-      Builder<PairOutputs.Pair<Long,Long>> b = new Builder<PairOutputs.Pair<Long,Long>>(FST.INPUT_TYPE.BYTE1, 0, 0, true, new PairOutputs<Long,Long>(posIntOutputs, posIntOutputs));
+      final Builder<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> b;
+      b = new Builder<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>>(FST.INPUT_TYPE.BYTE1,
+                                                                          0,
+                                                                          0,
+                                                                          true,
+                                                                          new PairOutputs<Long,PairOutputs.Pair<Long,Long>>(posIntOutputs,
+                                                                                                                            new PairOutputs<Long,Long>(posIntOutputs, posIntOutputs)));
       IndexInput in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
       in.seek(termsStart);
       final BytesRef lastTerm = new BytesRef(10);
       long lastDocsStart = -1;
       int docFreq = 0;
+      long totalTermFreq = 0;
       while(true) {
         readLine(in, scratch);
         if (scratch.equals(END) || scratch.startsWith(FIELD)) {
           if (lastDocsStart != -1) {
-            b.add(lastTerm, new PairOutputs.Pair<Long,Long>(lastDocsStart, Long.valueOf(docFreq)));
+            b.add(lastTerm, new PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>(lastDocsStart,
+                                                                                   new PairOutputs.Pair<Long,Long>((long) docFreq,
+                                                                                                                   posIntOutputs.get(totalTermFreq))));
+            sumTotalTermFreq += totalTermFreq;
           }
           break;
         } else if (scratch.startsWith(DOC)) {
           docFreq++;
+        } else if (scratch.startsWith(POS)) {
+          totalTermFreq++;
         } else if (scratch.startsWith(TERM)) {
           if (lastDocsStart != -1) {
-            b.add(lastTerm, new PairOutputs.Pair<Long,Long>(lastDocsStart, Long.valueOf(docFreq)));
+            b.add(lastTerm, new PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>(lastDocsStart,
+                                                                                   new PairOutputs.Pair<Long,Long>((long) docFreq,
+                                                                                                                   posIntOutputs.get(totalTermFreq))));
           }
           lastDocsStart = in.getFilePointer();
           final int len = scratch.length - TERM.length;
@@ -489,6 +509,9 @@ class SimpleTextFieldsReader extends Fie
           System.arraycopy(scratch.bytes, TERM.length, lastTerm.bytes, 0, len);
           lastTerm.length = len;
           docFreq = 0;
+          sumTotalTermFreq += totalTermFreq;
+          totalTermFreq = 0;
+          termCount++;
         }
       }
       fst = b.finish();
@@ -514,6 +537,16 @@ class SimpleTextFieldsReader extends Fie
     public Comparator<BytesRef> getComparator() {
       return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
+
+    @Override
+    public long getUniqueTermCount() {
+      return (long) termCount;
+    }
+
+    @Override
+    public long getSumTotalTermFreq() {
+      return sumTotalTermFreq;
+    }
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Wed Feb  9 09:35:27 2011
@@ -22,6 +22,7 @@ import org.apache.lucene.util.UnicodeUti
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.index.codecs.PostingsConsumer;
+import org.apache.lucene.index.codecs.TermStats;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.IndexOutput;
@@ -84,11 +85,11 @@ class SimpleTextFieldsWriter extends Fie
     }
 
     @Override
-    public void finishTerm(BytesRef term, int numDocs) throws IOException {
+    public void finishTerm(BytesRef term, TermStats stats) throws IOException {
     }
 
     @Override
-    public void finish() throws IOException {
+    public void finish(long sumTotalTermFreq) throws IOException {
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1068809&r1=1068808&r2=1068809&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Wed Feb  9 09:35:27 2011
@@ -33,8 +33,8 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsIndexReaderBase;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexWriter;
 import org.apache.lucene.index.codecs.VariableGapTermsIndexReader;
-import org.apache.lucene.index.codecs.PrefixCodedTermsWriter;
-import org.apache.lucene.index.codecs.PrefixCodedTermsReader;
+import org.apache.lucene.index.codecs.BlockTermsWriter;
+import org.apache.lucene.index.codecs.BlockTermsReader;
 import org.apache.lucene.store.Directory;
 
 /** Default codec. 
@@ -66,7 +66,7 @@ public class StandardCodec extends Codec
 
     success = false;
     try {
-      FieldsConsumer ret = new PrefixCodedTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
+      FieldsConsumer ret = new BlockTermsWriter(indexWriter, state, docs, BytesRef.getUTF8SortedAsUnicodeComparator());
       success = true;
       return ret;
     } finally {
@@ -103,15 +103,15 @@ public class StandardCodec extends Codec
 
     success = false;
     try {
-      FieldsProducer ret = new PrefixCodedTermsReader(indexReader,
-                                                       state.dir,
-                                                       state.fieldInfos,
-                                                       state.segmentInfo.name,
-                                                       postings,
-                                                       state.readBufferSize,
-                                                       BytesRef.getUTF8SortedAsUnicodeComparator(),
-                                                       TERMS_CACHE_SIZE,
-                                                       state.codecId);
+      FieldsProducer ret = new BlockTermsReader(indexReader,
+                                                state.dir,
+                                                state.fieldInfos,
+                                                state.segmentInfo.name,
+                                                postings,
+                                                state.readBufferSize,
+                                                BytesRef.getUTF8SortedAsUnicodeComparator(),
+                                                TERMS_CACHE_SIZE,
+                                                state.codecId);
       success = true;
       return ret;
     } finally {
@@ -134,7 +134,7 @@ public class StandardCodec extends Codec
   @Override
   public void files(Directory dir, SegmentInfo segmentInfo, String id, Set<String> files) throws IOException {
     StandardPostingsReader.files(dir, segmentInfo, id, files);
-    PrefixCodedTermsReader.files(dir, segmentInfo, id, files);
+    BlockTermsReader.files(dir, segmentInfo, id, files);
     VariableGapTermsIndexReader.files(dir, segmentInfo, id, files);
   }
 
@@ -146,7 +146,7 @@ public class StandardCodec extends Codec
   public static void getStandardExtensions(Set<String> extensions) {
     extensions.add(FREQ_EXTENSION);
     extensions.add(PROX_EXTENSION);
-    PrefixCodedTermsReader.getExtensions(extensions);
+    BlockTermsReader.getExtensions(extensions);
     VariableGapTermsIndexReader.getIndexExtensions(extensions);
   }
 }