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 2012/08/13 15:53:27 UTC

svn commit: r1372423 [29/45] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/common/ dev-tools/maven/lucene/analysis/icu/ d...

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java Mon Aug 13 13:52:46 2012
@@ -49,14 +49,17 @@ import org.apache.lucene.util.FixedBitSe
  */
 public abstract class PostingsConsumer {
 
-  /** Adds a new doc in this term. */
+  /** Adds a new doc in this term. 
+   * <code>freq</code> will be -1 when term frequencies are omitted
+   * for the field. */
   public abstract void startDoc(int docID, int freq) throws IOException;
 
   /** Add a new position & payload, and start/end offset.  A
    *  null payload means no payload; a non-null payload with
    *  zero length also means no payload.  Caller may reuse
    *  the {@link BytesRef} for the payload between calls
-   *  (method must fully consume the payload). */
+   *  (method must fully consume the payload). <code>startOffset</code>
+   *  and <code>endOffset</code> will be -1 when offsets are not indexed. */
   public abstract void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException;
 
   /** Called when we are done adding positions & payloads
@@ -78,7 +81,7 @@ public abstract class PostingsConsumer {
           break;
         }
         visitedDocs.set(doc);
-        this.startDoc(doc, 0);
+        this.startDoc(doc, -1);
         this.finishDoc();
         df++;
       }
@@ -109,12 +112,7 @@ public abstract class PostingsConsumer {
         totTF += freq;
         for(int i=0;i<freq;i++) {
           final int position = postingsEnum.nextPosition();
-          final BytesRef payload;
-          if (postingsEnum.hasPayload()) {
-            payload = postingsEnum.getPayload();
-          } else {
-            payload = null;
-          }
+          final BytesRef payload = postingsEnum.getPayload();
           this.addPosition(position, payload, -1, -1);
         }
         this.finishDoc();
@@ -134,18 +132,13 @@ public abstract class PostingsConsumer {
         totTF += freq;
         for(int i=0;i<freq;i++) {
           final int position = postingsEnum.nextPosition();
-          final BytesRef payload;
-          if (postingsEnum.hasPayload()) {
-            payload = postingsEnum.getPayload();
-          } else {
-            payload = null;
-          }
+          final BytesRef payload = postingsEnum.getPayload();
           this.addPosition(position, payload, postingsEnum.startOffset(), postingsEnum.endOffset());
         }
         this.finishDoc();
         df++;
       }
     }
-    return new TermStats(df, totTF);
+    return new TermStats(df, indexOptions == IndexOptions.DOCS_ONLY ? -1 : totTF);
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsFormat.java Mon Aug 13 13:52:46 2012
@@ -53,7 +53,13 @@ public abstract class PostingsFormat imp
 
   /** Reads a segment.  NOTE: by the time this call
    *  returns, it must hold open any files it will need to
-   *  use; else, those files may be deleted. */
+   *  use; else, those files may be deleted. 
+   *  Additionally, required files may be deleted during the execution of 
+   *  this call before there is a chance to open them. Under these 
+   *  circumstances an IOException should be thrown by the implementation. 
+   *  IOExceptions are expected and will automatically cause a retry of the 
+   *  segment opening logic with the newly revised segments.
+   *  */
   public abstract FieldsProducer fieldsProducer(SegmentReadState state) throws IOException;
 
   @Override
@@ -70,4 +76,19 @@ public abstract class PostingsFormat imp
   public static Set<String> availablePostingsFormats() {
     return loader.availableServices();
   }
+  
+  /** 
+   * Reloads the postings format list from the given {@link ClassLoader}.
+   * Changes to the postings formats are visible after the method ends, all
+   * iterators ({@link #availablePostingsFormats()},...) stay consistent. 
+   * 
+   * <p><b>NOTE:</b> Only new postings formats are added, existing ones are
+   * never removed or replaced.
+   * 
+   * <p><em>This method is expensive and should only be called for discovery
+   * of new postings formats on the given classpath/classloader!</em>
+   */
+  public static void reloadPostingsFormats(ClassLoader classloader) {
+    loader.reload(classloader);
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java Mon Aug 13 13:52:46 2012
@@ -51,12 +51,12 @@ public abstract class PostingsReaderBase
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
-  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse, boolean needsFreqs) throws IOException;
+  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse, int flags) throws IOException;
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
   public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse,
-                                                        boolean needsOffsets) throws IOException;
+                                                        int flags) throws IOException;
 
   public abstract void close() throws IOException;
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Mon Aug 13 13:52:46 2012
@@ -26,8 +26,9 @@ import org.apache.lucene.index.DocsAndPo
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.MergeState;
+import org.apache.lucene.index.PayloadProcessorProvider.PayloadProcessor;
+import org.apache.lucene.index.PayloadProcessorProvider.ReaderPayloadProcessor;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -41,14 +42,14 @@ import org.apache.lucene.util.BytesRef;
  * <ol>
  *   <li>For every document, {@link #startDocument(int)} is called,
  *       informing the Codec how many fields will be written.
- *   <li>{@link #startField(FieldInfo, int, boolean, boolean)} is called for 
+ *   <li>{@link #startField(FieldInfo, int, boolean, boolean, boolean)} is called for 
  *       each field in the document, informing the codec how many terms
- *       will be written for that field, and whether or not positions
- *       or offsets are enabled.
+ *       will be written for that field, and whether or not positions,
+ *       offsets, or payloads are enabled.
  *   <li>Within each field, {@link #startTerm(BytesRef, int)} is called
  *       for each term.
  *   <li>If offsets and/or positions are enabled, then 
- *       {@link #addPosition(int, int, int)} will be called for each term
+ *       {@link #addPosition(int, int, int, BytesRef)} will be called for each term
  *       occurrence.
  *   <li>After all documents have been written, {@link #finish(FieldInfos, int)} 
  *       is called for verification/sanity-checks.
@@ -60,7 +61,7 @@ import org.apache.lucene.util.BytesRef;
 public abstract class TermVectorsWriter implements Closeable {
   
   /** Called before writing the term vectors of the document.
-   *  {@link #startField(FieldInfo, int, boolean, boolean)} will 
+   *  {@link #startField(FieldInfo, int, boolean, boolean, boolean)} will 
    *  be called <code>numVectorFields</code> times. Note that if term 
    *  vectors are enabled, this is called even if the document 
    *  has no vector fields, in this case <code>numVectorFields</code> 
@@ -69,17 +70,17 @@ public abstract class TermVectorsWriter 
   
   /** Called before writing the terms of the field.
    *  {@link #startTerm(BytesRef, int)} will be called <code>numTerms</code> times. */
-  public abstract void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException;
+  public abstract void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads) throws IOException;
   
   /** Adds a term and its term frequency <code>freq</code>.
    * If this field has positions and/or offsets enabled, then
-   * {@link #addPosition(int, int, int)} will be called 
+   * {@link #addPosition(int, int, int, BytesRef)} will be called 
    * <code>freq</code> times respectively.
    */
   public abstract void startTerm(BytesRef term, int freq) throws IOException;
   
   /** Adds a term position and offsets */
-  public abstract void addPosition(int position, int startOffset, int endOffset) throws IOException;
+  public abstract void addPosition(int position, int startOffset, int endOffset, BytesRef payload) throws IOException;
   
   /** Aborts writing entirely, implementation should remove
    *  any partially-written files, etc. */
@@ -99,7 +100,7 @@ public abstract class TermVectorsWriter 
    * This is an expert API that allows the codec to consume 
    * positions and offsets directly from the indexer.
    * <p>
-   * The default implementation calls {@link #addPosition(int, int, int)},
+   * The default implementation calls {@link #addPosition(int, int, int, BytesRef)},
    * but subclasses can override this if they want to efficiently write 
    * all the positions, then all the offsets, for example.
    * <p>
@@ -111,15 +112,36 @@ public abstract class TermVectorsWriter 
   public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
     int position = 0;
     int lastOffset = 0;
+    BytesRef payload = null;
 
     for (int i = 0; i < numProx; i++) {
       final int startOffset;
       final int endOffset;
+      final BytesRef thisPayload;
       
       if (positions == null) {
         position = -1;
+        thisPayload = null;
       } else {
-        position += positions.readVInt();
+        int code = positions.readVInt();
+        position += code >>> 1;
+        if ((code & 1) != 0) {
+          // This position has a payload
+          final int payloadLength = positions.readVInt();
+
+          if (payload == null) {
+            payload = new BytesRef();
+            payload.bytes = new byte[payloadLength];
+          } else if (payload.bytes.length < payloadLength) {
+            payload.grow(payloadLength);
+          }
+
+          positions.readBytes(payload.bytes, 0, payloadLength);
+          payload.length = payloadLength;
+          thisPayload = payload;
+        } else {
+          thisPayload = null;
+        }
       }
       
       if (offsets == null) {
@@ -129,24 +151,31 @@ public abstract class TermVectorsWriter 
         endOffset = startOffset + offsets.readVInt();
         lastOffset = endOffset;
       }
-      addPosition(position, startOffset, endOffset);
+      addPosition(position, startOffset, endOffset, thisPayload);
     }
   }
   
   /** Merges in the term vectors from the readers in 
    *  <code>mergeState</code>. The default implementation skips
    *  over deleted documents, and uses {@link #startDocument(int)},
-   *  {@link #startField(FieldInfo, int, boolean, boolean)}, 
-   *  {@link #startTerm(BytesRef, int)}, {@link #addPosition(int, int, int)},
+   *  {@link #startField(FieldInfo, int, boolean, boolean, boolean)}, 
+   *  {@link #startTerm(BytesRef, int)}, {@link #addPosition(int, int, int, BytesRef)},
    *  and {@link #finish(FieldInfos, int)},
    *  returning the number of documents that were written.
    *  Implementations can override this method for more sophisticated
    *  merging (bulk-byte copying, etc). */
   public int merge(MergeState mergeState) throws IOException {
     int docCount = 0;
-    for (AtomicReader reader : mergeState.readers) {
+    for (int i = 0; i < mergeState.readers.size(); i++) {
+      final AtomicReader reader = mergeState.readers.get(i);
       final int maxDoc = reader.maxDoc();
       final Bits liveDocs = reader.getLiveDocs();
+      // set PayloadProcessor
+      if (mergeState.payloadProcessorProvider != null) {
+        mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
+      } else {
+        mergeState.currentReaderPayloadProcessor = null;
+      }
       for (int docID = 0; docID < maxDoc; docID++) {
         if (liveDocs != null && !liveDocs.get(docID)) {
           // skip deleted docs
@@ -155,7 +184,7 @@ public abstract class TermVectorsWriter 
         // NOTE: it's very important to first assign to vectors then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         Fields vectors = reader.getTermVectors(docID);
-        addAllDocVectors(vectors, mergeState.fieldInfos);
+        addAllDocVectors(vectors, mergeState);
         docCount++;
         mergeState.checkAbort.work(300);
       }
@@ -169,7 +198,7 @@ public abstract class TermVectorsWriter 
    *  implementation requires that the vectors implement
    *  both Fields.size and
    *  Terms.size. */
-  protected final void addAllDocVectors(Fields vectors, FieldInfos fieldInfos) throws IOException {
+  protected final void addAllDocVectors(Fields vectors, MergeState mergeState) throws IOException {
     if (vectors == null) {
       startDocument(0);
       return;
@@ -181,94 +210,78 @@ public abstract class TermVectorsWriter 
     }
     startDocument(numFields);
     
-    final FieldsEnum fieldsEnum = vectors.iterator();
-    String fieldName;
     String lastFieldName = null;
+    
+    TermsEnum termsEnum = null;
+    DocsAndPositionsEnum docsAndPositionsEnum = null;
+    
+    final ReaderPayloadProcessor readerPayloadProcessor = mergeState.currentReaderPayloadProcessor;
+    PayloadProcessor payloadProcessor = null;
 
-    while((fieldName = fieldsEnum.next()) != null) {
-      final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldName);
+    for(String fieldName : vectors) {
+      final FieldInfo fieldInfo = mergeState.fieldInfos.fieldInfo(fieldName);
 
       assert lastFieldName == null || fieldName.compareTo(lastFieldName) > 0: "lastFieldName=" + lastFieldName + " fieldName=" + fieldName;
       lastFieldName = fieldName;
 
-      final Terms terms = fieldsEnum.terms();
+      final Terms terms = vectors.terms(fieldName);
       if (terms == null) {
         // FieldsEnum shouldn't lie...
         continue;
       }
+      
+      final boolean hasPositions = terms.hasPositions();
+      final boolean hasOffsets = terms.hasOffsets();
+      final boolean hasPayloads = terms.hasPayloads();
+      assert !hasPayloads || hasPositions;
+      
       final int numTerms = (int) terms.size();
       if (numTerms == -1) {
         throw new IllegalStateException("terms.size() must be implemented (it returned -1)");
       }
-      final TermsEnum termsEnum = terms.iterator(null);
-
-      DocsAndPositionsEnum docsAndPositionsEnum = null;
-
-      boolean startedField = false;
-
-      // NOTE: this is tricky, because TermVectors allow
-      // indexing offsets but NOT positions.  So we must
-      // lazily init the field by checking whether first
-      // position we see is -1 or not.
+      
+      startField(fieldInfo, numTerms, hasPositions, hasOffsets, hasPayloads);
+      termsEnum = terms.iterator(termsEnum);
 
       int termCount = 0;
       while(termsEnum.next() != null) {
         termCount++;
 
         final int freq = (int) termsEnum.totalTermFreq();
-
-        if (startedField) {
-          startTerm(termsEnum.term(), freq);
+        
+        startTerm(termsEnum.term(), freq);
+        
+        if (hasPayloads && readerPayloadProcessor != null) {
+          payloadProcessor = readerPayloadProcessor.getProcessor(fieldName, termsEnum.term());
         }
 
-        // TODO: we need a "query" API where we can ask (via
-        // flex API) what this term was indexed with...
-        // Both positions & offsets:
-        docsAndPositionsEnum = termsEnum.docsAndPositions(null, null, true);
-        final boolean hasOffsets;
-        boolean hasPositions = false;
-        if (docsAndPositionsEnum == null) {
-          // Fallback: no offsets
-          docsAndPositionsEnum = termsEnum.docsAndPositions(null, null, false);
-          hasOffsets = false;
-        } else {
-          hasOffsets = true;
-        }
-
-        if (docsAndPositionsEnum != null) {
+        if (hasPositions || hasOffsets) {
+          docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
+          assert docsAndPositionsEnum != null;
+          
           final int docID = docsAndPositionsEnum.nextDoc();
           assert docID != DocIdSetIterator.NO_MORE_DOCS;
           assert docsAndPositionsEnum.freq() == freq;
 
           for(int posUpto=0; posUpto<freq; posUpto++) {
             final int pos = docsAndPositionsEnum.nextPosition();
-            if (!startedField) {
-              assert numTerms > 0;
-              hasPositions = pos != -1;
-              startField(fieldInfo, numTerms, hasPositions, hasOffsets);
-              startTerm(termsEnum.term(), freq);
-              startedField = true;
-            }
-            final int startOffset;
-            final int endOffset;
-            if (hasOffsets) {
-              startOffset = docsAndPositionsEnum.startOffset();
-              endOffset = docsAndPositionsEnum.endOffset();
-              assert startOffset != -1;
-              assert endOffset != -1;
-            } else {
-              startOffset = -1;
-              endOffset = -1;
+            final int startOffset = docsAndPositionsEnum.startOffset();
+            final int endOffset = docsAndPositionsEnum.endOffset();
+            
+            BytesRef payload = docsAndPositionsEnum.getPayload();
+                
+            if (payloadProcessor != null && payload != null) {
+              // to not violate the D&P api, we must give the processor a private copy
+              payload = BytesRef.deepCopyOf(payload);
+              payloadProcessor.processPayload(payload);
+              if (payload.length == 0) {
+                // don't let PayloadProcessors corrumpt the index
+                payload = null;
+              }
             }
+
             assert !hasPositions || pos >= 0;
-            addPosition(pos, startOffset, endOffset);
-          }
-        } else {
-          if (!startedField) {
-            assert numTerms > 0;
-            startField(fieldInfo, numTerms, hasPositions, hasOffsets);
-            startTerm(termsEnum.term(), freq);
-            startedField = true;
+            addPosition(pos, startOffset, endOffset, payload);
           }
         }
       }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java Mon Aug 13 13:52:46 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.Comparator;
 
 import org.apache.lucene.index.FieldInfo; // javadocs
+import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.TermsEnum;
@@ -57,10 +58,14 @@ public abstract class TermsConsumer {
    *  no docs. */
   public abstract PostingsConsumer startTerm(BytesRef text) throws IOException;
 
-  /** Finishes the current term; numDocs must be > 0. */
+  /** Finishes the current term; numDocs must be > 0.
+   *  <code>stats.totalTermFreq</code> will be -1 when term 
+   *  frequencies are omitted for the field. */
   public abstract void finishTerm(BytesRef text, TermStats stats) throws IOException;
 
-  /** Called when we are done adding terms to this field */
+  /** Called when we are done adding terms to this field.
+   *  <code>sumTotalTermFreq</code> will be -1 when term 
+   *  frequencies are omitted for the field. */
   public abstract void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException;
 
   /** Return the BytesRef Comparator used to sort terms
@@ -93,7 +98,7 @@ public abstract class TermsConsumer {
       while((term = termsEnum.next()) != null) {
         // We can pass null for liveDocs, because the
         // mapping enum will skip the non-live docs:
-        docsEnumIn = (MultiDocsEnum) termsEnum.docs(null, docsEnumIn, false);
+        docsEnumIn = (MultiDocsEnum) termsEnum.docs(null, docsEnumIn, 0);
         if (docsEnumIn != null) {
           docsEnum.reset(docsEnumIn);
           final PostingsConsumer postingsConsumer = startTerm(term);
@@ -121,7 +126,7 @@ public abstract class TermsConsumer {
       while((term = termsEnum.next()) != null) {
         // We can pass null for liveDocs, because the
         // mapping enum will skip the non-live docs:
-        docsAndFreqsEnumIn = (MultiDocsEnum) termsEnum.docs(null, docsAndFreqsEnumIn, true);
+        docsAndFreqsEnumIn = (MultiDocsEnum) termsEnum.docs(null, docsAndFreqsEnumIn);
         assert docsAndFreqsEnumIn != null;
         docsAndFreqsEnum.reset(docsAndFreqsEnumIn);
         final PostingsConsumer postingsConsumer = startTerm(term);
@@ -146,7 +151,7 @@ public abstract class TermsConsumer {
       while((term = termsEnum.next()) != null) {
         // We can pass null for liveDocs, because the
         // mapping enum will skip the non-live docs:
-        postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, false);
+        postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, DocsAndPositionsEnum.FLAG_PAYLOADS);
         assert postingsEnumIn != null;
         postingsEnum.reset(postingsEnumIn);
         // set PayloadProcessor
@@ -180,7 +185,7 @@ public abstract class TermsConsumer {
       while((term = termsEnum.next()) != null) {
         // We can pass null for liveDocs, because the
         // mapping enum will skip the non-live docs:
-        postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn, true);
+        postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(null, postingsEnumIn);
         assert postingsEnumIn != null;
         postingsEnum.reset(postingsEnumIn);
         // set PayloadProcessor
@@ -205,6 +210,6 @@ public abstract class TermsConsumer {
         }
       }
     }
-    finish(sumTotalTermFreq, sumDocFreq, visitedDocs.cardinality());
+    finish(indexOptions == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq, sumDocFreq, visitedDocs.cardinality());
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java Mon Aug 13 13:52:46 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene4
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
@@ -218,7 +219,7 @@ public class Lucene40PostingsReader exte
   }
     
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     if (canReuse(reuse, liveDocs)) {
       // if (DEBUG) System.out.println("SPR.docs ts=" + termState);
       return ((SegmentDocsEnumBase) reuse).reset(fieldInfo, (StandardTermState)termState);
@@ -250,11 +251,14 @@ public class Lucene40PostingsReader exte
 
   @Override
   public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, boolean needsOffsets)
+                                               DocsAndPositionsEnum reuse, int flags)
     throws IOException {
 
     boolean hasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
 
+    // TODO: can we optimize if FLAG_PAYLOADS / FLAG_OFFSETS
+    // isn't passed?
+
     // TODO: refactor
     if (fieldInfo.hasPayloads() || hasOffsets) {
       SegmentFullPositionsEnum docsEnum;
@@ -348,13 +352,16 @@ public class Lucene40PostingsReader exte
 
       start = -1;
       count = 0;
+      freq = 1;
+      if (indexOmitsTF) {
+        Arrays.fill(freqs, 1);
+      }
       maxBufferedDocId = -1;
       return this;
     }
     
     @Override
     public final int freq() {
-      assert !indexOmitsTF;
       return freq;
     }
 
@@ -866,12 +873,7 @@ public class Lucene40PostingsReader exte
      *  payload was indexed. */
     @Override
     public BytesRef getPayload() throws IOException {
-      throw new IOException("No payloads exist for this field!");
-    }
-
-    @Override
-    public boolean hasPayload() {
-      return false;
+      return null;
     }
   }
   
@@ -1145,28 +1147,26 @@ public class Lucene40PostingsReader exte
     @Override
     public BytesRef getPayload() throws IOException {
       if (storePayloads) {
+        if (payloadLength <= 0) {
+          return null;
+        }
         assert lazyProxPointer == -1;
         assert posPendingCount < freq;
-        if (!payloadPending) {
-          throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
-        }
-        if (payloadLength > payload.bytes.length) {
-          payload.grow(payloadLength);
-        }
+        
+        if (payloadPending) {
+          if (payloadLength > payload.bytes.length) {
+            payload.grow(payloadLength);
+          }
 
-        proxIn.readBytes(payload.bytes, 0, payloadLength);
-        payload.length = payloadLength;
-        payloadPending = false;
+          proxIn.readBytes(payload.bytes, 0, payloadLength);
+          payload.length = payloadLength;
+          payloadPending = false;
+        }
 
         return payload;
       } else {
-        throw new IOException("No payloads exist for this field!");
+        return null;
       }
     }
-
-    @Override
-    public boolean hasPayload() {
-      return payloadPending && payloadLength > 0;
-    }
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsFormat.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsFormat.java Mon Aug 13 13:52:46 2012
@@ -67,33 +67,46 @@ import org.apache.lucene.store.IOContext
  * <li><a name="tvf" id="tvf"></a>
  * <p>The Field or .tvf file.</p>
  * <p>This file contains, for each field that has a term vector stored, a list of
- * the terms, their frequencies and, optionally, position and offset
+ * the terms, their frequencies and, optionally, position, offset, and payload
  * information.</p>
- * <p>Field (.tvf) --&gt; Header,&lt;NumTerms, Position/Offset, TermFreqs&gt;
+ * <p>Field (.tvf) --&gt; Header,&lt;NumTerms, Flags, TermFreqs&gt;
  * <sup>NumFields</sup></p>
  * <ul>
  *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *   <li>NumTerms --&gt; {@link DataOutput#writeVInt VInt}</li>
- *   <li>Position/Offset --&gt; {@link DataOutput#writeByte Byte}</li>
- *   <li>TermFreqs --&gt; &lt;TermText, TermFreq, Positions?, Offsets?&gt;
+ *   <li>Flags --&gt; {@link DataOutput#writeByte Byte}</li>
+ *   <li>TermFreqs --&gt; &lt;TermText, TermFreq, Positions?, PayloadData?, Offsets?&gt;
  *       <sup>NumTerms</sup></li>
  *   <li>TermText --&gt; &lt;PrefixLength, Suffix&gt;</li>
  *   <li>PrefixLength --&gt; {@link DataOutput#writeVInt VInt}</li>
  *   <li>Suffix --&gt; {@link DataOutput#writeString String}</li>
  *   <li>TermFreq --&gt; {@link DataOutput#writeVInt VInt}</li>
- *   <li>Positions --&gt; &lt;{@link DataOutput#writeVInt VInt}&gt;<sup>TermFreq</sup></li>
+ *   <li>Positions --&gt; &lt;PositionDelta PayloadLength?&gt;<sup>TermFreq</sup></li>
+ *   <li>PositionDelta --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *   <li>PayloadLength --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *   <li>PayloadData --&gt; {@link DataOutput#writeByte Byte}<sup>NumPayloadBytes</sup></li>
  *   <li>Offsets --&gt; &lt;{@link DataOutput#writeVInt VInt}, {@link DataOutput#writeVInt VInt}&gt;<sup>TermFreq</sup></li>
  * </ul>
  * <p>Notes:</p>
  * <ul>
- * <li>Position/Offset byte stores whether this term vector has position or offset
+ * <li>Flags byte stores whether this term vector has position, offset, payload.
  * information stored.</li>
  * <li>Term byte prefixes are shared. The PrefixLength is the number of initial
  * bytes from the previous term which must be pre-pended to a term's suffix
  * in order to form the term's bytes. Thus, if the previous term's text was "bone"
  * and the term is "boy", the PrefixLength is two and the suffix is "y".</li>
- * <li>Positions are stored as delta encoded VInts. This means we only store the
- * difference of the current position from the last position</li>
+ * <li>PositionDelta is, if payloads are disabled for the term's field, the
+ * difference between the position of the current occurrence in the document and
+ * the previous occurrence (or zero, if this is the first occurrence in this
+ * document). If payloads are enabled for the term's field, then PositionDelta/2
+ * is the difference between the current and the previous position. If payloads
+ * are enabled and PositionDelta is odd, then PayloadLength is stored, indicating
+ * the length of the payload at the current term position.</li>
+ * <li>PayloadData is metadata associated with a term position. If
+ * PayloadLength is stored at the current position, then it indicates the length
+ * of this payload. If PayloadLength is not stored, then this payload has the same
+ * length as the payload at the previous position. PayloadData encodes the 
+ * concatenated bytes for all of a terms occurrences.</li>
  * <li>Offsets are stored as delta encoded VInts. The first VInt is the
  * startOffset, the second is the endOffset.</li>
  * </ul>

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsReader.java Mon Aug 13 13:52:46 2012
@@ -21,7 +21,9 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.NoSuchElementException;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.TermVectorsReader;
@@ -30,7 +32,6 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.Terms;
@@ -55,6 +56,8 @@ public class Lucene40TermVectorsReader e
 
   static final byte STORE_OFFSET_WITH_TERMVECTOR = 0x2;
   
+  static final byte STORE_PAYLOAD_WITH_TERMVECTOR = 0x4;
+  
   /** Extension of vectors fields file */
   static final String VECTORS_FIELDS_EXTENSION = "tvf";
 
@@ -68,8 +71,10 @@ public class Lucene40TermVectorsReader e
   static final String CODEC_NAME_DOCS = "Lucene40TermVectorsDocs";
   static final String CODEC_NAME_INDEX = "Lucene40TermVectorsIndex";
 
-  static final int VERSION_START = 0;
-  static final int VERSION_CURRENT = VERSION_START;
+  static final int VERSION_NO_PAYLOADS = 0;
+  static final int VERSION_PAYLOADS = 1;
+  static final int VERSION_START = VERSION_NO_PAYLOADS;
+  static final int VERSION_CURRENT = VERSION_PAYLOADS;
   
   static final long HEADER_LENGTH_FIELDS = CodecUtil.headerLength(CODEC_NAME_FIELDS);
   static final long HEADER_LENGTH_DOCS = CodecUtil.headerLength(CODEC_NAME_DOCS);
@@ -245,9 +250,8 @@ public class Lucene40TermVectorsReader e
     }
     
     @Override
-    public FieldsEnum iterator() throws IOException {
-
-      return new FieldsEnum() {
+    public Iterator<String> iterator() {
+      return new Iterator<String>() {
         private int fieldUpto;
 
         @Override
@@ -255,13 +259,18 @@ public class Lucene40TermVectorsReader e
           if (fieldNumbers != null && fieldUpto < fieldNumbers.length) {
             return fieldInfos.fieldInfo(fieldNumbers[fieldUpto++]).name;
           } else {
-            return null;
+            throw new NoSuchElementException();
           }
         }
 
         @Override
-        public Terms terms() throws IOException {
-          return TVFields.this.terms(fieldInfos.fieldInfo(fieldNumbers[fieldUpto-1]).name);
+        public boolean hasNext() {
+          return fieldNumbers != null && fieldUpto < fieldNumbers.length;
+        }
+
+        @Override
+        public void remove() {
+          throw new UnsupportedOperationException();
         }
       };
     }
@@ -296,10 +305,17 @@ public class Lucene40TermVectorsReader e
   private class TVTerms extends Terms {
     private final int numTerms;
     private final long tvfFPStart;
+    private final boolean storePositions;
+    private final boolean storeOffsets;
+    private final boolean storePayloads;
 
     public TVTerms(long tvfFP) throws IOException {
       tvf.seek(tvfFP);
       numTerms = tvf.readVInt();
+      final byte bits = tvf.readByte();
+      storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
+      storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
+      storePayloads = (bits & STORE_PAYLOAD_WITH_TERMVECTOR) != 0;
       tvfFPStart = tvf.getFilePointer();
     }
 
@@ -314,7 +330,7 @@ public class Lucene40TermVectorsReader e
       } else {
         termsEnum = new TVTermsEnum();
       }
-      termsEnum.reset(numTerms, tvfFPStart);
+      termsEnum.reset(numTerms, tvfFPStart, storePositions, storeOffsets, storePayloads);
       return termsEnum;
     }
 
@@ -345,6 +361,21 @@ public class Lucene40TermVectorsReader e
       // this...?  I guess codec could buffer and re-sort...
       return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
+
+    @Override
+    public boolean hasOffsets() {
+      return storeOffsets;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return storePositions;
+    }
+    
+    @Override
+    public boolean hasPayloads() {
+      return storePayloads;
+    }
   }
 
   private class TVTermsEnum extends TermsEnum {
@@ -357,11 +388,17 @@ public class Lucene40TermVectorsReader e
     private BytesRef term = new BytesRef();
     private boolean storePositions;
     private boolean storeOffsets;
+    private boolean storePayloads;
     private long tvfFP;
 
     private int[] positions;
     private int[] startOffsets;
     private int[] endOffsets;
+    
+    // one shared byte[] for any term's payloads
+    private int[] payloadOffsets;
+    private int lastPayloadLength;
+    private byte[] payloadData;
 
     // NOTE: tvf is pre-positioned by caller
     public TVTermsEnum() {
@@ -373,17 +410,20 @@ public class Lucene40TermVectorsReader e
       return tvf == origTVF;
     }
 
-    public void reset(int numTerms, long tvfFPStart) throws IOException {
+    public void reset(int numTerms, long tvfFPStart, boolean storePositions, boolean storeOffsets, boolean storePayloads) throws IOException {
       this.numTerms = numTerms;
+      this.storePositions = storePositions;
+      this.storeOffsets = storeOffsets;
+      this.storePayloads = storePayloads;
       nextTerm = 0;
       tvf.seek(tvfFPStart);
-      final byte bits = tvf.readByte();
-      storePositions = (bits & STORE_POSITIONS_WITH_TERMVECTOR) != 0;
-      storeOffsets = (bits & STORE_OFFSET_WITH_TERMVECTOR) != 0;
       tvfFP = 1+tvfFPStart;
       positions = null;
       startOffsets = null;
       endOffsets = null;
+      payloadOffsets = null;
+      payloadData = null;
+      lastPayloadLength = -1;
     }
 
     // NOTE: slow!  (linear scan)
@@ -430,7 +470,26 @@ public class Lucene40TermVectorsReader e
       tvf.readBytes(term.bytes, start, deltaLen);
       freq = tvf.readVInt();
 
-      if (storePositions) {
+      if (storePayloads) {
+        positions = new int[freq];
+        payloadOffsets = new int[freq];
+        int totalPayloadLength = 0;
+        int pos = 0;
+        for(int posUpto=0;posUpto<freq;posUpto++) {
+          int code = tvf.readVInt();
+          pos += code >>> 1;
+          positions[posUpto] = pos;
+          if ((code & 1) != 0) {
+            // length change
+            lastPayloadLength = tvf.readVInt();
+          }
+          payloadOffsets[posUpto] = totalPayloadLength;
+          totalPayloadLength += lastPayloadLength;
+          assert totalPayloadLength >= 0;
+        }
+        payloadData = new byte[totalPayloadLength];
+        tvf.readBytes(payloadData, 0, payloadData.length);
+      } else if (storePositions /* no payloads */) {
         // TODO: we could maybe reuse last array, if we can
         // somehow be careful about consumer never using two
         // D&PEnums at once...
@@ -478,7 +537,7 @@ public class Lucene40TermVectorsReader e
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs /* ignored */) throws IOException {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags /* ignored */) throws IOException {
       TVDocsEnum docsEnum;
       if (reuse != null && reuse instanceof TVDocsEnum) {
         docsEnum = (TVDocsEnum) reuse;
@@ -490,10 +549,7 @@ public class Lucene40TermVectorsReader e
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
-      if (needsOffsets && !storeOffsets) {
-        return null;
-      }
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
 
       if (!storePositions && !storeOffsets) {
         return null;
@@ -505,14 +561,12 @@ public class Lucene40TermVectorsReader e
       } else {
         docsAndPositionsEnum = new TVDocsAndPositionsEnum();
       }
-      docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets);
+      docsAndPositionsEnum.reset(liveDocs, positions, startOffsets, endOffsets, payloadOffsets, payloadData);
       return docsAndPositionsEnum;
     }
 
     @Override
     public Comparator<BytesRef> getComparator() {
-      // TODO: really indexer hardwires
-      // this...?  I guess codec could buffer and re-sort...
       return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
   }
@@ -570,6 +624,9 @@ public class Lucene40TermVectorsReader e
     private int[] positions;
     private int[] startOffsets;
     private int[] endOffsets;
+    private int[] payloadOffsets;
+    private BytesRef payload = new BytesRef();
+    private byte[] payloadBytes;
 
     @Override
     public int freq() throws IOException {
@@ -605,11 +662,13 @@ public class Lucene40TermVectorsReader e
       }
     }
 
-    public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets) {
+    public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets, int[] payloadLengths, byte[] payloadBytes) {
       this.liveDocs = liveDocs;
       this.positions = positions;
       this.startOffsets = startOffsets;
       this.endOffsets = endOffsets;
+      this.payloadOffsets = payloadLengths;
+      this.payloadBytes = payloadBytes;
       this.doc = -1;
       didNext = false;
       nextPos = 0;
@@ -617,12 +676,19 @@ public class Lucene40TermVectorsReader e
 
     @Override
     public BytesRef getPayload() {
-      return null;
-    }
-
-    @Override
-    public boolean hasPayload() {
-      return false;
+      if (payloadOffsets == null) {
+        return null;
+      } else {
+        int off = payloadOffsets[nextPos-1];
+        int end = nextPos == payloadOffsets.length ? payloadBytes.length : payloadOffsets[nextPos];
+        if (end - off == 0) {
+          return null;
+        }
+        payload.bytes = payloadBytes;
+        payload.offset = off;
+        payload.length = end - off;
+        return payload;
+      }
     }
 
     @Override
@@ -640,14 +706,20 @@ public class Lucene40TermVectorsReader e
 
     @Override
     public int startOffset() {
-      assert startOffsets != null;
-      return startOffsets[nextPos-1];
+      if (startOffsets == null) {
+        return -1;
+      } else {
+        return startOffsets[nextPos-1];
+      }
     }
 
     @Override
     public int endOffset() {
-      assert endOffsets != null;
-      return endOffsets[nextPos-1];
+      if (endOffsets == null) {
+        return -1;
+      } else {
+        return endOffsets[nextPos-1];
+      }
     }
   }
 

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40TermVectorsWriter.java Mon Aug 13 13:52:46 2012
@@ -106,12 +106,14 @@ public final class Lucene40TermVectorsWr
   private String lastFieldName;
 
   @Override
-  public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets) throws IOException {
+  public void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads) throws IOException {
     assert lastFieldName == null || info.name.compareTo(lastFieldName) > 0: "fieldName=" + info.name + " lastFieldName=" + lastFieldName;
     lastFieldName = info.name;
     this.positions = positions;
     this.offsets = offsets;
+    this.payloads = payloads;
     lastTerm.length = 0;
+    lastPayloadLength = -1; // force first payload to write its length
     fps[fieldCount++] = tvf.getFilePointer();
     tvd.writeVInt(info.number);
     tvf.writeVInt(numTerms);
@@ -120,6 +122,8 @@ public final class Lucene40TermVectorsWr
       bits |= Lucene40TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
     if (offsets)
       bits |= Lucene40TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
+    if (payloads)
+      bits |= Lucene40TermVectorsReader.STORE_PAYLOAD_WITH_TERMVECTOR;
     tvf.writeByte(bits);
     
     assert fieldCount <= numVectorFields;
@@ -138,10 +142,12 @@ public final class Lucene40TermVectorsWr
   // we also don't buffer during bulk merges.
   private int offsetStartBuffer[] = new int[10];
   private int offsetEndBuffer[] = new int[10];
-  private int offsetIndex = 0;
-  private int offsetFreq = 0;
+  private BytesRef payloadData = new BytesRef(10);
+  private int bufferedIndex = 0;
+  private int bufferedFreq = 0;
   private boolean positions = false;
   private boolean offsets = false;
+  private boolean payloads = false;
 
   @Override
   public void startTerm(BytesRef term, int freq) throws IOException {
@@ -158,20 +164,40 @@ public final class Lucene40TermVectorsWr
       // we might need to buffer if its a non-bulk merge
       offsetStartBuffer = ArrayUtil.grow(offsetStartBuffer, freq);
       offsetEndBuffer = ArrayUtil.grow(offsetEndBuffer, freq);
-      offsetIndex = 0;
-      offsetFreq = freq;
     }
+    bufferedIndex = 0;
+    bufferedFreq = freq;
+    payloadData.length = 0;
   }
 
   int lastPosition = 0;
   int lastOffset = 0;
+  int lastPayloadLength = -1; // force first payload to write its length
+
+  BytesRef scratch = new BytesRef(); // used only by this optimized flush below
 
   @Override
   public void addProx(int numProx, DataInput positions, DataInput offsets) throws IOException {
-    // TODO: technically we could just copy bytes and not re-encode if we knew the length...
-    if (positions != null) {
+    if (payloads) {
+      // TODO, maybe overkill and just call super.addProx() in this case?
+      // we do avoid buffering the offsets in RAM though.
       for (int i = 0; i < numProx; i++) {
-        tvf.writeVInt(positions.readVInt());
+        int code = positions.readVInt();
+        if ((code & 1) == 1) {
+          int length = positions.readVInt();
+          scratch.grow(length);
+          scratch.length = length;
+          positions.readBytes(scratch.bytes, scratch.offset, scratch.length);
+          writePosition(code >>> 1, scratch);
+        } else {
+          writePosition(code >>> 1, null);
+        }
+      }
+      tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
+    } else if (positions != null) {
+      // pure positions, no payloads
+      for (int i = 0; i < numProx; i++) {
+        tvf.writeVInt(positions.readVInt() >>> 1);
       }
     }
     
@@ -184,28 +210,36 @@ public final class Lucene40TermVectorsWr
   }
 
   @Override
-  public void addPosition(int position, int startOffset, int endOffset) throws IOException {
-    if (positions && offsets) {
+  public void addPosition(int position, int startOffset, int endOffset, BytesRef payload) throws IOException {
+    if (positions && (offsets || payloads)) {
       // write position delta
-      tvf.writeVInt(position - lastPosition);
+      writePosition(position - lastPosition, payload);
       lastPosition = position;
       
       // buffer offsets
-      offsetStartBuffer[offsetIndex] = startOffset;
-      offsetEndBuffer[offsetIndex] = endOffset;
-      offsetIndex++;
+      if (offsets) {
+        offsetStartBuffer[bufferedIndex] = startOffset;
+        offsetEndBuffer[bufferedIndex] = endOffset;
+      }
+      
+      bufferedIndex++;
       
       // dump buffer if we are done
-      if (offsetIndex == offsetFreq) {
-        for (int i = 0; i < offsetIndex; i++) {
-          tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
-          tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
-          lastOffset = offsetEndBuffer[i];
+      if (bufferedIndex == bufferedFreq) {
+        if (payloads) {
+          tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
+        }
+        for (int i = 0; i < bufferedIndex; i++) {
+          if (offsets) {
+            tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
+            tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
+            lastOffset = offsetEndBuffer[i];
+          }
         }
       }
     } else if (positions) {
       // write position delta
-      tvf.writeVInt(position - lastPosition);
+      writePosition(position - lastPosition, payload);
       lastPosition = position;
     } else if (offsets) {
       // write offset deltas
@@ -214,6 +248,30 @@ public final class Lucene40TermVectorsWr
       lastOffset = endOffset;
     }
   }
+  
+  private void writePosition(int delta, BytesRef payload) throws IOException {
+    if (payloads) {
+      int payloadLength = payload == null ? 0 : payload.length;
+
+      if (payloadLength != lastPayloadLength) {
+        lastPayloadLength = payloadLength;
+        tvf.writeVInt((delta<<1)|1);
+        tvf.writeVInt(payloadLength);
+      } else {
+        tvf.writeVInt(delta << 1);
+      }
+      if (payloadLength > 0) {
+        if (payloadLength + payloadData.length < 0) {
+          // we overflowed the payload buffer, just throw UOE
+          // having > Integer.MAX_VALUE bytes of payload for a single term in a single doc is nuts.
+          throw new UnsupportedOperationException("A term cannot have more than Integer.MAX_VALUE bytes of payload data in a single document");
+        }
+        payloadData.append(payload);
+      }
+    } else {
+      tvf.writeVInt(delta);
+    }
+  }
 
   @Override
   public void abort() {
@@ -255,7 +313,14 @@ public final class Lucene40TermVectorsWr
 
     int idx = 0;
     int numDocs = 0;
-    for (final AtomicReader reader : mergeState.readers) {
+    for (int i = 0; i < mergeState.readers.size(); i++) {
+      final AtomicReader reader = mergeState.readers.get(i);
+      // set PayloadProcessor
+      if (mergeState.payloadProcessorProvider != null) {
+        mergeState.currentReaderPayloadProcessor = mergeState.readerPayloadProcessor[i];
+      } else {
+        mergeState.currentReaderPayloadProcessor = null;
+      }
       final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
       Lucene40TermVectorsReader matchingVectorsReader = null;
       if (matchingSegmentReader != null) {
@@ -288,8 +353,8 @@ public final class Lucene40TermVectorsWr
     final int maxDoc = reader.maxDoc();
     final Bits liveDocs = reader.getLiveDocs();
     int totalNumDocs = 0;
-    if (matchingVectorsReader != null) {
-      // We can bulk-copy because the fieldInfos are "congruent"
+    if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
+      // We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
       for (int docNum = 0; docNum < maxDoc;) {
         if (!liveDocs.get(docNum)) {
           // skip deleted docs
@@ -324,7 +389,7 @@ public final class Lucene40TermVectorsWr
         // NOTE: it's very important to first assign to vectors then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         Fields vectors = reader.getTermVectors(docNum);
-        addAllDocVectors(vectors, mergeState.fieldInfos);
+        addAllDocVectors(vectors, mergeState);
         totalNumDocs++;
         mergeState.checkAbort.work(300);
       }
@@ -339,8 +404,8 @@ public final class Lucene40TermVectorsWr
                                       int rawDocLengths2[])
           throws IOException {
     final int maxDoc = reader.maxDoc();
-    if (matchingVectorsReader != null) {
-      // We can bulk-copy because the fieldInfos are "congruent"
+    if (matchingVectorsReader != null && mergeState.currentReaderPayloadProcessor == null) {
+      // We can bulk-copy because the fieldInfos are "congruent" and there is no payload processor
       int docCount = 0;
       while (docCount < maxDoc) {
         int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
@@ -354,7 +419,7 @@ public final class Lucene40TermVectorsWr
         // NOTE: it's very important to first assign to vectors then pass it to
         // termVectorsWriter.addAllDocVectors; see LUCENE-1282
         Fields vectors = reader.getTermVectors(docNum);
-        addAllDocVectors(vectors, mergeState.fieldInfos);
+        addAllDocVectors(vectors, mergeState);
         mergeState.checkAbort.work(300);
       }
     }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/lucene40/package.html Mon Aug 13 13:52:46 2012
@@ -366,7 +366,7 @@ the {@link org.apache.lucene.codecs.Code
 factors need no longer be a single byte, they can be any DocValues 
 {@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
 strings, they can be any byte sequence. Term offsets can optionally be indexed 
-into the postings lists.</li>
+into the postings lists. Payloads can be stored in the term vectors.</li>
 </ul>
 <a name="Limitations" id="Limitations"></a>
 <h2>Limitations</h2>

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Mon Aug 13 13:52:46 2012
@@ -34,7 +34,6 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -49,6 +48,7 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.UnmodifiableIterator;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
@@ -344,6 +344,7 @@ public class MemoryPostingsFormat extend
       docID = -1;
       accum = 0;
       docUpto = 0;
+      freq = 1;
       payloadLen = 0;
       this.numDocs = numDocs;
       return this;
@@ -428,7 +429,6 @@ public class MemoryPostingsFormat extend
 
     @Override
     public int freq() {
-      assert indexOptions != IndexOptions.DOCS_ONLY;
       return freq;
     }
   }
@@ -446,7 +446,6 @@ public class MemoryPostingsFormat extend
     private int numDocs;
     private int posPending;
     private int payloadLength;
-    private boolean payloadRetrieved;
     final boolean storeOffsets;
     int offsetLength;
     int startOffset;
@@ -484,7 +483,6 @@ public class MemoryPostingsFormat extend
       payloadLength = 0;
       this.numDocs = numDocs;
       posPending = 0;
-      payloadRetrieved = false;
       startOffset = storeOffsets ? 0 : -1; // always return -1 if no offsets are stored
       offsetLength = 0;
       return this;
@@ -577,10 +575,6 @@ public class MemoryPostingsFormat extend
         payload.offset = in.getPosition();
         in.skipBytes(payloadLength);
         payload.length = payloadLength;
-        // Necessary, in case caller changed the
-        // payload.bytes from prior call:
-        payload.bytes = buffer;
-        payloadRetrieved = false;
       }
 
       //System.out.println("      pos=" + pos + " payload=" + payload + " fp=" + in.getPosition());
@@ -599,13 +593,7 @@ public class MemoryPostingsFormat extend
 
     @Override
     public BytesRef getPayload() {
-      payloadRetrieved = true;
-      return payload;
-    }
-
-    @Override
-    public boolean hasPayload() {
-      return !payloadRetrieved && payload.length > 0;
+      return payload.length > 0 ? payload : null;
     }
 
     @Override
@@ -696,13 +684,11 @@ public class MemoryPostingsFormat extend
     }
     
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
       decodeMetaData();
       FSTDocsEnum docsEnum;
 
-      if (needsFreqs && field.getIndexOptions() == IndexOptions.DOCS_ONLY) {
-        return null;
-      } else if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
+      if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
         docsEnum = new FSTDocsEnum(field.getIndexOptions(), field.hasPayloads());
       } else {
         docsEnum = (FSTDocsEnum) reuse;        
@@ -714,13 +700,9 @@ public class MemoryPostingsFormat extend
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
 
       boolean hasOffsets = field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-      if (needsOffsets && !hasOffsets) {
-        return null; // not available
-      }
-      
       if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
         return null;
       }
@@ -840,6 +822,21 @@ public class MemoryPostingsFormat extend
     public Comparator<BytesRef> getComparator() {
       return BytesRef.getUTF8SortedAsUnicodeComparator();
     }
+
+    @Override
+    public boolean hasOffsets() {
+      return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    }
+    
+    @Override
+    public boolean hasPayloads() {
+      return field.hasPayloads();
+    }
   }
 
   @Override
@@ -865,24 +862,8 @@ public class MemoryPostingsFormat extend
 
     return new FieldsProducer() {
       @Override
-      public FieldsEnum iterator() {
-        final Iterator<TermsReader> iter = fields.values().iterator();
-
-        return new FieldsEnum() {
-
-          private TermsReader current;
-
-          @Override
-          public String next() {
-            current = iter.next();
-            return current.field.name;
-          }
-
-          @Override
-          public Terms terms() {
-            return current;
-          }
-        };
+      public Iterator<String> iterator() {
+        return new UnmodifiableIterator<String>(fields.keySet().iterator());
       }
 
       @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/perfield/PerFieldPostingsFormat.java Mon Aug 13 13:52:46 2012
@@ -30,11 +30,11 @@ import org.apache.lucene.codecs.FieldsPr
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.UnmodifiableIterator;
 
 /**
  * Enables per field format support.
@@ -197,34 +197,9 @@ public abstract class PerFieldPostingsFo
       }
     }
 
-    private final class FieldsIterator extends FieldsEnum {
-      private final Iterator<String> it;
-      private String current;
-
-      public FieldsIterator() {
-        it = fields.keySet().iterator();
-      }
-
-      @Override
-      public String next() {
-        if (it.hasNext()) {
-          current = it.next();
-        } else {
-          current = null;
-        }
-
-        return current;
-      }
-
-      @Override
-      public Terms terms() throws IOException {
-        return fields.get(current).terms(current);
-      }
-    }
-
     @Override
-    public FieldsEnum iterator() throws IOException {
-      return new FieldsIterator();
+    public Iterator<String> iterator() {
+      return new UnmodifiableIterator<String>(fields.keySet().iterator());
     }
 
     @Override

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsReader.java Mon Aug 13 13:52:46 2012
@@ -178,7 +178,7 @@ public class PulsingPostingsReader exten
   }
 
   @Override
-  public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     PulsingTermState termState = (PulsingTermState) _termState;
     if (termState.postingsSize != -1) {
       PulsingDocsEnum postings;
@@ -202,18 +202,18 @@ public class PulsingPostingsReader exten
       return postings.reset(liveDocs, termState);
     } else {
       if (reuse instanceof PulsingDocsEnum) {
-        DocsEnum wrapped = wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, getOther(reuse), needsFreqs);
+        DocsEnum wrapped = wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, getOther(reuse), flags);
         setOther(wrapped, reuse); // wrapped.other = reuse
         return wrapped;
       } else {
-        return wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, reuse, needsFreqs);
+        return wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, reuse, flags);
       }
     }
   }
 
   @Override
   public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse,
-                                               boolean needsOffsets) throws IOException {
+                                               int flags) throws IOException {
 
     final PulsingTermState termState = (PulsingTermState) _termState;
 
@@ -240,11 +240,11 @@ public class PulsingPostingsReader exten
     } else {
       if (reuse instanceof PulsingDocsAndPositionsEnum) {
         DocsAndPositionsEnum wrapped = wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, (DocsAndPositionsEnum) getOther(reuse),
-                                                                              needsOffsets);
+                                                                              flags);
         setOther(wrapped, reuse); // wrapped.other = reuse
         return wrapped;
       } else {
-        return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, reuse, needsOffsets);
+        return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, reuse, flags);
       }
     }
   }
@@ -282,6 +282,7 @@ public class PulsingPostingsReader exten
       postings.reset(postingsBytes, 0, termState.postingsSize);
       docID = -1;
       accum = 0;
+      freq = 1;
       payloadLength = 0;
       this.liveDocs = liveDocs;
       return this;
@@ -349,7 +350,6 @@ public class PulsingPostingsReader exten
 
     @Override
     public int freq() throws IOException {
-      assert indexOptions != IndexOptions.DOCS_ONLY;
       return freq;
     }
 
@@ -533,18 +533,12 @@ public class PulsingPostingsReader exten
     }
 
     @Override
-    public boolean hasPayload() {
-      return storePayloads && !payloadRetrieved && payloadLength > 0;
-    }
-
-    @Override
     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;
-      if (payloadLength > 0) {
+        return payload;
+      } else if (storePayloads && payloadLength > 0) {
+        payloadRetrieved = true;
         if (payload == null) {
           payload = new BytesRef(payloadLength);
         } else {

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/sep/SepPostingsReader.java Mon Aug 13 13:52:46 2012
@@ -258,7 +258,7 @@ public class SepPostingsReader extends P
   }
 
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
     SepDocsEnum docsEnum;
     if (reuse == null || !(reuse instanceof SepDocsEnum)) {
@@ -278,13 +278,9 @@ public class SepPostingsReader extends P
 
   @Override
   public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, boolean needsOffsets)
+                                               DocsAndPositionsEnum reuse, int flags)
     throws IOException {
 
-    if (needsOffsets) {
-      return null;
-    }
-
     assert fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
     final SepTermState termState = (SepTermState) _termState;
     SepDocsAndPositionsEnum postingsEnum;
@@ -370,6 +366,7 @@ public class SepPostingsReader extends P
       count = 0;
       doc = -1;
       accum = 0;
+      freq = 1;
       skipped = false;
 
       return this;
@@ -403,7 +400,6 @@ public class SepPostingsReader extends P
 
     @Override
     public int freq() throws IOException {
-      assert !omitTF;
       return freq;
     }
 
@@ -718,7 +714,11 @@ public class SepPostingsReader extends P
     @Override
     public BytesRef getPayload() throws IOException {
       if (!payloadPending) {
-        throw new IOException("Either no payload exists at this term position or an attempt was made to load it more than once.");
+        return null;
+      }
+      
+      if (pendingPayloadBytes == 0) {
+        return payload;
       }
 
       assert pendingPayloadBytes >= payloadLength;
@@ -735,15 +735,9 @@ public class SepPostingsReader extends P
       }
 
       payloadIn.readBytes(payload.bytes, 0, payloadLength);
-      payloadPending = false;
       payload.length = payloadLength;
       pendingPayloadBytes = 0;
       return payload;
     }
-
-    @Override
-    public boolean hasPayload() {
-      return payloadPending && payloadLength > 0;
-    }
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Mon Aug 13 13:52:46 2012
@@ -20,14 +20,17 @@ package org.apache.lucene.codecs.simplet
 import java.io.IOException;
 import java.util.Comparator;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.Map;
+import java.util.TreeMap;
+import java.util.TreeSet;
 
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -40,6 +43,7 @@ import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.OpenBitSet;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.UnmodifiableIterator;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
@@ -48,7 +52,7 @@ import org.apache.lucene.util.fst.Positi
 import org.apache.lucene.util.fst.Util;
 
 class SimpleTextFieldsReader extends FieldsProducer {
-
+  private final TreeMap<String,Long> fields;
   private final IndexInput in;
   private final FieldInfos fieldInfos;
 
@@ -66,35 +70,22 @@ class SimpleTextFieldsReader extends Fie
     in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
    
     fieldInfos = state.fieldInfos;
+    fields = readFields((IndexInput)in.clone());
   }
-
-  private class SimpleTextFieldsEnum extends FieldsEnum {
-    private final IndexInput in;
-    private final BytesRef scratch = new BytesRef(10);
-    private String current;
-
-    public SimpleTextFieldsEnum() {
-      this.in = (IndexInput) SimpleTextFieldsReader.this.in.clone();
-    }
-
-    @Override
-    public String next() throws IOException {
-      while(true) {
-        SimpleTextUtil.readLine(in, scratch);
-        if (scratch.equals(END)) {
-          current = null;
-          return null;
-        }
-        if (StringHelper.startsWith(scratch, FIELD)) {
-          return current = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
-        }
+  
+  private TreeMap<String,Long> readFields(IndexInput in) throws IOException {
+    BytesRef scratch = new BytesRef(10);
+    TreeMap<String,Long> fields = new TreeMap<String,Long>();
+    
+    while (true) {
+      SimpleTextUtil.readLine(in, scratch);
+      if (scratch.equals(END)) {
+        return fields;
+      } else if (StringHelper.startsWith(scratch, FIELD)) {
+        String fieldName = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
+        fields.put(fieldName, in.getFilePointer());
       }
     }
-
-    @Override
-    public Terms terms() throws IOException {
-      return SimpleTextFieldsReader.this.terms(current);
-    }
   }
 
   private class SimpleTextTermsEnum extends TermsEnum {
@@ -194,33 +185,24 @@ class SimpleTextFieldsReader extends Fie
     }
  
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
-      if (needsFreqs && indexOptions == IndexOptions.DOCS_ONLY) {
-        return null;
-      }
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
       SimpleTextDocsEnum docsEnum;
       if (reuse != null && reuse instanceof SimpleTextDocsEnum && ((SimpleTextDocsEnum) reuse).canReuse(SimpleTextFieldsReader.this.in)) {
         docsEnum = (SimpleTextDocsEnum) reuse;
       } else {
         docsEnum = new SimpleTextDocsEnum();
       }
-      return docsEnum.reset(docsStart, liveDocs, !needsFreqs);
+      return docsEnum.reset(docsStart, liveDocs, indexOptions == IndexOptions.DOCS_ONLY);
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
 
       if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
         // Positions were not indexed
         return null;
       }
 
-      if (needsOffsets &&
-          indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
-        // Offsets were not indexed
-        return null;
-      }
-
       SimpleTextDocsAndPositionsEnum docsAndPositionsEnum;
       if (reuse != null && reuse instanceof SimpleTextDocsAndPositionsEnum && ((SimpleTextDocsAndPositionsEnum) reuse).canReuse(SimpleTextFieldsReader.this.in)) {
         docsAndPositionsEnum = (SimpleTextDocsAndPositionsEnum) reuse;
@@ -260,6 +242,7 @@ class SimpleTextFieldsReader extends Fie
       in.seek(fp);
       this.omitTF = omitTF;
       docID = -1;
+      tf = 1;
       return this;
     }
 
@@ -270,7 +253,6 @@ class SimpleTextFieldsReader extends Fie
 
     @Override
     public int freq() throws IOException {
-      assert !omitTF;
       return tf;
     }
 
@@ -343,8 +325,8 @@ class SimpleTextFieldsReader extends Fie
     private long nextDocStart;
     private boolean readOffsets;
     private boolean readPositions;
-    private int startOffset = -1;
-    private int endOffset = -1;
+    private int startOffset;
+    private int endOffset;
 
     public SimpleTextDocsAndPositionsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
@@ -361,6 +343,10 @@ class SimpleTextFieldsReader extends Fie
       docID = -1;
       readPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
       readOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+      if (!readOffsets) {
+        startOffset = -1;
+        endOffset = -1;
+      }
       return this;
     }
 
@@ -476,18 +462,7 @@ class SimpleTextFieldsReader extends Fie
 
     @Override
     public BytesRef getPayload() {
-      // Some tests rely on only being able to retrieve the
-      // payload once
-      try {
-        return payload;
-      } finally {
-        payload = null;
-      }
-    }
-
-    @Override
-    public boolean hasPayload() {
-      return payload != null;
+      return payload;
     }
   }
 
@@ -503,7 +478,7 @@ class SimpleTextFieldsReader extends Fie
 
   private class SimpleTextTerms extends Terms {
     private final long termsStart;
-    private final IndexOptions indexOptions;
+    private final FieldInfo fieldInfo;
     private long sumTotalTermFreq;
     private long sumDocFreq;
     private int docCount;
@@ -514,7 +489,7 @@ class SimpleTextFieldsReader extends Fie
 
     public SimpleTextTerms(String field, long termsStart) throws IOException {
       this.termsStart = termsStart;
-      indexOptions = fieldInfos.fieldInfo(field).getIndexOptions();
+      fieldInfo = fieldInfos.fieldInfo(field);
       loadTerms();
     }
 
@@ -584,7 +559,7 @@ class SimpleTextFieldsReader extends Fie
     @Override
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
       if (fst != null) {
-        return new SimpleTextTermsEnum(fst, indexOptions);
+        return new SimpleTextTermsEnum(fst, fieldInfo.getIndexOptions());
       } else {
         return TermsEnum.EMPTY;
       }
@@ -602,7 +577,7 @@ class SimpleTextFieldsReader extends Fie
 
     @Override
     public long getSumTotalTermFreq() {
-      return indexOptions == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq;
+      return fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : sumTotalTermFreq;
     }
 
     @Override
@@ -614,11 +589,26 @@ class SimpleTextFieldsReader extends Fie
     public int getDocCount() throws IOException {
       return docCount;
     }
+
+    @Override
+    public boolean hasOffsets() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    }
+    
+    @Override
+    public boolean hasPayloads() {
+      return fieldInfo.hasPayloads();
+    }
   }
 
   @Override
-  public FieldsEnum iterator() throws IOException {
-    return new SimpleTextFieldsEnum();
+  public Iterator<String> iterator() {
+    return new UnmodifiableIterator<String>(fields.keySet().iterator());
   }
 
   private final Map<String,Terms> termsCache = new HashMap<String,Terms>();
@@ -627,15 +617,13 @@ class SimpleTextFieldsReader extends Fie
   synchronized public Terms terms(String field) throws IOException {
     Terms terms = termsCache.get(field);
     if (terms == null) {
-      SimpleTextFieldsEnum fe = (SimpleTextFieldsEnum) iterator();
-      String fieldUpto;
-      while((fieldUpto = fe.next()) != null) {
-        if (fieldUpto.equals(field)) {
-          terms = new SimpleTextTerms(field, fe.in.getFilePointer());
-          break;
-        }
+      Long fp = fields.get(field);
+      if (fp == null) {
+        return null;
+      } else {
+        terms = new SimpleTextTerms(field, fp);
+        termsCache.put(field, terms);
       }
-      termsCache.put(field, terms);
     }
     return terms;
   }

Modified: lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1372423&r1=1372422&r2=1372423&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/LUCENE-2878/lucene/core/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Mon Aug 13 13:52:46 2012
@@ -29,7 +29,6 @@ import org.apache.lucene.codecs.TermVect
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.FieldsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.Terms;
@@ -45,6 +44,7 @@ import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.UnmodifiableIterator;
 
 import static org.apache.lucene.codecs.simpletext.SimpleTextTermVectorsWriter.*;
 
@@ -127,10 +127,14 @@ public class SimpleTextTermVectorsReader
       boolean offsets = Boolean.parseBoolean(readString(FIELDOFFSETS.length, scratch));
       
       readLine();
+      assert StringHelper.startsWith(scratch, FIELDPAYLOADS);
+      boolean payloads = Boolean.parseBoolean(readString(FIELDPAYLOADS.length, scratch));
+      
+      readLine();
       assert StringHelper.startsWith(scratch, FIELDTERMCOUNT);
       int termCount = parseIntAt(FIELDTERMCOUNT.length);
       
-      SimpleTVTerms terms = new SimpleTVTerms();
+      SimpleTVTerms terms = new SimpleTVTerms(offsets, positions, payloads);
       fields.put(fieldName, terms);
       
       for (int j = 0; j < termCount; j++) {
@@ -152,6 +156,9 @@ public class SimpleTextTermVectorsReader
         if (positions || offsets) {
           if (positions) {
             postings.positions = new int[postings.freq];
+            if (payloads) {
+              postings.payloads = new BytesRef[postings.freq];
+            }
           }
         
           if (offsets) {
@@ -164,6 +171,17 @@ public class SimpleTextTermVectorsReader
               readLine();
               assert StringHelper.startsWith(scratch, POSITION);
               postings.positions[k] = parseIntAt(POSITION.length);
+              if (payloads) {
+                readLine();
+                assert StringHelper.startsWith(scratch, PAYLOAD);
+                if (scratch.length - PAYLOAD.length == 0) {
+                  postings.payloads[k] = null;
+                } else {
+                  byte payloadBytes[] = new byte[scratch.length - PAYLOAD.length];
+                  System.arraycopy(scratch.bytes, scratch.offset+PAYLOAD.length, payloadBytes, 0, payloadBytes.length);
+                  postings.payloads[k] = new BytesRef(payloadBytes);
+                }
+              }
             }
             
             if (offsets) {
@@ -222,26 +240,8 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
-    public FieldsEnum iterator() throws IOException {
-      return new FieldsEnum() {
-        private Iterator<Map.Entry<String,SimpleTVTerms>> iterator = fields.entrySet().iterator();
-        private Map.Entry<String,SimpleTVTerms> current = null;
-        
-        @Override
-        public String next() {
-          if (!iterator.hasNext()) {
-            return null;
-          } else {
-            current = iterator.next();
-            return current.getKey();
-          }
-        }
-
-        @Override
-        public Terms terms() {
-          return current.getValue();
-        }
-      };
+    public Iterator<String> iterator() {
+      return new UnmodifiableIterator<String>(fields.keySet().iterator());
     }
 
     @Override
@@ -257,8 +257,14 @@ public class SimpleTextTermVectorsReader
   
   private static class SimpleTVTerms extends Terms {
     final SortedMap<BytesRef,SimpleTVPostings> terms;
+    final boolean hasOffsets;
+    final boolean hasPositions;
+    final boolean hasPayloads;
     
-    SimpleTVTerms() {
+    SimpleTVTerms(boolean hasOffsets, boolean hasPositions, boolean hasPayloads) {
+      this.hasOffsets = hasOffsets;
+      this.hasPositions = hasPositions;
+      this.hasPayloads = hasPayloads;
       terms = new TreeMap<BytesRef,SimpleTVPostings>();
     }
     
@@ -292,6 +298,21 @@ public class SimpleTextTermVectorsReader
     public int getDocCount() throws IOException {
       return 1;
     }
+
+    @Override
+    public boolean hasOffsets() {
+      return hasOffsets;
+    }
+
+    @Override
+    public boolean hasPositions() {
+      return hasPositions;
+    }
+    
+    @Override
+    public boolean hasPayloads() {
+      return hasPayloads;
+    }
   }
   
   private static class SimpleTVPostings {
@@ -299,6 +320,7 @@ public class SimpleTextTermVectorsReader
     private int positions[];
     private int startOffsets[];
     private int endOffsets[];
+    private BytesRef payloads[];
   }
   
   private static class SimpleTVTermsEnum extends TermsEnum {
@@ -357,25 +379,22 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, boolean needsFreqs) throws IOException {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
       // TODO: reuse
       SimpleTVDocsEnum e = new SimpleTVDocsEnum();
-      e.reset(liveDocs, needsFreqs ? current.getValue().freq : -1);
+      e.reset(liveDocs, (flags & DocsEnum.FLAG_FREQS) == 0 ? 1 : current.getValue().freq);
       return e;
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, boolean needsOffsets) throws IOException {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
       SimpleTVPostings postings = current.getValue();
       if (postings.positions == null && postings.startOffsets == null) {
         return null;
       }
-      if (needsOffsets && (postings.startOffsets == null || postings.endOffsets == null)) {
-        return null;
-      }
       // TODO: reuse
       SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum();
-      e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets);
+      e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
       return e;
     }
 
@@ -436,6 +455,7 @@ public class SimpleTextTermVectorsReader
     private int nextPos;
     private Bits liveDocs;
     private int[] positions;
+    private BytesRef[] payloads;
     private int[] startOffsets;
     private int[] endOffsets;
 
@@ -473,11 +493,12 @@ public class SimpleTextTermVectorsReader
       }
     }
 
-    public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets) {
+    public void reset(Bits liveDocs, int[] positions, int[] startOffsets, int[] endOffsets, BytesRef payloads[]) {
       this.liveDocs = liveDocs;
       this.positions = positions;
       this.startOffsets = startOffsets;
       this.endOffsets = endOffsets;
+      this.payloads = payloads;
       this.doc = -1;
       didNext = false;
       nextPos = 0;
@@ -485,12 +506,7 @@ public class SimpleTextTermVectorsReader
 
     @Override
     public BytesRef getPayload() {
-      return null;
-    }
-
-    @Override
-    public boolean hasPayload() {
-      return false;
+      return payloads == null ? null : payloads[nextPos-1];
     }
 
     @Override
@@ -507,12 +523,20 @@ public class SimpleTextTermVectorsReader
 
     @Override
     public int startOffset() {
-      return startOffsets[nextPos-1];
+      if (startOffsets == null) {
+        return -1;
+      } else {
+        return startOffsets[nextPos-1];
+      }
     }
 
     @Override
     public int endOffset() {
-      return endOffsets[nextPos-1];
+      if (endOffsets == null) {
+        return -1;
+      } else {
+        return endOffsets[nextPos-1];
+      }
     }
   }
 }