You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by mi...@apache.org on 2013/10/14 17:55:58 UTC

svn commit: r1531949 [1/3] - in /lucene/dev/trunk/lucene: ./ codecs/src/java/org/apache/lucene/codecs/blockterms/ codecs/src/java/org/apache/lucene/codecs/bloom/ codecs/src/java/org/apache/lucene/codecs/memory/ codecs/src/java/org/apache/lucene/codecs/...

Author: mikemccand
Date: Mon Oct 14 15:55:57 2013
New Revision: 1531949

URL: http://svn.apache.org/r1531949
Log:
LUCENE-5268: cutover all postings formats to FieldsConsumer

Added:
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java   (with props)
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsBaseFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsBaseFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java   (with props)
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java   (with props)
Removed:
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PushFieldsConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermsConsumer.java
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPulsing41PostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPulsing41PostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
    lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/FieldsConsumer.java
    lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
    lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Mon Oct 14 15:55:57 2013
@@ -49,6 +49,11 @@ New Features
   encode term metadata, and all dictionary implementations can now plug in any 
   PostingsBaseFormat. (Han Jiang, Mike McCandless)
 
+* LUCENE-5268: Full cutover of all postings formats to the "pull"
+  FieldsConsumer API, removing PushFieldsConsumer.  Added new
+  PushPostingsWriterBase for single-pass push of docs/positions to the
+  postings format.  (Mike McCandless)
+
 Optimizations
 
 * LUCENE-4848: Use Java 7 NIO2-FileChannel instead of RandomAccessFile

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java Mon Oct 14 15:55:57 2013
@@ -17,26 +17,29 @@ package org.apache.lucene.codecs.blockte
  * limitations under the License.
  */
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.PushFieldsConsumer;
 import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.BlockTermState;
-import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -52,7 +55,7 @@ import org.apache.lucene.util.RamUsageEs
  * @lucene.experimental
  */
 
-public class BlockTermsWriter extends PushFieldsConsumer {
+public class BlockTermsWriter extends FieldsConsumer implements Closeable {
 
   final static String CODEC_NAME = "BLOCK_TERMS_DICT";
 
@@ -70,6 +73,7 @@ public class BlockTermsWriter extends Pu
   final FieldInfos fieldInfos;
   FieldInfo currentField;
   private final TermsIndexWriterBase termsIndexWriter;
+  private final int maxDoc;
 
   private static class FieldMetaData {
     public final FieldInfo fieldInfo;
@@ -99,9 +103,9 @@ public class BlockTermsWriter extends Pu
   public BlockTermsWriter(TermsIndexWriterBase termsIndexWriter,
       SegmentWriteState state, PostingsWriterBase postingsWriter)
       throws IOException {
-    super(state);
     final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
     this.termsIndexWriter = termsIndexWriter;
+    maxDoc = state.segmentInfo.getDocCount();
     out = state.directory.createOutput(termsFileName, state.context);
     boolean success = false;
     try {
@@ -127,7 +131,43 @@ public class BlockTermsWriter extends Pu
   }
 
   @Override
-  public TermsConsumer addField(FieldInfo field) throws IOException {
+  public void write(Fields fields) throws IOException {
+
+    boolean success = false;
+    try {
+      for(String field : fields) {
+
+        Terms terms = fields.terms(field);
+        if (terms == null) {
+          continue;
+        }
+
+        TermsEnum termsEnum = terms.iterator(null);
+
+        TermsWriter termsWriter = addField(fieldInfos.fieldInfo(field));
+
+        while (true) {
+          BytesRef term = termsEnum.next();
+          if (term == null) {
+            break;
+          }
+
+          termsWriter.write(term, termsEnum);
+        }
+
+        termsWriter.finish();
+      }
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(this);
+      } else {
+        IOUtils.closeWhileHandlingException(this);
+      }
+    }
+  }
+
+  private TermsWriter addField(FieldInfo field) throws IOException {
     //System.out.println("\nBTW.addField seg=" + segment + " field=" + field.name);
     assert currentField == null || currentField.name.compareTo(field.name) < 0;
     currentField = field;
@@ -135,7 +175,6 @@ public class BlockTermsWriter extends Pu
     return new TermsWriter(fieldIndexWriter, field, postingsWriter);
   }
 
-  @Override
   public void close() throws IOException {
     try {
       final long dirStart = out.getFilePointer();
@@ -169,12 +208,13 @@ public class BlockTermsWriter extends Pu
     public BlockTermState state;
   }
 
-  class TermsWriter extends TermsConsumer {
+  class TermsWriter {
     private final FieldInfo fieldInfo;
     private final PostingsWriterBase postingsWriter;
     private final long termsStartPointer;
     private long numTerms;
     private final TermsIndexWriterBase.FieldWriter fieldIndexWriter;
+    private final FixedBitSet docsSeen;
     long sumTotalTermFreq;
     long sumDocFreq;
     int docCount;
@@ -191,6 +231,7 @@ public class BlockTermsWriter extends Pu
     {
       this.fieldInfo = fieldInfo;
       this.fieldIndexWriter = fieldIndexWriter;
+      this.docsSeen = new FixedBitSet(maxDoc);
       pendingTerms = new TermEntry[32];
       for(int i=0;i<pendingTerms.length;i++) {
         pendingTerms[i] = new TermEntry();
@@ -200,21 +241,22 @@ public class BlockTermsWriter extends Pu
       this.longsSize = postingsWriter.setField(fieldInfo);
     }
     
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      //System.out.println("BTW: startTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment);
-      postingsWriter.startTerm();
-      return postingsWriter;
-    }
-
     private final BytesRef lastPrevTerm = new BytesRef();
 
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+    void write(BytesRef text, TermsEnum termsEnum) throws IOException {
+
+      BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen);
+      if (state == null) {
+        // No docs for this term:
+        return;
+      }
+      sumDocFreq += state.docFreq;
+      sumTotalTermFreq += state.totalTermFreq;
 
-      assert stats.docFreq > 0;
+      assert state.docFreq > 0;
       //System.out.println("BTW: finishTerm term=" + fieldInfo.name + ":" + text.utf8ToString() + " " + text + " seg=" + segment + " df=" + stats.docFreq);
 
+      TermStats stats = new TermStats(state.docFreq, state.totalTermFreq);
       final boolean isIndexTerm = fieldIndexWriter.checkIndexTerm(text, stats);
 
       if (isIndexTerm) {
@@ -238,18 +280,14 @@ public class BlockTermsWriter extends Pu
       }
       final TermEntry te = pendingTerms[pendingCount];
       te.term.copyBytes(text);
-      te.state = postingsWriter.newTermState();
-      te.state.docFreq = stats.docFreq;
-      te.state.totalTermFreq = stats.totalTermFreq;
-      postingsWriter.finishTerm(te.state);
+      te.state = state;
 
       pendingCount++;
       numTerms++;
     }
 
     // Finishes all terms in this field
-    @Override
-    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
+    void finish() throws IOException {
       if (pendingCount > 0) {
         flushBlock();
       }
@@ -264,9 +302,9 @@ public class BlockTermsWriter extends Pu
         fields.add(new FieldMetaData(fieldInfo,
                                      numTerms,
                                      termsStartPointer,
-                                     sumTotalTermFreq,
+                                     fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0 ? sumTotalTermFreq : -1,
                                      sumDocFreq,
-                                     docCount,
+                                     docsSeen.cardinality(),
                                      longsSize));
       }
     }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/bloom/BloomFilteringPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -28,15 +28,12 @@ import java.util.Map;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsConsumer;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.PushFieldsConsumer;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.codecs.bloom.FuzzySet.ContainsResult;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -118,9 +115,7 @@ public final class BloomFilteringPosting
    * "blm" file. This PostingsFormat delegates to a choice of delegate
    * PostingsFormat for encoding all other postings data. This choice of
    * constructor defaults to the {@link DefaultBloomFilterFactory} for
-   * configuring per-field BloomFilters.  Note that the
-   * wrapped PostingsFormat must use a {@link PushFieldsConsumer}
-   * for writing.
+   * configuring per-field BloomFilters.
    * 
    * @param delegatePostingsFormat
    *          The PostingsFormat that records all the non-bloom filter data i.e.
@@ -144,11 +139,7 @@ public final class BloomFilteringPosting
           + " has been constructed without a choice of PostingsFormat");
     }
     FieldsConsumer fieldsConsumer = delegatePostingsFormat.fieldsConsumer(state);
-    if (!(fieldsConsumer instanceof PushFieldsConsumer)) {
-      throw new UnsupportedOperationException("Wrapped PostingsFormat must return a PushFieldsConsumer");
-    }
-    return new BloomFilteredFieldsConsumer(
-              (PushFieldsConsumer) fieldsConsumer, state);
+    return new BloomFilteredFieldsConsumer(fieldsConsumer, state);
   }
   
   @Override
@@ -315,7 +306,7 @@ public final class BloomFilteringPosting
         this.delegateTermsEnum = null;
       }
       
-      private final TermsEnum delegate() throws IOException {
+      private TermsEnum delegate() throws IOException {
         if (delegateTermsEnum == null) {
           /* pull the iterator only if we really need it -
            * this can be a relativly heavy operation depending on the 
@@ -327,12 +318,12 @@ public final class BloomFilteringPosting
       }
       
       @Override
-      public final BytesRef next() throws IOException {
+      public BytesRef next() throws IOException {
         return delegate().next();
       }
       
       @Override
-      public final boolean seekExact(BytesRef text)
+      public boolean seekExact(BytesRef text)
           throws IOException {
         // The magical fail-fast speed up that is the entire point of all of
         // this code - save a disk seek if there is a match on an in-memory
@@ -346,33 +337,33 @@ public final class BloomFilteringPosting
       }
       
       @Override
-      public final SeekStatus seekCeil(BytesRef text)
+      public SeekStatus seekCeil(BytesRef text)
           throws IOException {
         return delegate().seekCeil(text);
       }
       
       @Override
-      public final void seekExact(long ord) throws IOException {
+      public void seekExact(long ord) throws IOException {
         delegate().seekExact(ord);
       }
       
       @Override
-      public final BytesRef term() throws IOException {
+      public BytesRef term() throws IOException {
         return delegate().term();
       }
       
       @Override
-      public final long ord() throws IOException {
+      public long ord() throws IOException {
         return delegate().ord();
       }
       
       @Override
-      public final int docFreq() throws IOException {
+      public int docFreq() throws IOException {
         return delegate().docFreq();
       }
       
       @Override
-      public final long totalTermFreq() throws IOException {
+      public long totalTermFreq() throws IOException {
         return delegate().totalTermFreq();
       }
       
@@ -401,35 +392,60 @@ public final class BloomFilteringPosting
     }
   }
   
-  class BloomFilteredFieldsConsumer extends PushFieldsConsumer {
-    private PushFieldsConsumer delegateFieldsConsumer;
+  class BloomFilteredFieldsConsumer extends FieldsConsumer {
+    private FieldsConsumer delegateFieldsConsumer;
     private Map<FieldInfo,FuzzySet> bloomFilters = new HashMap<FieldInfo,FuzzySet>();
     private SegmentWriteState state;
     
-    public BloomFilteredFieldsConsumer(PushFieldsConsumer fieldsConsumer,
+    public BloomFilteredFieldsConsumer(FieldsConsumer fieldsConsumer,
         SegmentWriteState state) {
-      super(state);
       this.delegateFieldsConsumer = fieldsConsumer;
       this.state = state;
     }
-    
+
     @Override
-    public TermsConsumer addField(FieldInfo field) throws IOException {
-      FuzzySet bloomFilter = bloomFilterFactory.getSetForField(state,field);
-      if (bloomFilter != null) {
-        assert bloomFilters.containsKey(field) == false;
-        bloomFilters.put(field, bloomFilter);
-        return new WrappedTermsConsumer(delegateFieldsConsumer.addField(field), bloomFilter);
-      } else {
-        // No, use the unfiltered fieldsConsumer - we are not interested in
-        // recording any term Bitsets.
-        return delegateFieldsConsumer.addField(field);
+    public void write(Fields fields) throws IOException {
+      try {
+        for(String field : fields) {
+          Terms terms = fields.terms(field);
+          if (terms == null) {
+            continue;
+          }
+          FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+          TermsEnum termsEnum = terms.iterator(null);
+
+          FuzzySet bloomFilter = null;
+
+          DocsEnum docsEnum = null;
+          while (true) {
+            BytesRef term = termsEnum.next();
+            if (term == null) {
+              break;
+            }
+            if (bloomFilter == null) {
+              bloomFilter = bloomFilterFactory.getSetForField(state, fieldInfo);
+              if (bloomFilter == null) {
+                // Field not bloom'd
+                break;
+              }
+              assert bloomFilters.containsKey(field) == false;
+              bloomFilters.put(fieldInfo, bloomFilter);
+            }
+            // Make sure there's at least one doc for this term:
+            docsEnum = termsEnum.docs(null, docsEnum, 0);
+            if (docsEnum.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+              bloomFilter.addValue(term);
+            }
+          }
+        }
+      } finally {
+        close();
       }
+
+      delegateFieldsConsumer.write(fields);
     }
-    
-    @Override
+
     public void close() throws IOException {
-      delegateFieldsConsumer.close();
       // Now we are done accumulating values for these fields
       List<Entry<FieldInfo,FuzzySet>> nonSaturatedBlooms = new ArrayList<Map.Entry<FieldInfo,FuzzySet>>();
       
@@ -475,37 +491,5 @@ public final class BloomFilteringPosting
       }
       rightSizedSet.serialize(bloomOutput);
     }
-    
-  }
-  
-  class WrappedTermsConsumer extends TermsConsumer {
-    private TermsConsumer delegateTermsConsumer;
-    private FuzzySet bloomFilter;
-    
-    public WrappedTermsConsumer(TermsConsumer termsConsumer,FuzzySet bloomFilter) {
-      this.delegateTermsConsumer = termsConsumer;
-      this.bloomFilter = bloomFilter;
-    }
-    
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      return delegateTermsConsumer.startTerm(text);
-    }
-    
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
-      
-      // Record this term in our BloomFilter
-      if (stats.docFreq > 0) {
-        bloomFilter.addValue(text);
-      }
-      delegateTermsConsumer.finishTerm(text, stats);
-    }
-    
-    @Override
-    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount)
-        throws IOException {
-      delegateTermsConsumer.finish(sumTotalTermFreq, sumDocFreq, docCount);
-    }
   }
 }

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -25,9 +25,8 @@ import org.apache.lucene.codecs.FieldsPr
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.IOUtils;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPulsing41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPulsing41PostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPulsing41PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdPulsing41PostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -25,12 +25,9 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
 import org.apache.lucene.codecs.pulsing.PulsingPostingsReader;
+import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.IOUtils;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java Mon Oct 14 15:55:57 2013
@@ -23,20 +23,21 @@ import java.util.List;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.PushFieldsConsumer;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.fst.Builder;
@@ -142,7 +143,7 @@ import org.apache.lucene.util.fst.Util;
  * @lucene.experimental 
  */
 
-public class FSTOrdTermsWriter extends PushFieldsConsumer {
+public class FSTOrdTermsWriter extends FieldsConsumer {
   static final String TERMS_INDEX_EXTENSION = "tix";
   static final String TERMS_BLOCK_EXTENSION = "tbk";
   static final String TERMS_CODEC_NAME = "FST_ORD_TERMS_DICT";
@@ -152,17 +153,18 @@ public class FSTOrdTermsWriter extends P
   
   final PostingsWriterBase postingsWriter;
   final FieldInfos fieldInfos;
+  final int maxDoc;
   final List<FieldMetaData> fields = new ArrayList<FieldMetaData>();
   IndexOutput blockOut = null;
   IndexOutput indexOut = null;
 
   public FSTOrdTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException {
-    super(state);
     final String termsIndexFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_INDEX_EXTENSION);
     final String termsBlockFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_BLOCK_EXTENSION);
 
     this.postingsWriter = postingsWriter;
     this.fieldInfos = state.fieldInfos;
+    this.maxDoc = state.segmentInfo.getDocCount();
 
     boolean success = false;
     try {
@@ -180,11 +182,41 @@ public class FSTOrdTermsWriter extends P
   }
 
   @Override
-  public TermsConsumer addField(FieldInfo field) throws IOException {
-    return new TermsWriter(field);
+  public void write(Fields fields) throws IOException {
+    try {
+      for(String field : fields) {
+        Terms terms = fields.terms(field);
+        if (terms == null) {
+          continue;
+        }
+        FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+        boolean hasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+        TermsEnum termsEnum = terms.iterator(null);
+        TermsWriter termsWriter = new TermsWriter(fieldInfo);
+
+        long sumTotalTermFreq = 0;
+        long sumDocFreq = 0;
+        FixedBitSet docsSeen = new FixedBitSet(maxDoc);
+        while (true) {
+          BytesRef term = termsEnum.next();
+          if (term == null) {
+            break;
+          }
+          BlockTermState termState = postingsWriter.writeTerm(term, termsEnum, docsSeen);
+          if (termState != null) {
+            termsWriter.finishTerm(term, termState);
+            sumTotalTermFreq += termState.totalTermFreq;
+            sumDocFreq += termState.docFreq;
+          }
+        }
+
+        termsWriter.finish(hasFreq ? sumTotalTermFreq : -1, sumDocFreq, docsSeen.cardinality());
+      }
+    } finally {
+      close();
+    }
   }
 
-  @Override
   public void close() throws IOException {
     IOException ioe = null;
     try {
@@ -247,7 +279,7 @@ public class FSTOrdTermsWriter extends P
     public RAMOutputStream metaBytesOut;
   }
 
-  final class TermsWriter extends TermsConsumer {
+  final class TermsWriter {
     private final Builder<Long> builder;
     private final PositiveIntOutputs outputs;
     private final FieldInfo fieldInfo;
@@ -284,34 +316,23 @@ public class FSTOrdTermsWriter extends P
       this.lastMetaBytesFP = 0;
     }
 
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      postingsWriter.startTerm();
-      return postingsWriter;
-    }
-
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+    public void finishTerm(BytesRef text, BlockTermState state) throws IOException {
       if (numTerms > 0 && numTerms % SKIP_INTERVAL == 0) {
         bufferSkip();
       }
       // write term meta data into fst
       final long longs[] = new long[longsSize];
-      final long delta = stats.totalTermFreq - stats.docFreq;
-      if (stats.totalTermFreq > 0) {
+      final long delta = state.totalTermFreq - state.docFreq;
+      if (state.totalTermFreq > 0) {
         if (delta == 0) {
-          statsOut.writeVInt(stats.docFreq<<1|1);
+          statsOut.writeVInt(state.docFreq<<1|1);
         } else {
-          statsOut.writeVInt(stats.docFreq<<1|0);
-          statsOut.writeVLong(stats.totalTermFreq-stats.docFreq);
+          statsOut.writeVInt(state.docFreq<<1|0);
+          statsOut.writeVLong(state.totalTermFreq-state.docFreq);
         }
       } else {
-        statsOut.writeVInt(stats.docFreq);
+        statsOut.writeVInt(state.docFreq);
       }
-      BlockTermState state = postingsWriter.newTermState();
-      state.docFreq = stats.docFreq;
-      state.totalTermFreq = stats.totalTermFreq;
-      postingsWriter.finishTerm(state);
       postingsWriter.encodeTerm(longs, metaBytesOut, fieldInfo, state, true);
       for (int i = 0; i < longsSize; i++) {
         metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
@@ -325,7 +346,6 @@ public class FSTOrdTermsWriter extends P
       lastMetaBytesFP = metaBytesOut.getFilePointer();
     }
 
-    @Override
     public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
       if (numTerms > 0) {
         final FieldMetaData metadata = new FieldMetaData();

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -25,9 +25,8 @@ import org.apache.lucene.codecs.FieldsPr
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.IOUtils;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPulsing41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPulsing41PostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPulsing41PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTPulsing41PostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -25,12 +25,9 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
-import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
-import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
 import org.apache.lucene.codecs.pulsing.PulsingPostingsReader;
+import org.apache.lucene.codecs.pulsing.PulsingPostingsWriter;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.IOUtils;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermOutputs.java Mon Oct 14 15:55:57 2013
@@ -25,7 +25,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.fst.Outputs;
-import org.apache.lucene.util.LongsRef;
 
 /**
  * An FST {@link Outputs} implementation for 
@@ -89,6 +88,11 @@ class FSTTermOutputs extends Outputs<FST
     }
 
     @Override
+    public String toString() {
+      return "FSTTermOutputs$TermData longs=" + Arrays.toString(longs) + " bytes=" + Arrays.toString(bytes) + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq;
+    }
+
+    @Override
     public boolean equals(Object other_) {
       if (other_ == this) {
         return true;
@@ -221,6 +225,7 @@ class FSTTermOutputs extends Outputs<FST
 
   @Override
   public void write(TermData data, DataOutput out) throws IOException {
+    assert hasPos || data.totalTermFreq == -1;
     int bit0 = allZero(data.longs) ? 0 : 1;
     int bit1 = ((data.bytes == null || data.bytes.length == 0) ? 0 : 1) << 1;
     int bit2 = ((data.docFreq == 0)  ? 0 : 1) << 2;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java Mon Oct 14 15:55:57 2013
@@ -23,20 +23,21 @@ import java.util.List;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PostingsConsumer;
+import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.codecs.PushFieldsConsumer;
-import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.fst.Builder;
@@ -119,7 +120,7 @@ import org.apache.lucene.util.fst.Util;
  * @lucene.experimental
  */
 
-public class FSTTermsWriter extends PushFieldsConsumer {
+public class FSTTermsWriter extends FieldsConsumer {
   static final String TERMS_EXTENSION = "tmp";
   static final String TERMS_CODEC_NAME = "FST_TERMS_DICT";
   public static final int TERMS_VERSION_START = 0;
@@ -128,15 +129,16 @@ public class FSTTermsWriter extends Push
   final PostingsWriterBase postingsWriter;
   final FieldInfos fieldInfos;
   final IndexOutput out;
+  final int maxDoc;
   final List<FieldMetaData> fields = new ArrayList<FieldMetaData>();
 
   public FSTTermsWriter(SegmentWriteState state, PostingsWriterBase postingsWriter) throws IOException {
-    super(state);
     final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
 
     this.postingsWriter = postingsWriter;
     this.fieldInfos = state.fieldInfos;
     this.out = state.directory.createOutput(termsFileName, state.context);
+    this.maxDoc = state.segmentInfo.getDocCount();
 
     boolean success = false;
     try {
@@ -149,19 +151,53 @@ public class FSTTermsWriter extends Push
       }
     }
   }
+
   private void writeHeader(IndexOutput out) throws IOException {
     CodecUtil.writeHeader(out, TERMS_CODEC_NAME, TERMS_VERSION_CURRENT);   
   }
+
   private void writeTrailer(IndexOutput out, long dirStart) throws IOException {
     out.writeLong(dirStart);
   }
 
   @Override
-  public TermsConsumer addField(FieldInfo field) throws IOException {
-    return new TermsWriter(field);
+  public void write(Fields fields) throws IOException {
+    try {
+      for(String field : fields) {
+        Terms terms = fields.terms(field);
+        if (terms == null) {
+          continue;
+        }
+        FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
+        boolean hasFreq = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+        TermsEnum termsEnum = terms.iterator(null);
+        TermsWriter termsWriter = termsWriter = new TermsWriter(fieldInfo);
+
+        long sumTotalTermFreq = 0;
+        long sumDocFreq = 0;
+        FixedBitSet docsSeen = new FixedBitSet(maxDoc);
+
+        while (true) {
+          BytesRef term = termsEnum.next();
+          if (term == null) {
+            break;
+          }
+            
+          BlockTermState termState = postingsWriter.writeTerm(term, termsEnum, docsSeen);
+          if (termState != null) {
+            termsWriter.finishTerm(term, termState);
+            sumTotalTermFreq += termState.totalTermFreq;
+            sumDocFreq += termState.docFreq;
+          }
+        }
+
+        termsWriter.finish(hasFreq ? sumTotalTermFreq : -1, sumDocFreq, docsSeen.cardinality());
+      }
+    } finally {
+      close();
+    }
   }
 
-  @Override
   public void close() throws IOException {
     IOException ioe = null;
     try {
@@ -208,7 +244,7 @@ public class FSTTermsWriter extends Push
     }
   }
 
-  final class TermsWriter extends TermsConsumer {
+  final class TermsWriter {
     private final Builder<FSTTermOutputs.TermData> builder;
     private final FSTTermOutputs outputs;
     private final FieldInfo fieldInfo;
@@ -226,22 +262,13 @@ public class FSTTermsWriter extends Push
       this.builder = new Builder<FSTTermOutputs.TermData>(FST.INPUT_TYPE.BYTE1, outputs);
     }
 
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      postingsWriter.startTerm();
-      return postingsWriter;
-    }
-
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+    public void finishTerm(BytesRef text, BlockTermState state) throws IOException {
       // write term meta data into fst
-      final BlockTermState state = postingsWriter.newTermState();
       final FSTTermOutputs.TermData meta = new FSTTermOutputs.TermData();
       meta.longs = new long[longsSize];
       meta.bytes = null;
-      meta.docFreq = state.docFreq = stats.docFreq;
-      meta.totalTermFreq = state.totalTermFreq = stats.totalTermFreq;
-      postingsWriter.finishTerm(state);
+      meta.docFreq = state.docFreq;
+      meta.totalTermFreq = state.totalTermFreq;
       postingsWriter.encodeTerm(meta.longs, metaWriter, fieldInfo, state, true);
       final int bytesSize = (int)metaWriter.getFilePointer();
       if (bytesSize > 0) {
@@ -253,7 +280,6 @@ public class FSTTermsWriter extends Push
       numTerms++;
     }
 
-    @Override
     public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
       // save FST dict
       if (numTerms > 0) {

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/memory/MemoryPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -17,6 +17,7 @@ package org.apache.lucene.codecs.memory;
  * limitations under the License.
  */
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collections;
 import java.util.Iterator;
@@ -26,16 +27,14 @@ import java.util.TreeMap;
 
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.codecs.PostingsConsumer;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.codecs.PushFieldsConsumer;
 import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.codecs.TermsConsumer;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 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.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -49,6 +48,8 @@ import org.apache.lucene.store.RAMOutput
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.fst.Builder;
@@ -104,7 +105,7 @@ public final class MemoryPostingsFormat 
     return "PostingsFormat(name=" + getName() + " doPackFST= " + doPackFST + ")";
   }
 
-  private final static class TermsWriter extends TermsConsumer {
+  private final static class TermsWriter {
     private final IndexOutput out;
     private final FieldInfo field;
     private final Builder<BytesRef> builder;
@@ -121,7 +122,7 @@ public final class MemoryPostingsFormat 
       builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio, true, 15);
     }
 
-    private class PostingsWriter extends PostingsConsumer {
+    private class PostingsWriter {
       private int lastDocID;
       private int lastPos;
       private int lastPayloadLen;
@@ -133,7 +134,6 @@ public final class MemoryPostingsFormat 
       int lastOffsetLength;
       int lastOffset;
 
-      @Override
       public void startDoc(int docID, int termDocFreq) throws IOException {
         //System.out.println("    startDoc docID=" + docID + " freq=" + termDocFreq);
         final int delta = docID - lastDocID;
@@ -155,7 +155,6 @@ public final class MemoryPostingsFormat 
         lastOffset = 0;
       }
 
-      @Override
       public void addPosition(int pos, BytesRef payload, int startOffset, int endOffset) throws IOException {
         assert payload == null || field.hasPayloads();
 
@@ -200,10 +199,6 @@ public final class MemoryPostingsFormat 
         }
       }
 
-      @Override
-      public void finishDoc() {
-      }
-
       public PostingsWriter reset() {
         assert buffer.getFilePointer() == 0;
         lastDocID = 0;
@@ -215,13 +210,7 @@ public final class MemoryPostingsFormat 
       }
     }
 
-    private final PostingsWriter postingsWriter = new PostingsWriter();
-
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) {
-      //System.out.println("  startTerm term=" + text.utf8ToString());
-      return postingsWriter.reset();
-    }
+    final PostingsWriter postingsWriter = new PostingsWriter();
 
     private final RAMOutputStream buffer2 = new RAMOutputStream();
     private final BytesRef spare = new BytesRef();
@@ -229,9 +218,11 @@ public final class MemoryPostingsFormat 
 
     private final IntsRef scratchIntsRef = new IntsRef();
 
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
+    private void finishTerm(BytesRef text, TermStats stats) throws IOException {
 
+      if (stats.docFreq == 0) {
+        return;
+      }
       assert postingsWriter.docCount == stats.docFreq;
 
       assert buffer2.getFilePointer() == 0;
@@ -263,7 +254,6 @@ public final class MemoryPostingsFormat 
       termCount++;
     }
 
-    @Override
     public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
       if (termCount > 0) {
         out.writeVInt(termCount);
@@ -282,29 +272,146 @@ public final class MemoryPostingsFormat 
 
   private static String EXTENSION = "ram";
 
-  @Override
-  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+  private class MemoryFieldsConsumer extends FieldsConsumer implements Closeable {
+    private final SegmentWriteState state;
+    private final IndexOutput out;
 
-    final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
-    final IndexOutput out = state.directory.createOutput(fileName, state.context);
-    
-    return new PushFieldsConsumer(state) {
-      @Override
-      public TermsConsumer addField(FieldInfo field) {
-        //System.out.println("\naddField field=" + field.name);
-        return new TermsWriter(out, field, doPackFST, acceptableOverheadRatio);
-      }
+    private MemoryFieldsConsumer(SegmentWriteState state) throws IOException {
+      final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
+      out = state.directory.createOutput(fileName, state.context);
+      this.state = state;
+    }
 
-      @Override
-      public void close() throws IOException {
-        // EOF marker:
-        try {
-          out.writeVInt(0);
-        } finally {
-          out.close();
+    @Override
+    public void write(Fields fields) throws IOException {
+      boolean success = false;
+      try {
+        for(String field : fields) {
+
+          Terms terms = fields.terms(field);
+          if (terms == null) {
+            continue;
+          }
+
+          TermsEnum termsEnum = terms.iterator(null);
+
+          FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+          TermsWriter termsWriter = new TermsWriter(out, fieldInfo,
+                                                    doPackFST, acceptableOverheadRatio);
+
+          FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.getDocCount());
+          long sumTotalTermFreq = 0;
+          long sumDocFreq = 0;
+          DocsEnum docsEnum = null;
+          DocsAndPositionsEnum posEnum = null;
+          int enumFlags;
+
+          IndexOptions indexOptions = fieldInfo.getIndexOptions();
+          boolean writeFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+          boolean writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+          boolean writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;        
+          boolean writePayloads = fieldInfo.hasPayloads();
+
+          if (writeFreqs == false) {
+            enumFlags = 0;
+          } else if (writePositions == false) {
+            enumFlags = DocsEnum.FLAG_FREQS;
+          } else if (writeOffsets == false) {
+            if (writePayloads) {
+              enumFlags = DocsAndPositionsEnum.FLAG_PAYLOADS;
+            } else {
+              enumFlags = 0;
+            }
+          } else {
+            if (writePayloads) {
+              enumFlags = DocsAndPositionsEnum.FLAG_PAYLOADS | DocsAndPositionsEnum.FLAG_OFFSETS;
+            } else {
+              enumFlags = DocsAndPositionsEnum.FLAG_OFFSETS;
+            }
+          }
+
+          while (true) {
+            BytesRef term = termsEnum.next();
+            if (term == null) {
+              break;
+            }
+            termsWriter.postingsWriter.reset();
+
+            if (writePositions) {
+              posEnum = termsEnum.docsAndPositions(null, posEnum, enumFlags);
+              docsEnum = posEnum;
+            } else {
+              docsEnum = termsEnum.docs(null, docsEnum, enumFlags);
+              posEnum = null;
+            }
+
+            int docFreq = 0;
+            long totalTermFreq = 0;
+            while (true) {
+              int docID = docsEnum.nextDoc();
+              if (docID == DocsEnum.NO_MORE_DOCS) {
+                break;
+              }
+              docsSeen.set(docID);
+              docFreq++;
+
+              int freq;
+              if (writeFreqs) {
+                freq = docsEnum.freq();
+                totalTermFreq += freq;
+              } else {
+                freq = -1;
+              }
+
+              termsWriter.postingsWriter.startDoc(docID, freq);
+              if (writePositions) {
+                for (int i=0;i<freq;i++) {
+                  int pos = posEnum.nextPosition();
+                  BytesRef payload = writePayloads ? posEnum.getPayload() : null;
+                  int startOffset;
+                  int endOffset;
+                  if (writeOffsets) {
+                    startOffset = posEnum.startOffset();
+                    endOffset = posEnum.endOffset();
+                  } else {
+                    startOffset = -1;
+                    endOffset = -1;
+                  }
+                  termsWriter.postingsWriter.addPosition(pos, payload, startOffset, endOffset);
+                }
+              }
+            }
+            termsWriter.finishTerm(term, new TermStats(docFreq, totalTermFreq));
+            sumDocFreq += docFreq;
+            sumTotalTermFreq += totalTermFreq;
+          }
+
+          termsWriter.finish(sumTotalTermFreq, sumDocFreq, docsSeen.cardinality());
+        }
+        success = true;
+      } finally {
+        if (success) {
+          IOUtils.close(this);
+        } else {
+          IOUtils.closeWhileHandlingException(this);
         }
       }
-    };
+    }
+
+    @Override
+    public void close() throws IOException {
+      // EOF marker:
+      try {
+        out.writeVInt(0);
+      } finally {
+        out.close();
+      }
+    }
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    return new MemoryFieldsConsumer(state);
   }
 
   private final static class FSTDocsEnum extends DocsEnum {

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java?rev=1531949&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/Pulsing41PostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -0,0 +1,45 @@
+package org.apache.lucene.codecs.pulsing;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsBaseFormat;
+import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat; // javadocs
+
+/**
+ * Concrete pulsing implementation over {@link Lucene41PostingsFormat}.
+ * 
+ * @lucene.experimental
+ */
+public class Pulsing41PostingsFormat extends PulsingPostingsFormat {
+
+  /** Inlines docFreq=1 terms, otherwise uses the normal "Lucene41" format. */
+  public Pulsing41PostingsFormat() {
+    this(1);
+  }
+
+  /** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
+  public Pulsing41PostingsFormat(int freqCutoff) {
+    this(freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  }
+
+  /** Inlines docFreq=<code>freqCutoff</code> terms, otherwise uses the normal "Lucene41" format. */
+  public Pulsing41PostingsFormat(int freqCutoff, int minBlockSize, int maxBlockSize) {
+    super("Pulsing41", new Lucene41PostingsBaseFormat(), freqCutoff, minBlockSize, maxBlockSize);
+  }
+}

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java?rev=1531949&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -0,0 +1,119 @@
+package org.apache.lucene.codecs.pulsing;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.codecs.BlockTreeTermsReader;
+import org.apache.lucene.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.PostingsBaseFormat;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.IOUtils;
+
+/** This postings format "inlines" the postings for terms that have
+ *  low docFreq.  It wraps another postings format, which is used for
+ *  writing the non-inlined terms.
+ *
+ *  @lucene.experimental */
+
+public abstract class PulsingPostingsFormat extends PostingsFormat {
+
+  private final int freqCutoff;
+  private final int minBlockSize;
+  private final int maxBlockSize;
+  private final PostingsBaseFormat wrappedPostingsBaseFormat;
+  
+  public PulsingPostingsFormat(String name, PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff) {
+    this(name, wrappedPostingsBaseFormat, freqCutoff, BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  }
+
+  /** Terms with freq <= freqCutoff are inlined into terms
+   *  dict. */
+  public PulsingPostingsFormat(String name, PostingsBaseFormat wrappedPostingsBaseFormat, int freqCutoff, int minBlockSize, int maxBlockSize) {
+    super(name);
+    this.freqCutoff = freqCutoff;
+    this.minBlockSize = minBlockSize;
+    assert minBlockSize > 1;
+    this.maxBlockSize = maxBlockSize;
+    this.wrappedPostingsBaseFormat = wrappedPostingsBaseFormat;
+  }
+
+  @Override
+  public String toString() {
+    return getName() + "(freqCutoff=" + freqCutoff + " minBlockSize=" + minBlockSize + " maxBlockSize=" + maxBlockSize + ")";
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    PostingsWriterBase docsWriter = null;
+
+    // Terms that have <= freqCutoff number of docs are
+    // "pulsed" (inlined):
+    PostingsWriterBase pulsingWriter = null;
+
+    // Terms dict
+    boolean success = false;
+    try {
+      docsWriter = wrappedPostingsBaseFormat.postingsWriterBase(state);
+
+      // Terms that have <= freqCutoff number of docs are
+      // "pulsed" (inlined):
+      pulsingWriter = new PulsingPostingsWriter(state, freqCutoff, docsWriter);
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, pulsingWriter, minBlockSize, maxBlockSize);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docsWriter, pulsingWriter);
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase docsReader = null;
+    PostingsReaderBase pulsingReader = null;
+
+    boolean success = false;
+    try {
+      docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
+      pulsingReader = new PulsingPostingsReader(state, docsReader);
+      FieldsProducer ret = new BlockTreeTermsReader(
+                                                    state.directory, state.fieldInfos, state.segmentInfo,
+                                                    pulsingReader,
+                                                    state.context,
+                                                    state.segmentSuffix);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docsReader, pulsingReader);
+      }
+    }
+  }
+
+  public int getFreqCutoff() {
+    return freqCutoff;
+  }
+}

Added: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java?rev=1531949&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java (added)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/pulsing/PulsingPostingsWriter.java Mon Oct 14 15:55:57 2013
@@ -0,0 +1,378 @@
+package org.apache.lucene.codecs.pulsing;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.codecs.BlockTermState;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.IOUtils;
+
+// 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...
+
+/** 
+ * Writer for the pulsing format. 
+ * <p>
+ * Wraps another postings implementation and decides 
+ * (based on total number of occurrences), whether a terms 
+ * postings should be inlined into the term dictionary,
+ * or passed through to the wrapped writer.
+ *
+ * @lucene.experimental */
+public final class PulsingPostingsWriter extends PostingsWriterBase {
+
+  final static String CODEC = "PulsedPostingsWriter";
+
+  // recording field summary
+  final static String SUMMARY_EXTENSION = "smy";
+
+  // To add a new version, increment from the last one, and
+  // change VERSION_CURRENT to point to your new version:
+  final static int VERSION_START = 0;
+
+  final static int VERSION_META_ARRAY = 1;
+
+  final static int VERSION_CURRENT = VERSION_META_ARRAY;
+
+  private SegmentWriteState segmentState;
+
+  private List<FieldMetaData> fields;
+
+  // Reused by writeTerm:
+  private DocsEnum docsEnum;
+  private DocsAndPositionsEnum posEnum;
+  private int enumFlags;
+
+  private final RAMOutputStream buffer = new RAMOutputStream();
+
+  private IndexOptions indexOptions;
+
+  // information for wrapped PF, in current field
+  private int longsSize;
+  private long[] longs;
+  private boolean fieldHasFreqs;
+  private boolean fieldHasPositions;
+  private boolean fieldHasOffsets;
+  private boolean fieldHasPayloads;
+  boolean absolute;
+
+  private static class PulsingTermState extends BlockTermState {
+    private byte[] bytes;
+    private BlockTermState wrappedState;
+
+    @Override
+    public String toString() {
+      if (bytes != null) {
+        return "inlined";
+      } else {
+        return "not inlined wrapped=" + wrappedState;
+      }
+    }
+  }
+
+  private static final class FieldMetaData {
+    int fieldNumber;
+    int longsSize;
+    FieldMetaData(int number, int size) {
+      fieldNumber = number;
+      longsSize = size;
+    }
+  }
+
+  // TODO: -- lazy init this?  ie, if every single term
+  // 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;
+
+  final int maxPositions;
+
+  /** If the total number of positions (summed across all docs
+   *  for this term) is <= maxPositions, then the postings are
+   *  inlined into terms dict */
+  public PulsingPostingsWriter(SegmentWriteState state, int maxPositions, PostingsWriterBase wrappedPostingsWriter) {
+    fields = new ArrayList<FieldMetaData>();
+    this.maxPositions = maxPositions;
+    // We simply wrap another postings writer, but only call
+    // on it when tot positions is >= the cutoff:
+    this.wrappedPostingsWriter = wrappedPostingsWriter;
+    this.segmentState = state;
+  }
+
+  @Override
+  public void init(IndexOutput termsOut) throws IOException {
+    CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
+    termsOut.writeVInt(maxPositions); // encode maxPositions in header
+    wrappedPostingsWriter.init(termsOut);
+  }
+
+  @Override
+  public BlockTermState writeTerm(BytesRef term, TermsEnum termsEnum, FixedBitSet docsSeen) throws IOException {
+
+    // First pass: figure out whether we should pulse this term
+    long posCount = 0;
+
+    if (fieldHasPositions == false) {
+      // No positions:
+      docsEnum = termsEnum.docs(null, docsEnum, enumFlags);
+      assert docsEnum != null;
+      while (posCount <= maxPositions) {
+        if (docsEnum.nextDoc() == DocsEnum.NO_MORE_DOCS) {
+          break;
+        }
+        posCount++;
+      }
+    } else {
+      posEnum = termsEnum.docsAndPositions(null, posEnum, enumFlags);
+      assert posEnum != null;
+      while (posCount <= maxPositions) {
+        if (posEnum.nextDoc() == DocsEnum.NO_MORE_DOCS) {
+          break;
+        }
+        posCount += posEnum.freq();
+      }
+    }
+
+    if (posCount == 0) {
+      // All docs were deleted
+      return null;
+    }
+
+    // Second pass: write postings
+    if (posCount > maxPositions) {
+      // Too many positions; do not pulse.  Just lset
+      // wrapped postingsWriter encode the postings:
+
+      PulsingTermState state = new PulsingTermState();
+      state.wrappedState = wrappedPostingsWriter.writeTerm(term, termsEnum, docsSeen);
+      state.docFreq = state.wrappedState.docFreq;
+      state.totalTermFreq = state.wrappedState.totalTermFreq;
+      return state;
+    } else {
+      // Pulsed:
+      if (fieldHasPositions == false) {
+        docsEnum = termsEnum.docs(null, docsEnum, enumFlags);
+      } else {
+        posEnum = termsEnum.docsAndPositions(null, posEnum, enumFlags);
+        docsEnum = posEnum;
+      }
+      assert docsEnum != null;
+
+      // There were few enough total occurrences for this
+      // term, so we fully inline our postings data into
+      // terms dict, now:
+
+      // 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 lastDocID = 0;
+      int lastPayloadLength = -1;
+      int lastOffsetLength = -1;
+
+      int docFreq = 0;
+      long totalTermFreq = 0;
+      while (true) {
+        int docID = docsEnum.nextDoc();
+        if (docID == DocsEnum.NO_MORE_DOCS) {
+          break;
+        }
+        docsSeen.set(docID);
+
+        int delta = docID - lastDocID;
+        lastDocID = docID;
+
+        docFreq++;
+
+        if (fieldHasFreqs) {
+          int freq = docsEnum.freq();
+          totalTermFreq += freq;
+
+          if (freq == 1) {
+            buffer.writeVInt((delta << 1) | 1);
+          } else {
+            buffer.writeVInt(delta << 1);
+            buffer.writeVInt(freq);
+          }
+
+          if (fieldHasPositions) {
+            int lastPos = 0;
+            int lastOffset = 0;
+            for(int posIDX=0;posIDX<freq;posIDX++) {
+              int pos = posEnum.nextPosition();
+              int posDelta = pos - lastPos;
+              lastPos = pos;
+              int payloadLength;
+              BytesRef payload;
+              if (fieldHasPayloads) {
+                payload = posEnum.getPayload();
+                payloadLength = payload == null ? 0 : payload.length;
+                if (payloadLength != lastPayloadLength) {
+                  buffer.writeVInt((posDelta << 1)|1);
+                  buffer.writeVInt(payloadLength);
+                  lastPayloadLength = payloadLength;
+                } else {
+                  buffer.writeVInt(posDelta << 1);
+                }
+              } else {
+                payloadLength = 0;
+                payload = null;
+                buffer.writeVInt(posDelta);
+              }
+
+              if (fieldHasOffsets) {
+                int startOffset = posEnum.startOffset();
+                int endOffset = posEnum.endOffset();
+                int offsetDelta = startOffset - lastOffset;
+                int offsetLength = endOffset - startOffset;
+                if (offsetLength != lastOffsetLength) {
+                  buffer.writeVInt(offsetDelta << 1 | 1);
+                  buffer.writeVInt(offsetLength);
+                } else {
+                  buffer.writeVInt(offsetDelta << 1);
+                }
+                lastOffset = startOffset;
+                lastOffsetLength = offsetLength;             
+              }
+            
+              if (payloadLength > 0) {
+                assert fieldHasPayloads;
+                assert payload != null;
+                buffer.writeBytes(payload.bytes, payload.offset, payload.length);
+              }
+            }
+          }
+        } else {
+          buffer.writeVInt(delta);
+        }
+      }
+      
+      PulsingTermState state = new PulsingTermState();
+      state.bytes = new byte[(int) buffer.getFilePointer()];
+      state.docFreq = docFreq;
+      state.totalTermFreq = fieldHasFreqs ? totalTermFreq : -1;
+      buffer.writeTo(state.bytes, 0);
+      buffer.reset();
+      return state;
+    }
+  }
+
+  // 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
+  public int setField(FieldInfo fieldInfo) {
+    this.indexOptions = fieldInfo.getIndexOptions();
+    //if (DEBUG) System.out.println("PW field=" + fieldInfo.name + " indexOptions=" + indexOptions);
+    fieldHasPayloads = fieldInfo.hasPayloads();
+    absolute = false;
+    longsSize = wrappedPostingsWriter.setField(fieldInfo);
+    longs = new long[longsSize];
+    fields.add(new FieldMetaData(fieldInfo.number, longsSize));
+
+    fieldHasFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+    fieldHasPositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    fieldHasOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+
+    if (fieldHasFreqs == false) {
+      enumFlags = 0;
+    } else if (fieldHasPositions == false) {
+      enumFlags = DocsEnum.FLAG_FREQS;
+    } else if (fieldHasOffsets == false) {
+      if (fieldHasPayloads) {
+        enumFlags = DocsAndPositionsEnum.FLAG_PAYLOADS;
+      } else {
+        enumFlags = 0;
+      }
+    } else {
+      if (fieldHasPayloads) {
+        enumFlags = DocsAndPositionsEnum.FLAG_PAYLOADS | DocsAndPositionsEnum.FLAG_OFFSETS;
+      } else {
+        enumFlags = DocsAndPositionsEnum.FLAG_OFFSETS;
+      }
+    }
+    return 0;
+    //DEBUG = BlockTreeTermsWriter.DEBUG;
+  }
+
+  @Override
+  public void encodeTerm(long[] empty, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    PulsingTermState state = (PulsingTermState)_state;
+    assert empty.length == 0;
+    this.absolute = this.absolute || absolute;
+    if (state.bytes == null) {
+      wrappedPostingsWriter.encodeTerm(longs, buffer, fieldInfo, state.wrappedState, this.absolute);
+      for (int i = 0; i < longsSize; i++) {
+        out.writeVLong(longs[i]);
+      }
+      buffer.writeTo(out);
+      buffer.reset();
+      this.absolute = false;
+    } else {
+      out.writeVInt(state.bytes.length);
+      out.writeBytes(state.bytes, 0, state.bytes.length);
+      this.absolute = this.absolute || absolute;
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    wrappedPostingsWriter.close();
+    if (wrappedPostingsWriter instanceof PulsingPostingsWriter ||
+        VERSION_CURRENT < VERSION_META_ARRAY) {
+      return;
+    }
+    String summaryFileName = IndexFileNames.segmentFileName(segmentState.segmentInfo.name, segmentState.segmentSuffix, SUMMARY_EXTENSION);
+    IndexOutput out = null;
+    try {
+      out = segmentState.directory.createOutput(summaryFileName, segmentState.context);
+      CodecUtil.writeHeader(out, CODEC, VERSION_CURRENT);
+      out.writeVInt(fields.size());
+      for (FieldMetaData field : fields) {
+        out.writeVInt(field.fieldNumber);
+        out.writeVInt(field.longsSize);
+      }
+      out.close();
+    } finally {
+      IOUtils.closeWhileHandlingException(out);
+    }
+  }
+}

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java Mon Oct 14 15:55:57 2013
@@ -21,16 +21,15 @@ import java.io.IOException;
 
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.codecs.PushPostingsWriterBase;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
@@ -38,7 +37,7 @@ import org.apache.lucene.util.IOUtils;
  *  to .pyl, skip data to .skp
  *
  * @lucene.experimental */
-public final class SepPostingsWriter extends PostingsWriterBase {
+public final class SepPostingsWriter extends PushPostingsWriterBase {
   final static String CODEC = "SepPostingsWriter";
 
   final static String DOC_EXTENSION = "doc";
@@ -85,11 +84,8 @@ public final class SepPostingsWriter ext
 
   final int totalNumDocs;
 
-  boolean storePayloads;
   IndexOptions indexOptions;
 
-  FieldInfo fieldInfo;
-
   int lastPayloadLength;
   int lastPosition;
   long payloadStart;
@@ -190,13 +186,12 @@ public final class SepPostingsWriter ext
   // our parent calls setField whenever the field changes
   @Override
   public int setField(FieldInfo fieldInfo) {
-    this.fieldInfo = fieldInfo;
+    super.setField(fieldInfo);
     this.indexOptions = fieldInfo.getIndexOptions();
     if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
       throw new UnsupportedOperationException("this codec cannot index offsets");
     }
     skipListWriter.setIndexOptions(indexOptions);
-    storePayloads = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS && fieldInfo.hasPayloads();
     lastPayloadFP = 0;
     lastSkipFP = 0;
     lastState = setEmptyState();
@@ -233,7 +228,7 @@ public final class SepPostingsWriter ext
       // 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.setSkipData(lastDocID, writePayloads, lastPayloadLength);
       skipListWriter.bufferSkip(df);
     }
 
@@ -254,7 +249,7 @@ public final class SepPostingsWriter ext
     assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;            // not quite right (if pos=0 is repeated twice we don't catch it)
     lastPosition = position;
 
-    if (storePayloads) {
+    if (writePayloads) {
       final int payloadLength = payload == null ? 0 : payload.length;
       if (payloadLength != lastPayloadLength) {
         lastPayloadLength = payloadLength;
@@ -344,7 +339,7 @@ public final class SepPostingsWriter ext
       if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
         lastState.posIndex.copyFrom(state.posIndex, false);
         lastState.posIndex.write(out, absolute);
-        if (storePayloads) {
+        if (writePayloads) {
           if (absolute) {
             out.writeVLong(state.payloadFP);
           } else {