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 [3/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/...

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java?rev=1531949&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -0,0 +1,451 @@
+package org.apache.lucene.codecs.lucene41;
+
+
+/*
+ * 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.CodecUtil;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.FieldsProducer;
+import org.apache.lucene.codecs.MultiLevelSkipListWriter;
+import org.apache.lucene.codecs.PostingsFormat;
+import org.apache.lucene.codecs.PostingsReaderBase;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * Lucene 4.1 postings format, which encodes postings in packed integer blocks 
+ * for fast decode.
+ *
+ * <p><b>NOTE</b>: this format is still experimental and
+ * subject to change without backwards compatibility.
+ *
+ * <p>
+ * Basic idea:
+ * <ul>
+ *   <li>
+ *   <b>Packed Blocks and VInt Blocks</b>: 
+ *   <p>In packed blocks, integers are encoded with the same bit width ({@link PackedInts packed format}):
+ *      the block size (i.e. number of integers inside block) is fixed (currently 128). Additionally blocks
+ *      that are all the same value are encoded in an optimized way.</p>
+ *   <p>In VInt blocks, integers are encoded as {@link DataOutput#writeVInt VInt}:
+ *      the block size is variable.</p>
+ *   </li>
+ *
+ *   <li> 
+ *   <b>Block structure</b>: 
+ *   <p>When the postings are long enough, Lucene41PostingsFormat will try to encode most integer data 
+ *      as a packed block.</p> 
+ *   <p>Take a term with 259 documents as an example, the first 256 document ids are encoded as two packed 
+ *      blocks, while the remaining 3 are encoded as one VInt block. </p>
+ *   <p>Different kinds of data are always encoded separately into different packed blocks, but may 
+ *      possibly be interleaved into the same VInt block. </p>
+ *   <p>This strategy is applied to pairs: 
+ *      &lt;document number, frequency&gt;,
+ *      &lt;position, payload length&gt;, 
+ *      &lt;position, offset start, offset length&gt;, and
+ *      &lt;position, payload length, offsetstart, offset length&gt;.</p>
+ *   </li>
+ *
+ *   <li>
+ *   <b>Skipdata settings</b>: 
+ *   <p>The structure of skip table is quite similar to previous version of Lucene. Skip interval is the 
+ *      same as block size, and each skip entry points to the beginning of each block. However, for 
+ *      the first block, skip data is omitted.</p>
+ *   </li>
+ *
+ *   <li>
+ *   <b>Positions, Payloads, and Offsets</b>: 
+ *   <p>A position is an integer indicating where the term occurs within one document. 
+ *      A payload is a blob of metadata associated with current position. 
+ *      An offset is a pair of integers indicating the tokenized start/end offsets for given term 
+ *      in current position: it is essentially a specialized payload. </p>
+ *   <p>When payloads and offsets are not omitted, numPositions==numPayloads==numOffsets (assuming a 
+ *      null payload contributes one count). As mentioned in block structure, it is possible to encode 
+ *      these three either combined or separately. 
+ *   <p>In all cases, payloads and offsets are stored together. When encoded as a packed block, 
+ *      position data is separated out as .pos, while payloads and offsets are encoded in .pay (payload 
+ *      metadata will also be stored directly in .pay). When encoded as VInt blocks, all these three are 
+ *      stored interleaved into the .pos (so is payload metadata).</p>
+ *   <p>With this strategy, the majority of payload and offset data will be outside .pos file. 
+ *      So for queries that require only position data, running on a full index with payloads and offsets, 
+ *      this reduces disk pre-fetches.</p>
+ *   </li>
+ * </ul>
+ * </p>
+ *
+ * <p>
+ * Files and detailed format:
+ * <ul>
+ *   <li><tt>.tim</tt>: <a href="#Termdictionary">Term Dictionary</a></li>
+ *   <li><tt>.tip</tt>: <a href="#Termindex">Term Index</a></li>
+ *   <li><tt>.doc</tt>: <a href="#Frequencies">Frequencies and Skip Data</a></li>
+ *   <li><tt>.pos</tt>: <a href="#Positions">Positions</a></li>
+ *   <li><tt>.pay</tt>: <a href="#Payloads">Payloads and Offsets</a></li>
+ * </ul>
+ * </p>
+ *
+ * <a name="Termdictionary" id="Termdictionary"></a>
+ * <dl>
+ * <dd>
+ * <b>Term Dictionary</b>
+ *
+ * <p>The .tim file contains the list of terms in each
+ * field along with per-term statistics (such as docfreq)
+ * and pointers to the frequencies, positions, payload and
+ * skip data in the .doc, .pos, and .pay files.
+ * See {@link BlockTreeTermsWriter} for more details on the format.
+ * </p>
+ *
+ * <p>NOTE: The term dictionary can plug into different postings implementations:
+ * the postings writer/reader are actually responsible for encoding 
+ * and decoding the PostingsHeader and TermMetadata sections described here:</p>
+ *
+ * <ul>
+ *   <li>PostingsHeader --&gt; Header, PackedBlockSize</li>
+ *   <li>TermMetadata --&gt; (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, PayFPDelta?, 
+ *                            SkipFPDelta?</li>
+ *   <li>Header, --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   <li>PackedBlockSize, SingletonDocID --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *   <li>DocFPDelta, PosFPDelta, PayFPDelta, PosVIntBlockFPDelta, SkipFPDelta --&gt; {@link DataOutput#writeVLong VLong}</li>
+ * </ul>
+ * <p>Notes:</p>
+ * <ul>
+ *    <li>Header is a {@link CodecUtil#writeHeader CodecHeader} storing the version information
+ *        for the postings.</li>
+ *    <li>PackedBlockSize is the fixed block size for packed blocks. In packed block, bit width is 
+ *        determined by the largest integer. Smaller block size result in smaller variance among width 
+ *        of integers hence smaller indexes. Larger block size result in more efficient bulk i/o hence
+ *        better acceleration. This value should always be a multiple of 64, currently fixed as 128 as 
+ *        a tradeoff. It is also the skip interval used to accelerate {@link DocsEnum#advance(int)}.
+ *    <li>DocFPDelta determines the position of this term's TermFreqs within the .doc file. 
+ *        In particular, it is the difference of file offset between this term's
+ *        data and previous term's data (or zero, for the first term in the block).On disk it is 
+ *        stored as the difference from previous value in sequence. </li>
+ *    <li>PosFPDelta determines the position of this term's TermPositions within the .pos file.
+ *        While PayFPDelta determines the position of this term's &lt;TermPayloads, TermOffsets?&gt; within 
+ *        the .pay file. Similar to DocFPDelta, it is the difference between two file positions (or 
+ *        neglected, for fields that omit payloads and offsets).</li>
+ *    <li>PosVIntBlockFPDelta determines the position of this term's last TermPosition in last pos packed
+ *        block within the .pos file. It is synonym for PayVIntBlockFPDelta or OffsetVIntBlockFPDelta. 
+ *        This is actually used to indicate whether it is necessary to load following
+ *        payloads and offsets from .pos instead of .pay. Every time a new block of positions are to be 
+ *        loaded, the PostingsReader will use this value to check whether current block is packed format
+ *        or VInt. When packed format, payloads and offsets are fetched from .pay, otherwise from .pos. 
+ *        (this value is neglected when total number of positions i.e. totalTermFreq is less or equal 
+ *        to PackedBlockSize).
+ *    <li>SkipFPDelta determines the position of this term's SkipData within the .doc
+ *        file. In particular, it is the length of the TermFreq data.
+ *        SkipDelta is only stored if DocFreq is not smaller than SkipMinimum
+ *        (i.e. 128 in Lucene41PostingsFormat).</li>
+ *    <li>SingletonDocID is an optimization when a term only appears in one document. In this case, instead
+ *        of writing a file pointer to the .doc file (DocFPDelta), and then a VIntBlock at that location, the 
+ *        single document ID is written to the term dictionary.</li>
+ * </ul>
+ * </dd>
+ * </dl>
+ *
+ * <a name="Termindex" id="Termindex"></a>
+ * <dl>
+ * <dd>
+ * <b>Term Index</b>
+ * <p>The .tip file contains an index into the term dictionary, so that it can be 
+ * accessed randomly.  See {@link BlockTreeTermsWriter} for more details on the format.</p>
+ * </dd>
+ * </dl>
+ *
+ *
+ * <a name="Frequencies" id="Frequencies"></a>
+ * <dl>
+ * <dd>
+ * <b>Frequencies and Skip Data</b>
+ *
+ * <p>The .doc file contains the lists of documents which contain each term, along
+ * with the frequency of the term in that document (except when frequencies are
+ * omitted: {@link IndexOptions#DOCS_ONLY}). It also saves skip data to the beginning of 
+ * each packed or VInt block, when the length of document list is larger than packed block size.</p>
+ *
+ * <ul>
+ *   <li>docFile(.doc) --&gt; Header, &lt;TermFreqs, SkipData?&gt;<sup>TermCount</sup></li>
+ *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   <li>TermFreqs --&gt; &lt;PackedBlock&gt; <sup>PackedDocBlockNum</sup>,  
+ *                        VIntBlock? </li>
+ *   <li>PackedBlock --&gt; PackedDocDeltaBlock, PackedFreqBlock?
+ *   <li>VIntBlock --&gt; &lt;DocDelta[, Freq?]&gt;<sup>DocFreq-PackedBlockSize*PackedDocBlockNum</sup>
+ *   <li>SkipData --&gt; &lt;&lt;SkipLevelLength, SkipLevel&gt;
+ *       <sup>NumSkipLevels-1</sup>, SkipLevel&gt;, SkipDatum?</li>
+ *   <li>SkipLevel --&gt; &lt;SkipDatum&gt; <sup>TrimmedDocFreq/(PackedBlockSize^(Level + 1))</sup></li>
+ *   <li>SkipDatum --&gt; DocSkip, DocFPSkip, &lt;PosFPSkip, PosBlockOffset, PayLength?, 
+ *                        PayFPSkip?&gt;?, SkipChildLevelPointer?</li>
+ *   <li>PackedDocDeltaBlock, PackedFreqBlock --&gt; {@link PackedInts PackedInts}</li>
+ *   <li>DocDelta, Freq, DocSkip, DocFPSkip, PosFPSkip, PosBlockOffset, PayByteUpto, PayFPSkip 
+ *       --&gt; 
+ *   {@link DataOutput#writeVInt VInt}</li>
+ *   <li>SkipChildLevelPointer --&gt; {@link DataOutput#writeVLong VLong}</li>
+ * </ul>
+ * <p>Notes:</p>
+ * <ul>
+ *   <li>PackedDocDeltaBlock is theoretically generated from two steps: 
+ *     <ol>
+ *       <li>Calculate the difference between each document number and previous one, 
+ *           and get a d-gaps list (for the first document, use absolute value); </li>
+ *       <li>For those d-gaps from first one to PackedDocBlockNum*PackedBlockSize<sup>th</sup>, 
+ *           separately encode as packed blocks.</li>
+ *     </ol>
+ *     If frequencies are not omitted, PackedFreqBlock will be generated without d-gap step.
+ *   </li>
+ *   <li>VIntBlock stores remaining d-gaps (along with frequencies when possible) with a format 
+ *       that encodes DocDelta and Freq:
+ *       <p>DocDelta: if frequencies are indexed, this determines both the document
+ *       number and the frequency. In particular, DocDelta/2 is the difference between
+ *       this document number and the previous document number (or zero when this is the
+ *       first document in a TermFreqs). When DocDelta is odd, the frequency is one.
+ *       When DocDelta is even, the frequency is read as another VInt. If frequencies
+ *       are omitted, DocDelta contains the gap (not multiplied by 2) between document
+ *       numbers and no frequency information is stored.</p>
+ *       <p>For example, the TermFreqs for a term which occurs once in document seven
+ *          and three times in document eleven, with frequencies indexed, would be the
+ *          following sequence of VInts:</p>
+ *       <p>15, 8, 3</p>
+ *       <p>If frequencies were omitted ({@link IndexOptions#DOCS_ONLY}) it would be this
+ *          sequence of VInts instead:</p>
+ *       <p>7,4</p>
+ *   </li>
+ *   <li>PackedDocBlockNum is the number of packed blocks for current term's docids or frequencies. 
+ *       In particular, PackedDocBlockNum = floor(DocFreq/PackedBlockSize) </li>
+ *   <li>TrimmedDocFreq = DocFreq % PackedBlockSize == 0 ? DocFreq - 1 : DocFreq. 
+ *       We use this trick since the definition of skip entry is a little different from base interface.
+ *       In {@link MultiLevelSkipListWriter}, skip data is assumed to be saved for
+ *       skipInterval<sup>th</sup>, 2*skipInterval<sup>th</sup> ... posting in the list. However, 
+ *       in Lucene41PostingsFormat, the skip data is saved for skipInterval+1<sup>th</sup>, 
+ *       2*skipInterval+1<sup>th</sup> ... posting (skipInterval==PackedBlockSize in this case). 
+ *       When DocFreq is multiple of PackedBlockSize, MultiLevelSkipListWriter will expect one 
+ *       more skip data than Lucene41SkipWriter. </li>
+ *   <li>SkipDatum is the metadata of one skip entry.
+ *      For the first block (no matter packed or VInt), it is omitted.</li>
+ *   <li>DocSkip records the document number of every PackedBlockSize<sup>th</sup> document number in
+ *       the postings (i.e. last document number in each packed block). On disk it is stored as the 
+ *       difference from previous value in the sequence. </li>
+ *   <li>DocFPSkip records the file offsets of each block (excluding )posting at 
+ *       PackedBlockSize+1<sup>th</sup>, 2*PackedBlockSize+1<sup>th</sup> ... , in DocFile. 
+ *       The file offsets are relative to the start of current term's TermFreqs. 
+ *       On disk it is also stored as the difference from previous SkipDatum in the sequence.</li>
+ *   <li>Since positions and payloads are also block encoded, the skip should skip to related block first,
+ *       then fetch the values according to in-block offset. PosFPSkip and PayFPSkip record the file 
+ *       offsets of related block in .pos and .pay, respectively. While PosBlockOffset indicates
+ *       which value to fetch inside the related block (PayBlockOffset is unnecessary since it is always
+ *       equal to PosBlockOffset). Same as DocFPSkip, the file offsets are relative to the start of 
+ *       current term's TermFreqs, and stored as a difference sequence.</li>
+ *   <li>PayByteUpto indicates the start offset of the current payload. It is equivalent to
+ *       the sum of the payload lengths in the current block up to PosBlockOffset</li>
+ * </ul>
+ * </dd>
+ * </dl>
+ *
+ * <a name="Positions" id="Positions"></a>
+ * <dl>
+ * <dd>
+ * <b>Positions</b>
+ * <p>The .pos file contains the lists of positions that each term occurs at within documents. It also
+ *    sometimes stores part of payloads and offsets for speedup.</p>
+ * <ul>
+ *   <li>PosFile(.pos) --&gt; Header, &lt;TermPositions&gt; <sup>TermCount</sup></li>
+ *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   <li>TermPositions --&gt; &lt;PackedPosDeltaBlock&gt; <sup>PackedPosBlockNum</sup>,  
+ *                            VIntBlock? </li>
+ *   <li>VIntBlock --&gt; &lt;PositionDelta[, PayloadLength?], PayloadData?, 
+ *                        OffsetDelta?, OffsetLength?&gt;<sup>PosVIntCount</sup>
+ *   <li>PackedPosDeltaBlock --&gt; {@link PackedInts PackedInts}</li>
+ *   <li>PositionDelta, OffsetDelta, OffsetLength --&gt; 
+ *       {@link DataOutput#writeVInt VInt}</li>
+ *   <li>PayloadData --&gt; {@link DataOutput#writeByte byte}<sup>PayLength</sup></li>
+ * </ul>
+ * <p>Notes:</p>
+ * <ul>
+ *   <li>TermPositions are order by term (terms are implicit, from the term dictionary), and position 
+ *       values for each term document pair are incremental, and ordered by document number.</li>
+ *   <li>PackedPosBlockNum is the number of packed blocks for current term's positions, payloads or offsets. 
+ *       In particular, PackedPosBlockNum = floor(totalTermFreq/PackedBlockSize) </li>
+ *   <li>PosVIntCount is the number of positions encoded as VInt format. In particular, 
+ *       PosVIntCount = totalTermFreq - PackedPosBlockNum*PackedBlockSize</li>
+ *   <li>The procedure how PackedPosDeltaBlock is generated is the same as PackedDocDeltaBlock 
+ *       in chapter <a href="#Frequencies">Frequencies and Skip Data</a>.</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>For example, the TermPositions for a term which occurs as the fourth term in
+ *       one document, and as the fifth and ninth term in a subsequent document, would
+ *       be the following sequence of VInts (payloads disabled):
+ *       <p>4, 5, 4</p></li>
+ *   <li>PayloadData is metadata associated with the current 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.</li>
+ *   <li>OffsetDelta/2 is the difference between this position's startOffset from the
+ *       previous occurrence (or zero, if this is the first occurrence in this document).
+ *       If OffsetDelta is odd, then the length (endOffset-startOffset) differs from the
+ *       previous occurrence and an OffsetLength follows. Offset data is only written for
+ *       {@link IndexOptions#DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS}.</li>
+ * </ul>
+ * </dd>
+ * </dl>
+ *
+ * <a name="Payloads" id="Payloads"></a>
+ * <dl>
+ * <dd>
+ * <b>Payloads and Offsets</b>
+ * <p>The .pay file will store payloads and offsets associated with certain term-document positions. 
+ *    Some payloads and offsets will be separated out into .pos file, for performance reasons.</p>
+ * <ul>
+ *   <li>PayFile(.pay): --&gt; Header, &lt;TermPayloads, TermOffsets?&gt; <sup>TermCount</sup></li>
+ *   <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
+ *   <li>TermPayloads --&gt; &lt;PackedPayLengthBlock, SumPayLength, PayData&gt; <sup>PackedPayBlockNum</sup>
+ *   <li>TermOffsets --&gt; &lt;PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock&gt; <sup>PackedPayBlockNum</sup>
+ *   <li>PackedPayLengthBlock, PackedOffsetStartDeltaBlock, PackedOffsetLengthBlock --&gt; {@link PackedInts PackedInts}</li>
+ *   <li>SumPayLength --&gt; {@link DataOutput#writeVInt VInt}</li>
+ *   <li>PayData --&gt; {@link DataOutput#writeByte byte}<sup>SumPayLength</sup></li>
+ * </ul>
+ * <p>Notes:</p>
+ * <ul>
+ *   <li>The order of TermPayloads/TermOffsets will be the same as TermPositions, note that part of 
+ *       payload/offsets are stored in .pos.</li>
+ *   <li>The procedure how PackedPayLengthBlock and PackedOffsetLengthBlock are generated is the 
+ *       same as PackedFreqBlock in chapter <a href="#Frequencies">Frequencies and Skip Data</a>. 
+ *       While PackedStartDeltaBlock follows a same procedure as PackedDocDeltaBlock.</li>
+ *   <li>PackedPayBlockNum is always equal to PackedPosBlockNum, for the same term. It is also synonym 
+ *       for PackedOffsetBlockNum.</li>
+ *   <li>SumPayLength is the total length of payloads written within one block, should be the sum
+ *       of PayLengths in one packed block.</li>
+ *   <li>PayLength in PackedPayLengthBlock is the length of each payload associated with the current 
+ *       position.</li>
+ * </ul>
+ * </dd>
+ * </dl>
+ * </p>
+ *
+ * @lucene.experimental
+ */
+
+public final class Lucene41PostingsFormat extends PostingsFormat {
+  /**
+   * Filename extension for document number, frequencies, and skip data.
+   * See chapter: <a href="#Frequencies">Frequencies and Skip Data</a>
+   */
+  public static final String DOC_EXTENSION = "doc";
+
+  /**
+   * Filename extension for positions. 
+   * See chapter: <a href="#Positions">Positions</a>
+   */
+  public static final String POS_EXTENSION = "pos";
+
+  /**
+   * Filename extension for payloads and offsets.
+   * See chapter: <a href="#Payloads">Payloads and Offsets</a>
+   */
+  public static final String PAY_EXTENSION = "pay";
+
+  private final int minTermBlockSize;
+  private final int maxTermBlockSize;
+
+  /**
+   * Fixed packed block size, number of integers encoded in 
+   * a single packed block.
+   */
+  // NOTE: must be multiple of 64 because of PackedInts long-aligned encoding/decoding
+  public final static int BLOCK_SIZE = 128;
+
+  /** Creates {@code Lucene41PostingsFormat} with default
+   *  settings. */
+  public Lucene41PostingsFormat() {
+    this(BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE, BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
+  }
+
+  /** Creates {@code Lucene41PostingsFormat} with custom
+   *  values for {@code minBlockSize} and {@code
+   *  maxBlockSize} passed to block terms dictionary.
+   *  @see BlockTreeTermsWriter#BlockTreeTermsWriter(SegmentWriteState,PostingsWriterBase,int,int) */
+  public Lucene41PostingsFormat(int minTermBlockSize, int maxTermBlockSize) {
+    super("Lucene41");
+    this.minTermBlockSize = minTermBlockSize;
+    assert minTermBlockSize > 1;
+    this.maxTermBlockSize = maxTermBlockSize;
+    assert minTermBlockSize <= maxTermBlockSize;
+  }
+
+  @Override
+  public String toString() {
+    return getName() + "(blocksize=" + BLOCK_SIZE + ")";
+  }
+
+  @Override
+  public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
+    PostingsWriterBase postingsWriter = new Lucene41PostingsWriter(state);
+
+    boolean success = false;
+    try {
+      FieldsConsumer ret = new BlockTreeTermsWriter(state, 
+                                                    postingsWriter,
+                                                    minTermBlockSize, 
+                                                    maxTermBlockSize);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(postingsWriter);
+      }
+    }
+  }
+
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
+    PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
+                                                                state.fieldInfos,
+                                                                state.segmentInfo,
+                                                                state.context,
+                                                                state.segmentSuffix);
+    boolean success = false;
+    try {
+      FieldsProducer ret = new BlockTreeTermsReader(state.directory,
+                                                    state.fieldInfos,
+                                                    state.segmentInfo,
+                                                    postingsReader,
+                                                    state.context,
+                                                    state.segmentSuffix);
+      success = true;
+      return ret;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(postingsReader);
+      }
+    }
+  }
+}

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java?rev=1531949&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java Mon Oct 14 15:55:57 2013
@@ -0,0 +1,574 @@
+package org.apache.lucene.codecs.lucene41;
+
+/*
+ * 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.BlockTermState;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.PushPostingsWriterBase;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.TermState;
+import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.packed.PackedInts;
+
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_DATA_SIZE;
+import static org.apache.lucene.codecs.lucene41.ForUtil.MAX_ENCODED_SIZE;
+import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
+
+
+/**
+ * Concrete class that writes docId(maybe frq,pos,offset,payloads) list
+ * with postings format.
+ *
+ * Postings list for each term will be stored separately. 
+ *
+ * @see Lucene41SkipWriter for details about skipping setting and postings layout.
+ * @lucene.experimental
+ */
+public final class Lucene41PostingsWriter extends PushPostingsWriterBase {
+
+  /** 
+   * Expert: The maximum number of skip levels. Smaller values result in 
+   * slightly smaller indexes, but slower skipping in big posting lists.
+   */
+  static final int maxSkipLevels = 10;
+
+  final static String TERMS_CODEC = "Lucene41PostingsWriterTerms";
+  final static String DOC_CODEC = "Lucene41PostingsWriterDoc";
+  final static String POS_CODEC = "Lucene41PostingsWriterPos";
+  final static String PAY_CODEC = "Lucene41PostingsWriterPay";
+
+  // Increment version to change it
+  final static int VERSION_START = 0;
+  final static int VERSION_META_ARRAY = 1;
+  final static int VERSION_CURRENT = VERSION_META_ARRAY;
+
+  final IndexOutput docOut;
+  final IndexOutput posOut;
+  final IndexOutput payOut;
+
+  final static IntBlockTermState emptyState = new IntBlockTermState();
+  IntBlockTermState lastState;
+
+  // Holds starting file pointers for current term:
+  private long docStartFP;
+  private long posStartFP;
+  private long payStartFP;
+
+  final int[] docDeltaBuffer;
+  final int[] freqBuffer;
+  private int docBufferUpto;
+
+  final int[] posDeltaBuffer;
+  final int[] payloadLengthBuffer;
+  final int[] offsetStartDeltaBuffer;
+  final int[] offsetLengthBuffer;
+  private int posBufferUpto;
+
+  private byte[] payloadBytes;
+  private int payloadByteUpto;
+
+  private int lastBlockDocID;
+  private long lastBlockPosFP;
+  private long lastBlockPayFP;
+  private int lastBlockPosBufferUpto;
+  private int lastBlockPayloadByteUpto;
+
+  private int lastDocID;
+  private int lastPosition;
+  private int lastStartOffset;
+  private int docCount;
+
+  final byte[] encoded;
+
+  private final ForUtil forUtil;
+  private final Lucene41SkipWriter skipWriter;
+  
+  /** Creates a postings writer with the specified PackedInts overhead ratio */
+  // TODO: does this ctor even make sense?
+  public Lucene41PostingsWriter(SegmentWriteState state, float acceptableOverheadRatio) throws IOException {
+    super();
+
+    docOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
+                                          state.context);
+    IndexOutput posOut = null;
+    IndexOutput payOut = null;
+    boolean success = false;
+    try {
+      CodecUtil.writeHeader(docOut, DOC_CODEC, VERSION_CURRENT);
+      forUtil = new ForUtil(acceptableOverheadRatio, docOut);
+      if (state.fieldInfos.hasProx()) {
+        posDeltaBuffer = new int[MAX_DATA_SIZE];
+        posOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
+                                              state.context);
+        CodecUtil.writeHeader(posOut, POS_CODEC, VERSION_CURRENT);
+
+        if (state.fieldInfos.hasPayloads()) {
+          payloadBytes = new byte[128];
+          payloadLengthBuffer = new int[MAX_DATA_SIZE];
+        } else {
+          payloadBytes = null;
+          payloadLengthBuffer = null;
+        }
+
+        if (state.fieldInfos.hasOffsets()) {
+          offsetStartDeltaBuffer = new int[MAX_DATA_SIZE];
+          offsetLengthBuffer = new int[MAX_DATA_SIZE];
+        } else {
+          offsetStartDeltaBuffer = null;
+          offsetLengthBuffer = null;
+        }
+
+        if (state.fieldInfos.hasPayloads() || state.fieldInfos.hasOffsets()) {
+          payOut = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
+                                                state.context);
+          CodecUtil.writeHeader(payOut, PAY_CODEC, VERSION_CURRENT);
+        }
+      } else {
+        posDeltaBuffer = null;
+        payloadLengthBuffer = null;
+        offsetStartDeltaBuffer = null;
+        offsetLengthBuffer = null;
+        payloadBytes = null;
+      }
+      this.payOut = payOut;
+      this.posOut = posOut;
+      success = true;
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(docOut, posOut, payOut);
+      }
+    }
+
+    docDeltaBuffer = new int[MAX_DATA_SIZE];
+    freqBuffer = new int[MAX_DATA_SIZE];
+
+    // TODO: should we try skipping every 2/4 blocks...?
+    skipWriter = new Lucene41SkipWriter(maxSkipLevels,
+                                     BLOCK_SIZE, 
+                                     state.segmentInfo.getDocCount(),
+                                     docOut,
+                                     posOut,
+                                     payOut);
+
+    encoded = new byte[MAX_ENCODED_SIZE];
+  }
+
+  /** Creates a postings writer with <code>PackedInts.COMPACT</code> */
+  public Lucene41PostingsWriter(SegmentWriteState state) throws IOException {
+    this(state, PackedInts.COMPACT);
+  }
+
+  final static class IntBlockTermState extends BlockTermState {
+    long docStartFP = 0;
+    long posStartFP = 0;
+    long payStartFP = 0;
+    long skipOffset = -1;
+    long lastPosBlockOffset = -1;
+    // docid when there is a single pulsed posting, otherwise -1
+    // freq is always implicitly totalTermFreq in this case.
+    int singletonDocID = -1;
+
+    @Override
+    public IntBlockTermState clone() {
+      IntBlockTermState other = new IntBlockTermState();
+      other.copyFrom(this);
+      return other;
+    }
+
+    @Override
+    public void copyFrom(TermState _other) {
+      super.copyFrom(_other);
+      IntBlockTermState other = (IntBlockTermState) _other;
+      docStartFP = other.docStartFP;
+      posStartFP = other.posStartFP;
+      payStartFP = other.payStartFP;
+      lastPosBlockOffset = other.lastPosBlockOffset;
+      skipOffset = other.skipOffset;
+      singletonDocID = other.singletonDocID;
+    }
+
+
+    @Override
+    public String toString() {
+      return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
+    }
+  }
+
+  @Override
+  public IntBlockTermState newTermState() {
+    return new IntBlockTermState();
+  }
+
+  @Override
+  public void init(IndexOutput termsOut) throws IOException {
+    CodecUtil.writeHeader(termsOut, TERMS_CODEC, VERSION_CURRENT);
+    termsOut.writeVInt(BLOCK_SIZE);
+  }
+
+  @Override
+  public int setField(FieldInfo fieldInfo) {
+    super.setField(fieldInfo);
+    skipWriter.setField(writePositions, writeOffsets, writePayloads);
+    lastState = emptyState;
+    if (writePositions) {
+      if (writePayloads || writeOffsets) {
+        return 3;  // doc + pos + pay FP
+      } else {
+        return 2;  // doc + pos FP
+      }
+    } else {
+      return 1;    // doc FP
+    }
+  }
+
+  @Override
+  public void startTerm() {
+    docStartFP = docOut.getFilePointer();
+    if (writePositions) {
+      posStartFP = posOut.getFilePointer();
+      if (writePayloads || writeOffsets) {
+        payStartFP = payOut.getFilePointer();
+      }
+    }
+    lastDocID = 0;
+    lastBlockDocID = -1;
+    // if (DEBUG) {
+    //   System.out.println("FPW.startTerm startFP=" + docStartFP);
+    // }
+    skipWriter.resetSkip();
+  }
+
+  @Override
+  public void startDoc(int docID, int termDocFreq) throws IOException {
+    // if (DEBUG) {
+    //   System.out.println("FPW.startDoc docID["+docBufferUpto+"]=" + docID);
+    // }
+    // Have collected a block of docs, and get a new doc. 
+    // Should write skip data as well as postings list for
+    // current block.
+    if (lastBlockDocID != -1 && docBufferUpto == 0) {
+      // if (DEBUG) {
+      //   System.out.println("  bufferSkip at writeBlock: lastDocID=" + lastBlockDocID + " docCount=" + (docCount-1));
+      // }
+      skipWriter.bufferSkip(lastBlockDocID, docCount, lastBlockPosFP, lastBlockPayFP, lastBlockPosBufferUpto, lastBlockPayloadByteUpto);
+    }
+
+    final int docDelta = docID - lastDocID;
+
+    if (docID < 0 || (docCount > 0 && docDelta <= 0)) {
+      throw new CorruptIndexException("docs out of order (" + docID + " <= " + lastDocID + " ) (docOut: " + docOut + ")");
+    }
+
+    docDeltaBuffer[docBufferUpto] = docDelta;
+    // if (DEBUG) {
+    //   System.out.println("  docDeltaBuffer[" + docBufferUpto + "]=" + docDelta);
+    // }
+    if (writeFreqs) {
+      freqBuffer[docBufferUpto] = termDocFreq;
+    }
+    docBufferUpto++;
+    docCount++;
+
+    if (docBufferUpto == BLOCK_SIZE) {
+      // if (DEBUG) {
+      //   System.out.println("  write docDelta block @ fp=" + docOut.getFilePointer());
+      // }
+      forUtil.writeBlock(docDeltaBuffer, encoded, docOut);
+      if (writeFreqs) {
+        // if (DEBUG) {
+        //   System.out.println("  write freq block @ fp=" + docOut.getFilePointer());
+        // }
+        forUtil.writeBlock(freqBuffer, encoded, docOut);
+      }
+      // NOTE: don't set docBufferUpto back to 0 here;
+      // finishDoc will do so (because it needs to see that
+      // the block was filled so it can save skip data)
+    }
+
+
+    lastDocID = docID;
+    lastPosition = 0;
+    lastStartOffset = 0;
+  }
+
+  @Override
+  public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
+    // if (DEBUG) {
+    //   System.out.println("FPW.addPosition pos=" + position + " posBufferUpto=" + posBufferUpto + (writePayloads ? " payloadByteUpto=" + payloadByteUpto: ""));
+    // }
+    posDeltaBuffer[posBufferUpto] = position - lastPosition;
+    if (writePayloads) {
+      if (payload == null || payload.length == 0) {
+        // no payload
+        payloadLengthBuffer[posBufferUpto] = 0;
+      } else {
+        payloadLengthBuffer[posBufferUpto] = payload.length;
+        if (payloadByteUpto + payload.length > payloadBytes.length) {
+          payloadBytes = ArrayUtil.grow(payloadBytes, payloadByteUpto + payload.length);
+        }
+        System.arraycopy(payload.bytes, payload.offset, payloadBytes, payloadByteUpto, payload.length);
+        payloadByteUpto += payload.length;
+      }
+    }
+
+    if (writeOffsets) {
+      assert startOffset >= lastStartOffset;
+      assert endOffset >= startOffset;
+      offsetStartDeltaBuffer[posBufferUpto] = startOffset - lastStartOffset;
+      offsetLengthBuffer[posBufferUpto] = endOffset - startOffset;
+      lastStartOffset = startOffset;
+    }
+    
+    posBufferUpto++;
+    lastPosition = position;
+    if (posBufferUpto == BLOCK_SIZE) {
+      // if (DEBUG) {
+      //   System.out.println("  write pos bulk block @ fp=" + posOut.getFilePointer());
+      // }
+      forUtil.writeBlock(posDeltaBuffer, encoded, posOut);
+
+      if (writePayloads) {
+        forUtil.writeBlock(payloadLengthBuffer, encoded, payOut);
+        payOut.writeVInt(payloadByteUpto);
+        payOut.writeBytes(payloadBytes, 0, payloadByteUpto);
+        payloadByteUpto = 0;
+      }
+      if (writeOffsets) {
+        forUtil.writeBlock(offsetStartDeltaBuffer, encoded, payOut);
+        forUtil.writeBlock(offsetLengthBuffer, encoded, payOut);
+      }
+      posBufferUpto = 0;
+    }
+  }
+
+  @Override
+  public void finishDoc() throws IOException {
+    // Since we don't know df for current term, we had to buffer
+    // those skip data for each block, and when a new doc comes, 
+    // write them to skip file.
+    if (docBufferUpto == BLOCK_SIZE) {
+      lastBlockDocID = lastDocID;
+      if (posOut != null) {
+        if (payOut != null) {
+          lastBlockPayFP = payOut.getFilePointer();
+        }
+        lastBlockPosFP = posOut.getFilePointer();
+        lastBlockPosBufferUpto = posBufferUpto;
+        lastBlockPayloadByteUpto = payloadByteUpto;
+      }
+      // if (DEBUG) {
+      //   System.out.println("  docBufferUpto="+docBufferUpto+" now get lastBlockDocID="+lastBlockDocID+" lastBlockPosFP=" + lastBlockPosFP + " lastBlockPosBufferUpto=" +  lastBlockPosBufferUpto + " lastBlockPayloadByteUpto=" + lastBlockPayloadByteUpto);
+      // }
+      docBufferUpto = 0;
+    }
+  }
+
+  /** Called when we are done adding docs to this term */
+  @Override
+  public void finishTerm(BlockTermState _state) throws IOException {
+    IntBlockTermState state = (IntBlockTermState) _state;
+    assert state.docFreq > 0;
+
+    // TODO: wasteful we are counting this (counting # docs
+    // for this term) in two places?
+    assert state.docFreq == docCount: state.docFreq + " vs " + docCount;
+
+    // if (DEBUG) {
+    //   System.out.println("FPW.finishTerm docFreq=" + state.docFreq);
+    // }
+
+    // if (DEBUG) {
+    //   if (docBufferUpto > 0) {
+    //     System.out.println("  write doc/freq vInt block (count=" + docBufferUpto + ") at fp=" + docOut.getFilePointer() + " docStartFP=" + docStartFP);
+    //   }
+    // }
+    
+    // docFreq == 1, don't write the single docid/freq to a separate file along with a pointer to it.
+    final int singletonDocID;
+    if (state.docFreq == 1) {
+      // pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq
+      singletonDocID = docDeltaBuffer[0];
+    } else {
+      singletonDocID = -1;
+      // vInt encode the remaining doc deltas and freqs:
+      for(int i=0;i<docBufferUpto;i++) {
+        final int docDelta = docDeltaBuffer[i];
+        final int freq = freqBuffer[i];
+        if (!writeFreqs) {
+          docOut.writeVInt(docDelta);
+        } else if (freqBuffer[i] == 1) {
+          docOut.writeVInt((docDelta<<1)|1);
+        } else {
+          docOut.writeVInt(docDelta<<1);
+          docOut.writeVInt(freq);
+        }
+      }
+    }
+
+    final long lastPosBlockOffset;
+
+    if (writePositions) {
+      // if (DEBUG) {
+      //   if (posBufferUpto > 0) {
+      //     System.out.println("  write pos vInt block (count=" + posBufferUpto + ") at fp=" + posOut.getFilePointer() + " posStartFP=" + posStartFP + " hasPayloads=" + writePayloads + " hasOffsets=" + writeOffsets);
+      //   }
+      // }
+
+      // totalTermFreq is just total number of positions(or payloads, or offsets)
+      // associated with current term.
+      assert state.totalTermFreq != -1;
+      if (state.totalTermFreq > BLOCK_SIZE) {
+        // record file offset for last pos in last block
+        lastPosBlockOffset = posOut.getFilePointer() - posStartFP;
+      } else {
+        lastPosBlockOffset = -1;
+      }
+      if (posBufferUpto > 0) {       
+        // TODO: should we send offsets/payloads to
+        // .pay...?  seems wasteful (have to store extra
+        // vLong for low (< BLOCK_SIZE) DF terms = vast vast
+        // majority)
+
+        // vInt encode the remaining positions/payloads/offsets:
+        int lastPayloadLength = -1;  // force first payload length to be written
+        int lastOffsetLength = -1;   // force first offset length to be written
+        int payloadBytesReadUpto = 0;
+        for(int i=0;i<posBufferUpto;i++) {
+          final int posDelta = posDeltaBuffer[i];
+          if (writePayloads) {
+            final int payloadLength = payloadLengthBuffer[i];
+            if (payloadLength != lastPayloadLength) {
+              lastPayloadLength = payloadLength;
+              posOut.writeVInt((posDelta<<1)|1);
+              posOut.writeVInt(payloadLength);
+            } else {
+              posOut.writeVInt(posDelta<<1);
+            }
+
+            // if (DEBUG) {
+            //   System.out.println("        i=" + i + " payloadLen=" + payloadLength);
+            // }
+
+            if (payloadLength != 0) {
+              // if (DEBUG) {
+              //   System.out.println("          write payload @ pos.fp=" + posOut.getFilePointer());
+              // }
+              posOut.writeBytes(payloadBytes, payloadBytesReadUpto, payloadLength);
+              payloadBytesReadUpto += payloadLength;
+            }
+          } else {
+            posOut.writeVInt(posDelta);
+          }
+
+          if (writeOffsets) {
+            // if (DEBUG) {
+            //   System.out.println("          write offset @ pos.fp=" + posOut.getFilePointer());
+            // }
+            int delta = offsetStartDeltaBuffer[i];
+            int length = offsetLengthBuffer[i];
+            if (length == lastOffsetLength) {
+              posOut.writeVInt(delta << 1);
+            } else {
+              posOut.writeVInt(delta << 1 | 1);
+              posOut.writeVInt(length);
+              lastOffsetLength = length;
+            }
+          }
+        }
+
+        if (writePayloads) {
+          assert payloadBytesReadUpto == payloadByteUpto;
+          payloadByteUpto = 0;
+        }
+      }
+      // if (DEBUG) {
+      //   System.out.println("  totalTermFreq=" + state.totalTermFreq + " lastPosBlockOffset=" + lastPosBlockOffset);
+      // }
+    } else {
+      lastPosBlockOffset = -1;
+    }
+
+    long skipOffset;
+    if (docCount > BLOCK_SIZE) {
+      skipOffset = skipWriter.writeSkip(docOut) - docStartFP;
+      
+      // if (DEBUG) {
+      //   System.out.println("skip packet " + (docOut.getFilePointer() - (docStartFP + skipOffset)) + " bytes");
+      // }
+    } else {
+      skipOffset = -1;
+      // if (DEBUG) {
+      //   System.out.println("  no skip: docCount=" + docCount);
+      // }
+    }
+    // if (DEBUG) {
+    //   System.out.println("  payStartFP=" + payStartFP);
+    // }
+    state.docStartFP = docStartFP;
+    state.posStartFP = posStartFP;
+    state.payStartFP = payStartFP;
+    state.singletonDocID = singletonDocID;
+    state.skipOffset = skipOffset;
+    state.lastPosBlockOffset = lastPosBlockOffset;
+    docBufferUpto = 0;
+    posBufferUpto = 0;
+    lastDocID = 0;
+    docCount = 0;
+  }
+  
+  @Override
+  public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    IntBlockTermState state = (IntBlockTermState)_state;
+    if (absolute) {
+      lastState = emptyState;
+    }
+    longs[0] = state.docStartFP - lastState.docStartFP;
+    if (writePositions) {
+      longs[1] = state.posStartFP - lastState.posStartFP;
+      if (writePayloads || writeOffsets) {
+        longs[2] = state.payStartFP - lastState.payStartFP;
+      }
+    }
+    if (state.singletonDocID != -1) {
+      out.writeVInt(state.singletonDocID);
+    }
+    if (writePositions) {
+      if (state.lastPosBlockOffset != -1) {
+        out.writeVLong(state.lastPosBlockOffset);
+      }
+    }
+    if (state.skipOffset != -1) {
+      out.writeVLong(state.skipOffset);
+    }
+    lastState = state;
+  }
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(docOut, posOut, payOut);
+  }
+}

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -18,21 +18,15 @@ package org.apache.lucene.codecs.asserti
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 import java.util.Iterator;
 
 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.lucene41.Lucene41PostingsFormat;
 import org.apache.lucene.index.AssertingAtomicReader;
 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.Fields;
 import org.apache.lucene.index.SegmentReadState;
@@ -40,7 +34,6 @@ import org.apache.lucene.index.SegmentWr
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.OpenBitSet;
 
 /**
  * Just like {@link Lucene41PostingsFormat} but with additional asserts.
@@ -54,12 +47,7 @@ public final class AssertingPostingsForm
   
   @Override
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
-    FieldsConsumer fieldsConsumer = in.fieldsConsumer(state);
-    if (fieldsConsumer instanceof PushFieldsConsumer) {
-      return new AssertingPushFieldsConsumer(state, (PushFieldsConsumer) fieldsConsumer);
-    } else {
-      return new AssertingFieldsConsumer(state, fieldsConsumer);
-    }
+    return new AssertingFieldsConsumer(state, in.fieldsConsumer(state));
   }
 
   @Override
@@ -103,27 +91,6 @@ public final class AssertingPostingsForm
     }
   }
 
-  static class AssertingPushFieldsConsumer extends PushFieldsConsumer {
-    private final PushFieldsConsumer in;
-    
-    AssertingPushFieldsConsumer(SegmentWriteState writeState, PushFieldsConsumer in) {
-      super(writeState);
-      this.in = in;
-    }
-    
-    @Override
-    public TermsConsumer addField(FieldInfo field) throws IOException {
-      TermsConsumer consumer = in.addField(field);
-      assert consumer != null;
-      return new AssertingTermsConsumer(consumer, field);
-    }
-
-    @Override
-    public void close() throws IOException {
-      in.close();
-    }
-  }
-
   static class AssertingFieldsConsumer extends FieldsConsumer {
     private final FieldsConsumer in;
     private final SegmentWriteState writeState;
@@ -153,6 +120,9 @@ public final class AssertingPostingsForm
         lastField = field;
 
         Terms terms = fields.terms(field);
+        if (terms == null) {
+          continue;
+        }
         assert terms != null;
 
         termsEnum = terms.iterator(termsEnum);
@@ -163,6 +133,7 @@ public final class AssertingPostingsForm
         boolean hasFreqs = fieldInfo.getIndexOptions().compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS) >= 0;
         boolean hasPositions = fieldInfo.getIndexOptions().compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
         boolean hasOffsets = fieldInfo.getIndexOptions().compareTo(FieldInfo.IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+        boolean hasPayloads = terms.hasPayloads();
 
         assert hasPositions == terms.hasPositions();
         assert hasOffsets == terms.hasOffsets();
@@ -179,14 +150,16 @@ public final class AssertingPostingsForm
             lastTerm.copyBytes(term);
           }
 
+          int flags = 0;
           if (hasPositions == false) {
-            int flags = 0;
             if (hasFreqs) {
               flags = flags | DocsEnum.FLAG_FREQS;
             }
             docsEnum = termsEnum.docs(null, docsEnum, flags);
           } else {
-            int flags = DocsAndPositionsEnum.FLAG_PAYLOADS;
+            if (hasPayloads) {
+              flags |= DocsAndPositionsEnum.FLAG_PAYLOADS;
+            }
             if (hasOffsets) {
               flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS;
             }
@@ -194,6 +167,8 @@ public final class AssertingPostingsForm
             docsEnum = posEnum;
           }
 
+          assert docsEnum != null : "termsEnum=" + termsEnum + " hasPositions=" + hasPositions;
+
           int lastDocID = -1;
 
           while(true) {
@@ -212,13 +187,13 @@ public final class AssertingPostingsForm
                 int lastStartOffset = -1;
                 for(int i=0;i<freq;i++) {
                   int pos = posEnum.nextPosition();
-                  assert pos > lastPos;
+                  assert pos >= lastPos: "pos=" + pos + " vs lastPos=" + lastPos + " i=" + i + " freq=" + freq;
                   lastPos = pos;
 
                   if (hasOffsets) {
                     int startOffset = posEnum.startOffset();
                     int endOffset = posEnum.endOffset();
-                    assert endOffset > startOffset;
+                    assert endOffset >= startOffset;
                     assert startOffset >= lastStartOffset;
                     lastStartOffset = startOffset;
                   }
@@ -230,140 +205,4 @@ public final class AssertingPostingsForm
       }
     }
   }
-  
-  static enum TermsConsumerState { INITIAL, START, FINISHED };
-  static class AssertingTermsConsumer extends TermsConsumer {
-    private final TermsConsumer in;
-    private final FieldInfo fieldInfo;
-    private BytesRef lastTerm = null;
-    private TermsConsumerState state = TermsConsumerState.INITIAL;
-    private AssertingPostingsConsumer lastPostingsConsumer = null;
-    private long sumTotalTermFreq = 0;
-    private long sumDocFreq = 0;
-    private OpenBitSet visitedDocs = new OpenBitSet();
-    private static final Comparator<BytesRef> termComp = BytesRef.getUTF8SortedAsUnicodeComparator();
-    
-    AssertingTermsConsumer(TermsConsumer in, FieldInfo fieldInfo) {
-      this.in = in;
-      this.fieldInfo = fieldInfo;
-    }
-    
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) throws IOException {
-      assert state == TermsConsumerState.INITIAL || state == TermsConsumerState.START && lastPostingsConsumer.docFreq == 0;
-      state = TermsConsumerState.START;
-      assert lastTerm == null || termComp.compare(text, lastTerm) > 0;
-      lastTerm = BytesRef.deepCopyOf(text);
-      return lastPostingsConsumer = new AssertingPostingsConsumer(in.startTerm(text), fieldInfo, visitedDocs);
-    }
-
-    @Override
-    public void finishTerm(BytesRef text, TermStats stats) throws IOException {
-      assert state == TermsConsumerState.START;
-      state = TermsConsumerState.INITIAL;
-      assert text.equals(lastTerm);
-      assert stats.docFreq > 0; // otherwise, this method should not be called.
-      assert stats.docFreq == lastPostingsConsumer.docFreq;
-      sumDocFreq += stats.docFreq;
-      if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY) {
-        assert stats.totalTermFreq == -1;
-      } else {
-        assert stats.totalTermFreq == lastPostingsConsumer.totalTermFreq;
-        sumTotalTermFreq += stats.totalTermFreq;
-      }
-      in.finishTerm(text, stats);
-    }
-
-    @Override
-    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
-      assert state == TermsConsumerState.INITIAL || state == TermsConsumerState.START && lastPostingsConsumer.docFreq == 0;
-      state = TermsConsumerState.FINISHED;
-      assert docCount >= 0;
-      assert docCount == visitedDocs.cardinality();
-      assert sumDocFreq >= docCount;
-      assert sumDocFreq == this.sumDocFreq;
-      if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY) {
-        assert sumTotalTermFreq == -1;
-      } else {
-        assert sumTotalTermFreq >= sumDocFreq;
-        assert sumTotalTermFreq == this.sumTotalTermFreq;
-      }
-      in.finish(sumTotalTermFreq, sumDocFreq, docCount);
-    }
-  }
-  
-  static enum PostingsConsumerState { INITIAL, START };
-  static class AssertingPostingsConsumer extends PostingsConsumer {
-    private final PostingsConsumer in;
-    private final FieldInfo fieldInfo;
-    private final OpenBitSet visitedDocs;
-    private PostingsConsumerState state = PostingsConsumerState.INITIAL;
-    private int freq;
-    private int positionCount;
-    private int lastPosition = 0;
-    private int lastStartOffset = 0;
-    int docFreq = 0;
-    long totalTermFreq = 0;
-    
-    AssertingPostingsConsumer(PostingsConsumer in, FieldInfo fieldInfo, OpenBitSet visitedDocs) {
-      this.in = in;
-      this.fieldInfo = fieldInfo;
-      this.visitedDocs = visitedDocs;
-    }
-
-    @Override
-    public void startDoc(int docID, int freq) throws IOException {
-      assert state == PostingsConsumerState.INITIAL;
-      state = PostingsConsumerState.START;
-      assert docID >= 0;
-      if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY) {
-        assert freq == -1;
-        this.freq = 0; // we don't expect any positions here
-      } else {
-        assert freq > 0;
-        this.freq = freq;
-        totalTermFreq += freq;
-      }
-      this.positionCount = 0;
-      this.lastPosition = 0;
-      this.lastStartOffset = 0;
-      docFreq++;
-      visitedDocs.set(docID);
-      in.startDoc(docID, freq);
-    }
-
-    @Override
-    public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
-      assert state == PostingsConsumerState.START;
-      assert positionCount < freq;
-      positionCount++;
-      assert position >= lastPosition || position == -1; /* we still allow -1 from old 3.x indexes */
-      lastPosition = position;
-      if (fieldInfo.getIndexOptions() == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
-        assert startOffset >= 0;
-        assert startOffset >= lastStartOffset;
-        lastStartOffset = startOffset;
-        assert endOffset >= startOffset;
-      } else {
-        assert startOffset == -1;
-        assert endOffset == -1;
-      }
-      if (payload != null) {
-        assert fieldInfo.hasPayloads();
-      }
-      in.addPosition(position, payload, startOffset, endOffset);
-    }
-
-    @Override
-    public void finishDoc() throws IOException {
-      assert state == PostingsConsumerState.START;
-      state = PostingsConsumerState.INITIAL;
-      if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-        assert positionCount == 0; // we should not have fed any positions!
-      } else {
-        assert positionCount == freq;
-      }
-      in.finishDoc();
-    }
-  }
 }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java Mon Oct 14 15:55:57 2013
@@ -21,22 +21,18 @@ package org.apache.lucene.codecs.lucene4
  *  index file format */
 
 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.codecs.TermStats;
+import org.apache.lucene.codecs.PushPostingsWriterBase;
 import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.DocsEnum;  // javadocs
 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;
 
@@ -46,7 +42,7 @@ import org.apache.lucene.util.IOUtils;
  * @see Lucene40PostingsFormat
  * @lucene.experimental 
  */
-public final class Lucene40PostingsWriter extends PostingsWriterBase {
+public final class Lucene40PostingsWriter extends PushPostingsWriterBase {
 
   final IndexOutput freqOut;
   final IndexOutput proxOut;
@@ -70,13 +66,9 @@ public final class Lucene40PostingsWrite
   final int maxSkipLevels = 10;
   final int totalNumDocs;
 
-  IndexOptions indexOptions;
-  boolean storePayloads;
-  boolean storeOffsets;
   // Starts a new term
   long freqStart;
   long proxStart;
-  FieldInfo fieldInfo;
   int lastPayloadLength;
   int lastOffsetLength;
   int lastPosition;
@@ -150,7 +142,6 @@ public final class Lucene40PostingsWrite
     return new StandardTermState();
   }
 
-
   @Override
   public void startTerm() {
     freqStart = freqOut.getFilePointer();
@@ -169,6 +160,7 @@ public final class Lucene40PostingsWrite
   // our parent calls setField whenever the field changes
   @Override
   public int setField(FieldInfo fieldInfo) {
+    super.setField(fieldInfo);
     //System.out.println("SPW: setField");
     /*
     if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) {
@@ -177,11 +169,7 @@ public final class Lucene40PostingsWrite
       DEBUG = false;
     }
     */
-    this.fieldInfo = fieldInfo;
-    indexOptions = fieldInfo.getIndexOptions();
-    
-    storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;        
-    storePayloads = fieldInfo.hasPayloads();
+
     lastState = emptyState;
     //System.out.println("  set init blockFreqStart=" + freqStart);
     //System.out.println("  set init blockProxStart=" + proxStart);
@@ -190,7 +178,7 @@ public final class Lucene40PostingsWrite
 
   int lastDocID;
   int df;
-  
+
   @Override
   public void startDoc(int docID, int termDocFreq) throws IOException {
     // if (DEBUG) System.out.println("SPW:   startDoc seg=" + segment + " docID=" + docID + " tf=" + termDocFreq + " freqOut.fp=" + freqOut.getFilePointer());
@@ -202,7 +190,7 @@ public final class Lucene40PostingsWrite
     }
 
     if ((++df % skipInterval) == 0) {
-      skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength, storeOffsets, lastOffsetLength);
+      skipListWriter.setSkipData(lastDocID, writePayloads, lastPayloadLength, writeOffsets, lastOffsetLength);
       skipListWriter.bufferSkip(df);
     }
 
@@ -237,7 +225,7 @@ public final class Lucene40PostingsWrite
 
     int payloadLength = 0;
 
-    if (storePayloads) {
+    if (writePayloads) {
       payloadLength = payload == null ? 0 : payload.length;
 
       if (payloadLength != lastPayloadLength) {
@@ -251,7 +239,7 @@ public final class Lucene40PostingsWrite
       proxOut.writeVInt(delta);
     }
     
-    if (storeOffsets) {
+    if (writeOffsets) {
       // don't use startOffset - lastEndOffset, because this creates lots of negative vints for synonyms,
       // and the numbers aren't that much smaller anyways.
       int offsetDelta = startOffset - lastOffset;
@@ -285,7 +273,7 @@ public final class Lucene40PostingsWrite
   /** Called when we are done adding docs to this term */
   @Override
   public void finishTerm(BlockTermState _state) throws IOException {
-    StandardTermState state = (StandardTermState)_state;
+    StandardTermState state = (StandardTermState) _state;
     // if (DEBUG) System.out.println("SPW: finishTerm seg=" + segment + " freqStart=" + freqStart);
     assert state.docFreq > 0;
 

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40RWPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -1,13 +1,5 @@
 package org.apache.lucene.codecs.lucene40;
 
-import java.io.IOException;
-
-import org.apache.lucene.codecs.BlockTreeTermsWriter;
-import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.util.LuceneTestCase;
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -25,6 +17,14 @@ import org.apache.lucene.util.LuceneTest
  * limitations under the License.
  */
 
+import java.io.IOException;
+
+import org.apache.lucene.codecs.BlockTreeTermsWriter;
+import org.apache.lucene.codecs.FieldsConsumer;
+import org.apache.lucene.codecs.PostingsWriterBase;
+import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.LuceneTestCase;
+
 /**
  * Read-write version of {@link Lucene40PostingsFormat} for testing.
  */

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockFixedIntBlockPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -37,8 +37,8 @@ import org.apache.lucene.codecs.sep.IntI
 import org.apache.lucene.codecs.sep.IntStreamFactory;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.*;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockintblock/MockVariableIntBlockPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -37,8 +37,8 @@ import org.apache.lucene.codecs.sep.IntI
 import org.apache.lucene.codecs.sep.IntStreamFactory;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -40,6 +40,10 @@ import org.apache.lucene.codecs.blockter
 import org.apache.lucene.codecs.blockterms.VariableGapTermsIndexWriter;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsReader;
 import org.apache.lucene.codecs.lucene41.Lucene41PostingsWriter;
+import org.apache.lucene.codecs.memory.FSTOrdTermsReader;
+import org.apache.lucene.codecs.memory.FSTOrdTermsWriter;
+import org.apache.lucene.codecs.memory.FSTTermsReader;
+import org.apache.lucene.codecs.memory.FSTTermsWriter;
 import org.apache.lucene.codecs.mockintblock.MockFixedIntBlockPostingsFormat;
 import org.apache.lucene.codecs.mockintblock.MockVariableIntBlockPostingsFormat;
 import org.apache.lucene.codecs.mocksep.MockSingleIntFactory;
@@ -50,10 +54,6 @@ import org.apache.lucene.codecs.sep.IntI
 import org.apache.lucene.codecs.sep.IntStreamFactory;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.codecs.memory.FSTTermsWriter;
-import org.apache.lucene.codecs.memory.FSTTermsReader;
-import org.apache.lucene.codecs.memory.FSTOrdTermsWriter;
-import org.apache.lucene.codecs.memory.FSTOrdTermsReader;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/mocksep/MockSepPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -32,8 +32,8 @@ import org.apache.lucene.codecs.blockter
 import org.apache.lucene.codecs.blockterms.TermsIndexWriterBase;
 import org.apache.lucene.codecs.sep.SepPostingsReader;
 import org.apache.lucene.codecs.sep.SepPostingsWriter;
-import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.util.BytesRef;
 
 /**

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java Mon Oct 14 15:55:57 2013
@@ -31,15 +31,13 @@ import java.util.concurrent.atomic.Atomi
 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.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.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -49,6 +47,7 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.store.IndexOutput;
 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.RamUsageEstimator;
 
@@ -208,34 +207,132 @@ public final class RAMOnlyPostingsFormat
   }
 
   // Classes for writing to the postings state
-  private static class RAMFieldsConsumer extends PushFieldsConsumer {
+  private static class RAMFieldsConsumer extends FieldsConsumer {
 
     private final RAMPostings postings;
     private final RAMTermsConsumer termsConsumer = new RAMTermsConsumer();
+    private final SegmentWriteState state;
 
     public RAMFieldsConsumer(SegmentWriteState writeState, RAMPostings postings) {
-      super(writeState);
       this.postings = postings;
+      this.state = writeState;
     }
 
     @Override
-    public TermsConsumer addField(FieldInfo field) {
-      if (field.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
-        throw new UnsupportedOperationException("this codec cannot index offsets");
-      }
-      RAMField ramField = new RAMField(field.name, field);
-      postings.fieldToTerms.put(field.name, ramField);
-      termsConsumer.reset(ramField);
-      return termsConsumer;
-    }
+    public void write(Fields fields) throws IOException {
+      for(String field : fields) {
 
-    @Override
-    public void close() {
-      // TODO: finalize stuff
+        Terms terms = fields.terms(field);
+        if (terms == null) {
+          continue;
+        }
+
+        TermsEnum termsEnum = terms.iterator(null);
+
+        FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+        if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
+          throw new UnsupportedOperationException("this codec cannot index offsets");
+        }
+
+        RAMField ramField = new RAMField(field, fieldInfo);
+        postings.fieldToTerms.put(field, ramField);
+        termsConsumer.reset(ramField);
+
+        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;
+          }
+          RAMPostingsWriterImpl postingsWriter = termsConsumer.startTerm(term);
+
+          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;
+            }
+
+            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;
+                }
+                postingsWriter.addPosition(pos, payload, startOffset, endOffset);
+              }
+            }
+
+            postingsWriter.finishDoc();
+          }
+          termsConsumer.finishTerm(term, new TermStats(docFreq, totalTermFreq));
+          sumDocFreq += docFreq;
+          sumTotalTermFreq += totalTermFreq;
+        }
+
+        termsConsumer.finish(sumTotalTermFreq, sumDocFreq, docsSeen.cardinality());
+      }
     }
   }
 
-  private static class RAMTermsConsumer extends TermsConsumer {
+  private static class RAMTermsConsumer {
     private RAMField field;
     private final RAMPostingsWriterImpl postingsWriter = new RAMPostingsWriterImpl();
     RAMTerm current;
@@ -244,15 +341,13 @@ public final class RAMOnlyPostingsFormat
       this.field = field;
     }
       
-    @Override
-    public PostingsConsumer startTerm(BytesRef text) {
+    public RAMPostingsWriterImpl startTerm(BytesRef text) {
       final String term = text.utf8ToString();
       current = new RAMTerm(term);
       postingsWriter.reset(current);
       return postingsWriter;
     }
 
-    @Override
     public void finishTerm(BytesRef text, TermStats stats) {
       assert stats.docFreq > 0;
       assert stats.docFreq == current.docs.size();
@@ -260,7 +355,6 @@ public final class RAMOnlyPostingsFormat
       field.termToDocs.put(current.term, current);
     }
 
-    @Override
     public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) {
       field.sumTotalTermFreq = sumTotalTermFreq;
       field.sumDocFreq = sumDocFreq;
@@ -268,7 +362,7 @@ public final class RAMOnlyPostingsFormat
     }
   }
 
-  static class RAMPostingsWriterImpl extends PostingsConsumer {
+  static class RAMPostingsWriterImpl {
     private RAMTerm term;
     private RAMDoc current;
     private int posUpto = 0;
@@ -277,14 +371,12 @@ public final class RAMOnlyPostingsFormat
       this.term = term;
     }
 
-    @Override
     public void startDoc(int docID, int freq) {
       current = new RAMDoc(docID, freq);
       term.docs.add(current);
       posUpto = 0;
     }
 
-    @Override
     public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) {
       assert startOffset == -1;
       assert endOffset == -1;
@@ -299,7 +391,6 @@ public final class RAMOnlyPostingsFormat
       posUpto++;
     }
 
-    @Override
     public void finishDoc() {
       assert posUpto == current.positions.length;
     }

Modified: lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1531949&r1=1531948&r2=1531949&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/trunk/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Mon Oct 14 15:55:57 2013
@@ -543,7 +543,7 @@ public abstract class BasePostingsFormat
   
     @Override
     public boolean hasPayloads() {
-      return fieldInfo.hasPayloads();
+      return allowPayloads && fieldInfo.hasPayloads();
     }
   }
 
@@ -633,15 +633,12 @@ public abstract class BasePostingsFormat
         throw new IllegalArgumentException("liveDocs must be null");
       }
       if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-        System.out.println("no: max");
         return null;
       }
       if ((flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
-        System.out.println("no: offsets");
         return null;
       }
       if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) {
-        System.out.println("no: payloads");
         return null;
       }
       return getSeedPostings(current.getKey().utf8ToString(), current.getValue(), false, maxAllowed, allowPayloads);