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 2014/01/29 17:21:50 UTC

svn commit: r1562497 [2/3] - in /lucene/dev/branches/branch_4x: ./ dev-tools/ lucene/ lucene/analysis/ lucene/benchmark/ lucene/classification/ lucene/codecs/ lucene/codecs/src/java/org/apache/lucene/codecs/appending/ lucene/codecs/src/java/org/apache/...

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsReader.java Wed Jan 29 16:21:48 2014
@@ -162,6 +162,7 @@ public class BlockTreeTermsReader extend
         final long sumTotalTermFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY ? -1 : in.readVLong();
         final long sumDocFreq = in.readVLong();
         final int docCount = in.readVInt();
+        final int longsSize = version >= BlockTreeTermsWriter.TERMS_VERSION_META_ARRAY ? in.readVInt() : 0;
         if (docCount < 0 || docCount > info.getDocCount()) { // #docs with field must be <= #docs
           throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + info.getDocCount() + " (resource=" + in + ")");
         }
@@ -172,7 +173,7 @@ public class BlockTreeTermsReader extend
           throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq + " (resource=" + in + ")");
         }
         final long indexStartFP = indexDivisor != -1 ? indexIn.readVLong() : 0;
-        FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, indexIn));
+        FieldReader previous = fields.put(fieldInfo.name, new FieldReader(fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount, indexStartFP, longsSize, indexIn));
         if (previous != null) {
           throw new CorruptIndexException("duplicate field: " + fieldInfo.name + " (resource=" + in + ")");
         }
@@ -454,11 +455,12 @@ public class BlockTreeTermsReader extend
     final long indexStartFP;
     final long rootBlockFP;
     final BytesRef rootCode;
-    private final FST<BytesRef> index;
+    final int longsSize;
 
+    private final FST<BytesRef> index;
     //private boolean DEBUG;
 
-    FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, long indexStartFP, IndexInput indexIn) throws IOException {
+    FieldReader(FieldInfo fieldInfo, long numTerms, BytesRef rootCode, long sumTotalTermFreq, long sumDocFreq, int docCount, long indexStartFP, int longsSize, IndexInput indexIn) throws IOException {
       assert numTerms > 0;
       this.fieldInfo = fieldInfo;
       //DEBUG = BlockTreeTermsReader.DEBUG && fieldInfo.name.equals("id");
@@ -468,6 +470,7 @@ public class BlockTreeTermsReader extend
       this.docCount = docCount;
       this.indexStartFP = indexStartFP;
       this.rootCode = rootCode;
+      this.longsSize = longsSize;
       // if (DEBUG) {
       //   System.out.println("BTTR: seg=" + segment + " field=" + fieldInfo.name + " rootBlockCode=" + rootCode + " divisor=" + indexDivisor);
       // }
@@ -628,6 +631,12 @@ public class BlockTreeTermsReader extend
         FST.Arc<BytesRef> arc;
 
         final BlockTermState termState;
+  
+        // metadata buffer, holding monotonic values
+        public long[] longs;
+        // metadata buffer, holding general values
+        public byte[] bytes;
+        ByteArrayDataInput bytesReader;
 
         // Cumulative output so far
         BytesRef outputPrefix;
@@ -637,8 +646,9 @@ public class BlockTreeTermsReader extend
 
         public Frame(int ord) throws IOException {
           this.ord = ord;
-          termState = postingsReader.newTermState();
-          termState.totalTermFreq = -1;
+          this.termState = postingsReader.newTermState();
+          this.termState.totalTermFreq = -1;
+          this.longs = new long[longsSize];
         }
 
         void loadNextFloorBlock() throws IOException {
@@ -736,8 +746,17 @@ public class BlockTreeTermsReader extend
 
           termState.termBlockOrd = 0;
           nextEnt = 0;
-          
-          postingsReader.readTermsBlock(in, fieldInfo, termState);
+         
+          // metadata
+          numBytes = in.readVInt();
+          if (bytes == null) {
+            bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+            bytesReader = new ByteArrayDataInput();
+          } else if (bytes.length < numBytes) {
+            bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(bytes, 0, numBytes);
+          bytesReader.reset(bytes, 0, numBytes);
 
           if (!isLastInFloor) {
             // Sub-blocks of a single floor block are always
@@ -790,12 +809,9 @@ public class BlockTreeTermsReader extend
 
           // lazily catch up on metadata decode:
           final int limit = getTermBlockOrd();
+          boolean absolute = metaDataUpto == 0;
           assert limit > 0;
 
-          // We must set/incr state.termCount because
-          // postings impl can look at this
-          termState.termBlockOrd = metaDataUpto;
-      
           // TODO: better API would be "jump straight to term=N"???
           while (metaDataUpto < limit) {
 
@@ -807,17 +823,24 @@ public class BlockTreeTermsReader extend
 
             // TODO: if docFreq were bulk decoded we could
             // just skipN here:
+
+            // stats
             termState.docFreq = statsReader.readVInt();
             //if (DEBUG) System.out.println("    dF=" + state.docFreq);
             if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
               termState.totalTermFreq = termState.docFreq + statsReader.readVLong();
               //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
             }
+            // metadata 
+            for (int i = 0; i < longsSize; i++) {
+              longs[i] = bytesReader.readVLong();
+            }
+            postingsReader.decodeTerm(longs, bytesReader, fieldInfo, termState, absolute);
 
-            postingsReader.nextTerm(fieldInfo, termState);
             metaDataUpto++;
-            termState.termBlockOrd++;
+            absolute = false;
           }
+          termState.termBlockOrd = metaDataUpto;
         }
       }
 
@@ -1723,6 +1746,7 @@ public class BlockTreeTermsReader extend
             if (arc.output != NO_OUTPUT) {
               output = fstOutputs.add(output, arc.output);
             }
+
             // if (DEBUG) {
             //   System.out.println("    index: follow label=" + toHex(target.bytes[target.offset + targetUpto]&0xff) + " arc.output=" + arc.output + " arc.nfo=" + arc.nextFinalOutput);
             // }
@@ -2306,10 +2330,17 @@ public class BlockTreeTermsReader extend
 
         final BlockTermState state;
 
+        // metadata buffer, holding monotonic values
+        public long[] longs;
+        // metadata buffer, holding general values
+        public byte[] bytes;
+        ByteArrayDataInput bytesReader;
+
         public Frame(int ord) throws IOException {
           this.ord = ord;
-          state = postingsReader.newTermState();
-          state.totalTermFreq = -1;
+          this.state = postingsReader.newTermState();
+          this.state.totalTermFreq = -1;
+          this.longs = new long[longsSize];
         }
 
         public void setFloorData(ByteArrayDataInput in, BytesRef source) {
@@ -2407,7 +2438,17 @@ public class BlockTreeTermsReader extend
 
           // TODO: we could skip this if !hasTerms; but
           // that's rare so won't help much
-          postingsReader.readTermsBlock(in, fieldInfo, state);
+          // metadata
+          numBytes = in.readVInt();
+          if (bytes == null) {
+            bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+            bytesReader = new ByteArrayDataInput();
+          } else if (bytes.length < numBytes) {
+            bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
+          }
+          in.readBytes(bytes, 0, numBytes);
+          bytesReader.reset(bytes, 0, numBytes);
+
 
           // Sub-blocks of a single floor block are always
           // written one after another -- tail recurse:
@@ -2591,12 +2632,9 @@ public class BlockTreeTermsReader extend
 
           // lazily catch up on metadata decode:
           final int limit = getTermBlockOrd();
+          boolean absolute = metaDataUpto == 0;
           assert limit > 0;
 
-          // We must set/incr state.termCount because
-          // postings impl can look at this
-          state.termBlockOrd = metaDataUpto;
-      
           // TODO: better API would be "jump straight to term=N"???
           while (metaDataUpto < limit) {
 
@@ -2608,17 +2646,24 @@ public class BlockTreeTermsReader extend
 
             // TODO: if docFreq were bulk decoded we could
             // just skipN here:
+
+            // stats
             state.docFreq = statsReader.readVInt();
             //if (DEBUG) System.out.println("    dF=" + state.docFreq);
             if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
               state.totalTermFreq = state.docFreq + statsReader.readVLong();
               //if (DEBUG) System.out.println("    totTF=" + state.totalTermFreq);
             }
+            // metadata 
+            for (int i = 0; i < longsSize; i++) {
+              longs[i] = bytesReader.readVLong();
+            }
+            postingsReader.decodeTerm(longs, bytesReader, fieldInfo, state, absolute);
 
-            postingsReader.nextTerm(fieldInfo, state);
             metaDataUpto++;
-            state.termBlockOrd++;
+            absolute = false;
           }
+          state.termBlockOrd = metaDataUpto;
         }
 
         // Used only by assert

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/BlockTreeTermsWriter.java Wed Jan 29 16:21:48 2014
@@ -104,15 +104,14 @@ import org.apache.lucene.util.packed.Pac
  * and decoding the Postings Metadata and Term Metadata sections.</p>
  *
  * <ul>
- * <!-- TODO: expand on this, its not really correct and doesnt explain sub-blocks etc -->
- *    <li>TermsDict (.tim) --&gt; Header, <i>Postings Metadata</i>, Block<sup>NumBlocks</sup>,
+ *    <li>TermsDict (.tim) --&gt; Header, <i>PostingsHeader</i>, NodeBlock<sup>NumBlocks</sup>,
  *                               FieldSummary, DirOffset</li>
- *    <li>Block --&gt; SuffixBlock, StatsBlock, MetadataBlock</li>
- *    <li>SuffixBlock --&gt; EntryCount, SuffixLength, Byte<sup>SuffixLength</sup></li>
- *    <li>StatsBlock --&gt; StatsLength, &lt;DocFreq, TotalTermFreq&gt;<sup>EntryCount</sup></li>
- *    <li>MetadataBlock --&gt; MetaLength, &lt;<i>Term Metadata</i>&gt;<sup>EntryCount</sup></li>
+ *    <li>NodeBlock --&gt; (OuterNode | InnerNode)</li>
+ *    <li>OuterNode --&gt; EntryCount, SuffixLength, Byte<sup>SuffixLength</sup>, StatsLength, &lt; TermStats &gt;<sup>EntryCount</sup>, MetaLength, &lt;<i>TermMetadata</i>&gt;<sup>EntryCount</sup></li>
+ *    <li>InnerNode --&gt; EntryCount, SuffixLength[,Sub?], Byte<sup>SuffixLength</sup>, StatsLength, &lt; TermStats ? &gt;<sup>EntryCount</sup>, MetaLength, &lt;<i>TermMetadata ? </i>&gt;<sup>EntryCount</sup></li>
+ *    <li>TermStats --&gt; DocFreq, TotalTermFreq </li>
  *    <li>FieldSummary --&gt; NumFields, &lt;FieldNumber, NumTerms, RootCodeLength, Byte<sup>RootCodeLength</sup>,
- *                            SumDocFreq, DocCount&gt;<sup>NumFields</sup></li>
+ *                            SumTotalTermFreq?, SumDocFreq, DocCount&gt;<sup>NumFields</sup></li>
  *    <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
  *    <li>DirOffset --&gt; {@link DataOutput#writeLong Uint64}</li>
  *    <li>EntryCount,SuffixLength,StatsLength,DocFreq,MetaLength,NumFields,
@@ -134,9 +133,11 @@ import org.apache.lucene.util.packed.Pac
  *    <li>SumDocFreq is the total number of postings, the number of term-document pairs across
  *        the entire field.</li>
  *    <li>DocCount is the number of documents that have at least one posting for this field.</li>
- *    <li>PostingsMetadata and TermMetadata are plugged into by the specific postings implementation:
+ *    <li>PostingsHeader and TermMetadata are plugged into by the specific postings implementation:
  *        these contain arbitrary per-file data (such as parameters or versioning information) 
- *        and per-term data (such as pointers to inverted files).
+ *        and per-term data (such as pointers to inverted files).</li>
+ *    <li>For inner nodes of the tree, every entry will steal one bit to mark whether it points
+ *        to child nodes(sub-block). If so, the corresponding TermStats and TermMetaData are omitted </li>
  * </ul>
  * <a name="Termindex" id="Termindex"></a>
  * <h3>Term Index</h3>
@@ -204,8 +205,11 @@ public class BlockTreeTermsWriter extend
   /** Append-only */
   public static final int TERMS_VERSION_APPEND_ONLY = 1;
 
+  /** Meta data as array */
+  public static final int TERMS_VERSION_META_ARRAY = 2;
+
   /** Current terms format. */
-  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_APPEND_ONLY;
+  public static final int TERMS_VERSION_CURRENT = TERMS_VERSION_META_ARRAY;
 
   /** Extension of terms index file */
   static final String TERMS_INDEX_EXTENSION = "tip";
@@ -217,8 +221,11 @@ public class BlockTreeTermsWriter extend
   /** Append-only */
   public static final int TERMS_INDEX_VERSION_APPEND_ONLY = 1;
 
+  /** Meta data as array */
+  public static final int TERMS_INDEX_VERSION_META_ARRAY = 2;
+
   /** Current index format. */
-  public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_APPEND_ONLY;
+  public static final int TERMS_INDEX_VERSION_CURRENT = TERMS_INDEX_VERSION_META_ARRAY;
 
   private final IndexOutput out;
   private final IndexOutput indexOut;
@@ -237,8 +244,9 @@ public class BlockTreeTermsWriter extend
     public final long sumTotalTermFreq;
     public final long sumDocFreq;
     public final int docCount;
+    private final int longsSize;
 
-    public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount) {
+    public FieldMetaData(FieldInfo fieldInfo, BytesRef rootCode, long numTerms, long indexStartFP, long sumTotalTermFreq, long sumDocFreq, int docCount, int longsSize) {
       assert numTerms > 0;
       this.fieldInfo = fieldInfo;
       assert rootCode != null: "field=" + fieldInfo.name + " numTerms=" + numTerms;
@@ -248,6 +256,7 @@ public class BlockTreeTermsWriter extend
       this.sumTotalTermFreq = sumTotalTermFreq;
       this.sumDocFreq = sumDocFreq;
       this.docCount = docCount;
+      this.longsSize = longsSize;
     }
   }
 
@@ -300,7 +309,7 @@ public class BlockTreeTermsWriter extend
 
       // System.out.println("BTW.init seg=" + state.segmentName);
 
-      postingsWriter.start(out);                          // have consumer write its format/header
+      postingsWriter.init(out);                          // have consumer write its format/header
       success = true;
     } finally {
       if (!success) {
@@ -354,12 +363,13 @@ public class BlockTreeTermsWriter extend
 
   private static final class PendingTerm extends PendingEntry {
     public final BytesRef term;
-    public final TermStats stats;
+    // stats + metadata
+    public final BlockTermState state;
 
-    public PendingTerm(BytesRef term, TermStats stats) {
+    public PendingTerm(BytesRef term, BlockTermState state) {
       super(true);
       this.term = term;
-      this.stats = stats;
+      this.state = state;
     }
 
     @Override
@@ -480,6 +490,7 @@ public class BlockTreeTermsWriter extend
 
   class TermsWriter extends TermsConsumer {
     private final FieldInfo fieldInfo;
+    private final int longsSize;
     private long numTerms;
     long sumTotalTermFreq;
     long sumDocFreq;
@@ -839,11 +850,16 @@ public class BlockTreeTermsWriter extend
       final List<FST<BytesRef>> subIndices;
 
       int termCount;
+
+      long[] longs = new long[longsSize];
+      boolean absolute = true;
+
       if (isLeafBlock) {
         subIndices = null;
         for (PendingEntry ent : slice) {
           assert ent.isTerm;
           PendingTerm term = (PendingTerm) ent;
+          BlockTermState state = term.state;
           final int suffix = term.term.length - prefixLength;
           // if (DEBUG) {
           //   BytesRef suffixBytes = new BytesRef(suffix);
@@ -852,15 +868,25 @@ public class BlockTreeTermsWriter extend
           //   System.out.println("    write term suffix=" + suffixBytes);
           // }
           // For leaf block we write suffix straight
-          bytesWriter.writeVInt(suffix);
-          bytesWriter.writeBytes(term.term.bytes, prefixLength, suffix);
+          suffixWriter.writeVInt(suffix);
+          suffixWriter.writeBytes(term.term.bytes, prefixLength, suffix);
 
           // Write term stats, to separate byte[] blob:
-          bytesWriter2.writeVInt(term.stats.docFreq);
+          statsWriter.writeVInt(state.docFreq);
           if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
-            assert term.stats.totalTermFreq >= term.stats.docFreq: term.stats.totalTermFreq + " vs " + term.stats.docFreq;
-            bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
+            assert state.totalTermFreq >= state.docFreq: state.totalTermFreq + " vs " + state.docFreq;
+            statsWriter.writeVLong(state.totalTermFreq - state.docFreq);
+          }
+
+          // Write term meta data
+          postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
+          for (int pos = 0; pos < longsSize; pos++) {
+            assert longs[pos] >= 0;
+            metaWriter.writeVLong(longs[pos]);
           }
+          bytesWriter.writeTo(metaWriter);
+          bytesWriter.reset();
+          absolute = false;
         }
         termCount = length;
       } else {
@@ -869,6 +895,7 @@ public class BlockTreeTermsWriter extend
         for (PendingEntry ent : slice) {
           if (ent.isTerm) {
             PendingTerm term = (PendingTerm) ent;
+            BlockTermState state = term.state;
             final int suffix = term.term.length - prefixLength;
             // if (DEBUG) {
             //   BytesRef suffixBytes = new BytesRef(suffix);
@@ -878,16 +905,34 @@ public class BlockTreeTermsWriter extend
             // }
             // For non-leaf block we borrow 1 bit to record
             // if entry is term or sub-block
-            bytesWriter.writeVInt(suffix<<1);
-            bytesWriter.writeBytes(term.term.bytes, prefixLength, suffix);
+            suffixWriter.writeVInt(suffix<<1);
+            suffixWriter.writeBytes(term.term.bytes, prefixLength, suffix);
 
             // Write term stats, to separate byte[] blob:
-            bytesWriter2.writeVInt(term.stats.docFreq);
+            statsWriter.writeVInt(state.docFreq);
             if (fieldInfo.getIndexOptions() != IndexOptions.DOCS_ONLY) {
-              assert term.stats.totalTermFreq >= term.stats.docFreq;
-              bytesWriter2.writeVLong(term.stats.totalTermFreq - term.stats.docFreq);
+              assert state.totalTermFreq >= state.docFreq;
+              statsWriter.writeVLong(state.totalTermFreq - state.docFreq);
             }
 
+            // TODO: now that terms dict "sees" these longs,
+            // we can explore better column-stride encodings
+            // to encode all long[0]s for this block at
+            // once, all long[1]s, etc., e.g. using
+            // Simple64.  Alternatively, we could interleave
+            // stats + meta ... no reason to have them
+            // separate anymore:
+
+            // Write term meta data
+            postingsWriter.encodeTerm(longs, bytesWriter, fieldInfo, state, absolute);
+            for (int pos = 0; pos < longsSize; pos++) {
+              assert longs[pos] >= 0;
+              metaWriter.writeVLong(longs[pos]);
+            }
+            bytesWriter.writeTo(metaWriter);
+            bytesWriter.reset();
+            absolute = false;
+
             termCount++;
           } else {
             PendingBlock block = (PendingBlock) ent;
@@ -897,8 +942,8 @@ public class BlockTreeTermsWriter extend
 
             // For non-leaf block we borrow 1 bit to record
             // if entry is term or sub-block
-            bytesWriter.writeVInt((suffix<<1)|1);
-            bytesWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
+            suffixWriter.writeVInt((suffix<<1)|1);
+            suffixWriter.writeBytes(block.prefix.bytes, prefixLength, suffix);
             assert block.fp < startFP;
 
             // if (DEBUG) {
@@ -908,7 +953,7 @@ public class BlockTreeTermsWriter extend
             //   System.out.println("    write sub-block suffix=" + toString(suffixBytes) + " subFP=" + block.fp + " subCode=" + (startFP-block.fp) + " floor=" + block.isFloor);
             // }
 
-            bytesWriter.writeVLong(startFP - block.fp);
+            suffixWriter.writeVLong(startFP - block.fp);
             subIndices.add(block.index);
           }
         }
@@ -921,17 +966,19 @@ public class BlockTreeTermsWriter extend
       // search on lookup
 
       // Write suffixes byte[] blob to terms dict output:
-      out.writeVInt((int) (bytesWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
-      bytesWriter.writeTo(out);
-      bytesWriter.reset();
+      out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
+      suffixWriter.writeTo(out);
+      suffixWriter.reset();
 
       // Write term stats byte[] blob
-      out.writeVInt((int) bytesWriter2.getFilePointer());
-      bytesWriter2.writeTo(out);
-      bytesWriter2.reset();
-
-      // Have postings writer write block
-      postingsWriter.flushTermsBlock(futureTermCount+termCount, termCount);
+      out.writeVInt((int) statsWriter.getFilePointer());
+      statsWriter.writeTo(out);
+      statsWriter.reset();
+
+      // Write term meta data byte[] blob
+      out.writeVInt((int) metaWriter.getFilePointer());
+      metaWriter.writeTo(out);
+      metaWriter.reset();
 
       // Remove slice replaced by block:
       slice.clear();
@@ -967,7 +1014,7 @@ public class BlockTreeTermsWriter extend
                                          PackedInts.COMPACT,
                                          true, 15);
 
-      postingsWriter.setField(fieldInfo);
+      this.longsSize = postingsWriter.setField(fieldInfo);
     }
     
     @Override
@@ -998,8 +1045,13 @@ public class BlockTreeTermsWriter extend
       //if (DEBUG) System.out.println("BTTW.finishTerm term=" + fieldInfo.name + ":" + toString(text) + " seg=" + segment + " df=" + stats.docFreq);
 
       blockBuilder.add(Util.toIntsRef(text, scratchIntsRef), noOutputs.getNoOutput());
-      pending.add(new PendingTerm(BytesRef.deepCopyOf(text), stats));
-      postingsWriter.finishTerm(stats);
+      BlockTermState state = postingsWriter.newTermState();
+      state.docFreq = stats.docFreq;
+      state.totalTermFreq = stats.totalTermFreq;
+      postingsWriter.finishTerm(state);
+
+      PendingTerm term = new PendingTerm(BytesRef.deepCopyOf(text), state);
+      pending.add(term);
       numTerms++;
     }
 
@@ -1038,7 +1090,8 @@ public class BlockTreeTermsWriter extend
                                      indexStartFP,
                                      sumTotalTermFreq,
                                      sumDocFreq,
-                                     docCount));
+                                     docCount,
+                                     longsSize));
       } else {
         assert sumTotalTermFreq == 0 || fieldInfo.getIndexOptions() == IndexOptions.DOCS_ONLY && sumTotalTermFreq == -1;
         assert sumDocFreq == 0;
@@ -1046,8 +1099,10 @@ public class BlockTreeTermsWriter extend
       }
     }
 
+    private final RAMOutputStream suffixWriter = new RAMOutputStream();
+    private final RAMOutputStream statsWriter = new RAMOutputStream();
+    private final RAMOutputStream metaWriter = new RAMOutputStream();
     private final RAMOutputStream bytesWriter = new RAMOutputStream();
-    private final RAMOutputStream bytesWriter2 = new RAMOutputStream();
   }
 
   @Override
@@ -1072,6 +1127,9 @@ public class BlockTreeTermsWriter extend
         }
         out.writeVLong(field.sumDocFreq);
         out.writeVInt(field.docCount);
+        if (TERMS_VERSION_CURRENT >= TERMS_VERSION_META_ARRAY) {
+          out.writeVInt(field.longsSize);
+        }
         indexOut.writeVLong(field.indexStartFP);
       }
       writeTrailer(out, dirStart);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java Wed Jan 29 16:21:48 2014
@@ -24,6 +24,7 @@ import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.util.Bits;
 
 /** The core terms dictionaries (BlockTermsReader,
@@ -54,8 +55,10 @@ public abstract class PostingsReaderBase
   /** Return a newly created empty TermState */
   public abstract BlockTermState newTermState() throws IOException;
 
-  /** Actually decode metadata for next term */
-  public abstract void nextTerm(FieldInfo fieldInfo, BlockTermState state) throws IOException;
+  /** Actually decode metadata for next term 
+   *  @see PostingsWriterBase#encodeTerm 
+   */
+  public abstract void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
@@ -71,9 +74,4 @@ public abstract class PostingsReaderBase
   
   @Override
   public abstract void close() throws IOException;
-
-  /** Reads data for all terms in the next block; this
-   *  method should merely load the byte[] blob but not
-   *  decode, which is done in {@link #nextTerm}. */
-  public abstract void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState termState) throws IOException;
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java Wed Jan 29 16:21:48 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs;
 import java.io.IOException;
 import java.io.Closeable;
 
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.index.FieldInfo;
 
@@ -48,25 +49,42 @@ public abstract class PostingsWriterBase
   /** Called once after startup, before any terms have been
    *  added.  Implementations typically write a header to
    *  the provided {@code termsOut}. */
-  public abstract void start(IndexOutput termsOut) throws IOException;
+  public abstract void init(IndexOutput termsOut) throws IOException;
+
+  /** Return a newly created empty TermState */
+  public abstract BlockTermState newTermState() throws IOException;
 
   /** Start a new term.  Note that a matching call to {@link
-   *  #finishTerm(TermStats)} is done, only if the term has at least one
+   *  #finishTerm(BlockTermState)} is done, only if the term has at least one
    *  document. */
   public abstract void startTerm() throws IOException;
 
-  /** Flush count terms starting at start "backwards", as a
-   *  block. start is a negative offset from the end of the
-   *  terms stack, ie bigger start means further back in
-   *  the stack. */
-  public abstract void flushTermsBlock(int start, int count) throws IOException;
-
   /** Finishes the current term.  The provided {@link
-   *  TermStats} contains the term's summary statistics. */
-  public abstract void finishTerm(TermStats stats) throws IOException;
-
-  /** Called when the writing switches to another field. */
-  public abstract void setField(FieldInfo fieldInfo);
+   *  BlockTermState} contains the term's summary statistics, 
+   *  and will holds metadata from PBF when returned */
+  public abstract void finishTerm(BlockTermState state) throws IOException;
+
+  /**
+   * Encode metadata as long[] and byte[]. {@code absolute} controls whether 
+   * current term is delta encoded according to latest term. 
+   * Usually elements in {@code longs} are file pointers, so each one always 
+   * increases when a new term is consumed. {@code out} is used to write generic
+   * bytes, which are not monotonic.
+   *
+   * NOTE: sometimes long[] might contain "don't care" values that are unused, e.g. 
+   * the pointer to postings list may not be defined for some terms but is defined
+   * for others, if it is designed to inline  some postings data in term dictionary.
+   * In this case, the postings writer should always use the last value, so that each
+   * element in metadata long[] remains monotonic.
+   */
+  public abstract void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState state, boolean absolute) throws IOException;
+
+  /** 
+   * Sets the current field for writing, and returns the
+   * fixed length of long[] metadata (which is fixed per
+   * field), called when the writing switches to another field. */
+  // TODO: better name?
+  public abstract int setField(FieldInfo fieldInfo);
 
   @Override
   public abstract void close() throws IOException;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsReader.java Wed Jan 29 16:21:48 2014
@@ -32,6 +32,7 @@ import org.apache.lucene.index.IndexFile
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -121,11 +122,6 @@ public class Lucene40PostingsReader exte
     long proxOffset;
     long skipOffset;
 
-    // Only used by the "primary" TermState -- clones don't
-    // copy this (basically they are "transient"):
-    ByteArrayDataInput bytesReader;  // TODO: should this NOT be in the TermState...?
-    byte[] bytes;
-
     @Override
     public StandardTermState clone() {
       StandardTermState other = new StandardTermState();
@@ -140,11 +136,6 @@ public class Lucene40PostingsReader exte
       freqOffset = other.freqOffset;
       proxOffset = other.proxOffset;
       skipOffset = other.skipOffset;
-
-      // Do not copy bytes, bytesReader (else TermState is
-      // very heavy, ie drags around the entire block's
-      // byte[]).  On seek back, if next() is in fact used
-      // (rare!), they will be re-read from disk.
     }
 
     @Override
@@ -171,38 +162,18 @@ public class Lucene40PostingsReader exte
     }
   }
 
-  /* Reads but does not decode the byte[] blob holding
-     metadata for the current terms block */
   @Override
-  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
-    final StandardTermState termState = (StandardTermState) _termState;
-
-    final int len = termsIn.readVInt();
-
-    // if (DEBUG) System.out.println("  SPR.readTermsBlock bytes=" + len + " ts=" + _termState);
-    if (termState.bytes == null) {
-      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
-      termState.bytesReader = new ByteArrayDataInput();
-    } else if (termState.bytes.length < len) {
-      termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
-    }
-
-    termsIn.readBytes(termState.bytes, 0, len);
-    termState.bytesReader.reset(termState.bytes, 0, len);
-  }
-
-  @Override
-  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
+  public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
     throws IOException {
     final StandardTermState termState = (StandardTermState) _termState;
     // if (DEBUG) System.out.println("SPR: nextTerm seg=" + segment + " tbOrd=" + termState.termBlockOrd + " bytesReader.fp=" + termState.bytesReader.getPosition());
     final boolean isFirstTerm = termState.termBlockOrd == 0;
-
-    if (isFirstTerm) {
-      termState.freqOffset = termState.bytesReader.readVLong();
-    } else {
-      termState.freqOffset += termState.bytesReader.readVLong();
+    if (absolute) {
+      termState.freqOffset = 0;
+      termState.proxOffset = 0;
     }
+
+    termState.freqOffset += in.readVLong();
     /*
     if (DEBUG) {
       System.out.println("  dF=" + termState.docFreq);
@@ -212,7 +183,7 @@ public class Lucene40PostingsReader exte
     assert termState.freqOffset < freqIn.length();
 
     if (termState.docFreq >= skipMinimum) {
-      termState.skipOffset = termState.bytesReader.readVLong();
+      termState.skipOffset = in.readVLong();
       // if (DEBUG) System.out.println("  skipOffset=" + termState.skipOffset + " vs freqIn.length=" + freqIn.length());
       assert termState.freqOffset + termState.skipOffset < freqIn.length();
     } else {
@@ -220,11 +191,7 @@ public class Lucene40PostingsReader exte
     }
 
     if (fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-      if (isFirstTerm) {
-        termState.proxOffset = termState.bytesReader.readVLong();
-      } else {
-        termState.proxOffset += termState.bytesReader.readVLong();
-      }
+      termState.proxOffset += in.readVLong();
       // if (DEBUG) System.out.println("  proxFP=" + termState.proxOffset);
     }
   }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsFormat.java Wed Jan 29 16:21:48 2014
@@ -123,11 +123,11 @@ import org.apache.lucene.util.packed.Pac
  *
  * <p>NOTE: The term dictionary can plug into different postings implementations:
  * the postings writer/reader are actually responsible for encoding 
- * and decoding the Postings Metadata and Term Metadata sections described here:</p>
+ * and decoding the PostingsHeader and TermMetadata sections described here:</p>
  *
  * <ul>
- *   <li>Postings Metadata --&gt; Header, PackedBlockSize</li>
- *   <li>Term Metadata --&gt; (DocFPDelta|SingletonDocID), PosFPDelta?, PosVIntBlockFPDelta?, PayFPDelta?, 
+ *   <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>

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsReader.java Wed Jan 29 16:21:48 2014
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
 import static org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat.BLOCK_SIZE;
 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.Lucene41PostingsWriter.IntBlockTermState;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -34,7 +35,6 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.TermState;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
@@ -59,6 +59,7 @@ public final class Lucene41PostingsReade
   private final IndexInput payIn;
 
   private final ForUtil forUtil;
+  private int version;
 
   // public static boolean DEBUG = false;
 
@@ -71,27 +72,21 @@ public final class Lucene41PostingsReade
     try {
       docIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.DOC_EXTENSION),
                             ioContext);
-      CodecUtil.checkHeader(docIn,
+      version = CodecUtil.checkHeader(docIn,
                             Lucene41PostingsWriter.DOC_CODEC,
-                            Lucene41PostingsWriter.VERSION_CURRENT,
+                            Lucene41PostingsWriter.VERSION_START,
                             Lucene41PostingsWriter.VERSION_CURRENT);
       forUtil = new ForUtil(docIn);
 
       if (fieldInfos.hasProx()) {
         posIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.POS_EXTENSION),
                               ioContext);
-        CodecUtil.checkHeader(posIn,
-                              Lucene41PostingsWriter.POS_CODEC,
-                              Lucene41PostingsWriter.VERSION_CURRENT,
-                              Lucene41PostingsWriter.VERSION_CURRENT);
+        CodecUtil.checkHeader(posIn, Lucene41PostingsWriter.POS_CODEC, version, version);
 
         if (fieldInfos.hasPayloads() || fieldInfos.hasOffsets()) {
           payIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, segmentSuffix, Lucene41PostingsFormat.PAY_EXTENSION),
                                 ioContext);
-          CodecUtil.checkHeader(payIn,
-                                Lucene41PostingsWriter.PAY_CODEC,
-                                Lucene41PostingsWriter.VERSION_CURRENT,
-                                Lucene41PostingsWriter.VERSION_CURRENT);
+          CodecUtil.checkHeader(payIn, Lucene41PostingsWriter.PAY_CODEC, version, version);
         }
       }
 
@@ -111,7 +106,7 @@ public final class Lucene41PostingsReade
     // Make sure we are talking to the matching postings writer
     CodecUtil.checkHeader(termsIn,
                           Lucene41PostingsWriter.TERMS_CODEC,
-                          Lucene41PostingsWriter.VERSION_CURRENT,
+                          Lucene41PostingsWriter.VERSION_START,
                           Lucene41PostingsWriter.VERSION_CURRENT);
     final int indexBlockSize = termsIn.readVInt();
     if (indexBlockSize != BLOCK_SIZE) {
@@ -141,54 +136,8 @@ public final class Lucene41PostingsReade
     }
   }
 
-  // Must keep final because we do non-standard clone
-  private final static class IntBlockTermState extends BlockTermState {
-    long docStartFP;
-    long posStartFP;
-    long payStartFP;
-    long skipOffset;
-    long lastPosBlockOffset;
-    // docid when there is a single pulsed posting, otherwise -1
-    // freq is always implicitly totalTermFreq in this case.
-    int singletonDocID;
-
-    // Only used by the "primary" TermState -- clones don't
-    // copy this (basically they are "transient"):
-    ByteArrayDataInput bytesReader;  // TODO: should this NOT be in the TermState...?
-    byte[] bytes;
-
-    @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;
-
-      // Do not copy bytes, bytesReader (else TermState is
-      // very heavy, ie drags around the entire block's
-      // byte[]).  On seek back, if next() is in fact used
-      // (rare!), they will be re-read from disk.
-    }
-
-    @Override
-    public String toString() {
-      return super.toString() + " docStartFP=" + docStartFP + " posStartFP=" + posStartFP + " payStartFP=" + payStartFP + " lastPosBlockOffset=" + lastPosBlockOffset + " singletonDocID=" + singletonDocID;
-    }
-  }
-
   @Override
-  public IntBlockTermState newTermState() {
+  public BlockTermState newTermState() {
     return new IntBlockTermState();
   }
 
@@ -197,81 +146,69 @@ public final class Lucene41PostingsReade
     IOUtils.close(docIn, posIn, payIn);
   }
 
-  /* Reads but does not decode the byte[] blob holding
-     metadata for the current terms block */
-  @Override
-  public void readTermsBlock(IndexInput termsIn, FieldInfo fieldInfo, BlockTermState _termState) throws IOException {
-    final IntBlockTermState termState = (IntBlockTermState) _termState;
-
-    final int numBytes = termsIn.readVInt();
-
-    if (termState.bytes == null) {
-      termState.bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
-      termState.bytesReader = new ByteArrayDataInput();
-    } else if (termState.bytes.length < numBytes) {
-      termState.bytes = new byte[ArrayUtil.oversize(numBytes, 1)];
-    }
-
-    termsIn.readBytes(termState.bytes, 0, numBytes);
-    termState.bytesReader.reset(termState.bytes, 0, numBytes);
-  }
-
   @Override
-  public void nextTerm(FieldInfo fieldInfo, BlockTermState _termState)
+  public void decodeTerm(long[] longs, DataInput in, FieldInfo fieldInfo, BlockTermState _termState, boolean absolute)
     throws IOException {
     final IntBlockTermState termState = (IntBlockTermState) _termState;
-    final boolean isFirstTerm = termState.termBlockOrd == 0;
     final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
     final boolean fieldHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
     final boolean fieldHasPayloads = fieldInfo.hasPayloads();
 
-    final DataInput in = termState.bytesReader;
-    if (isFirstTerm) {
-      if (termState.docFreq == 1) {
-        termState.singletonDocID = in.readVInt();
-        termState.docStartFP = 0;
-      } else {
-        termState.singletonDocID = -1;
-        termState.docStartFP = in.readVLong();
+    if (absolute) {
+      termState.docStartFP = 0;
+      termState.posStartFP = 0;
+      termState.payStartFP = 0;
+    }
+    if (version < Lucene41PostingsWriter.VERSION_META_ARRAY) {  // backward compatibility
+      _decodeTerm(in, fieldInfo, termState);
+      return;
+    }
+    termState.docStartFP += longs[0];
+    if (fieldHasPositions) {
+      termState.posStartFP += longs[1];
+      if (fieldHasOffsets || fieldHasPayloads) {
+        termState.payStartFP += longs[2];
       }
-      if (fieldHasPositions) {
-        termState.posStartFP = in.readVLong();
-        if (termState.totalTermFreq > BLOCK_SIZE) {
-          termState.lastPosBlockOffset = in.readVLong();
-        } else {
-          termState.lastPosBlockOffset = -1;
-        }
-        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
-          termState.payStartFP = in.readVLong();
-        } else {
-          termState.payStartFP = -1;
-        }
+    }
+    if (termState.docFreq == 1) {
+      termState.singletonDocID = in.readVInt();
+    } else {
+      termState.singletonDocID = -1;
+    }
+    if (fieldHasPositions) {
+      if (termState.totalTermFreq > BLOCK_SIZE) {
+        termState.lastPosBlockOffset = in.readVLong();
+      } else {
+        termState.lastPosBlockOffset = -1;
       }
+    }
+    if (termState.docFreq > BLOCK_SIZE) {
+      termState.skipOffset = in.readVLong();
     } else {
-      if (termState.docFreq == 1) {
-        termState.singletonDocID = in.readVInt();
+      termState.skipOffset = -1;
+    }
+  }
+  private void _decodeTerm(DataInput in, FieldInfo fieldInfo, IntBlockTermState termState) throws IOException {
+    final boolean fieldHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
+    final boolean fieldHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+    final boolean fieldHasPayloads = fieldInfo.hasPayloads();
+    if (termState.docFreq == 1) {
+      termState.singletonDocID = in.readVInt();
+    } else {
+      termState.singletonDocID = -1;
+      termState.docStartFP += in.readVLong();
+    }
+    if (fieldHasPositions) {
+      termState.posStartFP += in.readVLong();
+      if (termState.totalTermFreq > BLOCK_SIZE) {
+        termState.lastPosBlockOffset = in.readVLong();
       } else {
-        termState.singletonDocID = -1;
-        termState.docStartFP += in.readVLong();
+        termState.lastPosBlockOffset = -1;
       }
-      if (fieldHasPositions) {
-        termState.posStartFP += in.readVLong();
-        if (termState.totalTermFreq > BLOCK_SIZE) {
-          termState.lastPosBlockOffset = in.readVLong();
-        } else {
-          termState.lastPosBlockOffset = -1;
-        }
-        if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
-          long delta = in.readVLong();
-          if (termState.payStartFP == -1) {
-            termState.payStartFP = delta;
-          } else {
-            termState.payStartFP += delta;
-          }
-        }
+      if ((fieldHasPayloads || fieldHasOffsets) && termState.totalTermFreq >= BLOCK_SIZE) {
+        termState.payStartFP += in.readVLong();
       }
     }
-
     if (termState.docFreq > BLOCK_SIZE) {
       termState.skipOffset = in.readVLong();
     } else {

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/codecs/lucene41/Lucene41PostingsWriter.java Wed Jan 29 16:21:48 2014
@@ -25,14 +25,16 @@ 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.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
 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.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
@@ -65,13 +67,15 @@ public final class Lucene41PostingsWrite
 
   // Increment version to change it
   final static int VERSION_START = 0;
-  final static int VERSION_CURRENT = VERSION_START;
+  final static int VERSION_META_ARRAY = 1;
+  final static int VERSION_CURRENT = VERSION_META_ARRAY;
 
   final IndexOutput docOut;
   final IndexOutput posOut;
   final IndexOutput payOut;
 
-  private IndexOutput termsOut;
+  final static IntBlockTermState emptyState = new IntBlockTermState();
+  IntBlockTermState lastState;
 
   // How current field indexes postings:
   private boolean fieldHasFreqs;
@@ -79,10 +83,10 @@ public final class Lucene41PostingsWrite
   private boolean fieldHasOffsets;
   private boolean fieldHasPayloads;
 
-  // Holds starting file pointers for each term:
-  private long docTermStartFP;
-  private long posTermStartFP;
-  private long payTermStartFP;
+  // Holds starting file pointers for current term:
+  private long docStartFP;
+  private long posStartFP;
+  private long payStartFP;
 
   final int[] docDeltaBuffer;
   final int[] freqBuffer;
@@ -188,36 +192,86 @@ public final class Lucene41PostingsWrite
     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 void start(IndexOutput termsOut) throws IOException {
-    this.termsOut = termsOut;
+  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 void setField(FieldInfo fieldInfo) {
+  public int setField(FieldInfo fieldInfo) {
     IndexOptions indexOptions = fieldInfo.getIndexOptions();
     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;
     fieldHasPayloads = fieldInfo.hasPayloads();
     skipWriter.setField(fieldHasPositions, fieldHasOffsets, fieldHasPayloads);
+    lastState = emptyState;
+    if (fieldHasPositions) {
+      if (fieldHasPayloads || fieldHasOffsets) {
+        return 3;  // doc + pos + pay FP
+      } else {
+        return 2;  // doc + pos FP
+      }
+    } else {
+      return 1;    // doc FP
+    }
   }
 
   @Override
   public void startTerm() {
-    docTermStartFP = docOut.getFilePointer();
+    docStartFP = docOut.getFilePointer();
     if (fieldHasPositions) {
-      posTermStartFP = posOut.getFilePointer();
+      posStartFP = posOut.getFilePointer();
       if (fieldHasPayloads || fieldHasOffsets) {
-        payTermStartFP = payOut.getFilePointer();
+        payStartFP = payOut.getFilePointer();
       }
     }
     lastDocID = 0;
     lastBlockDocID = -1;
     // if (DEBUG) {
-    //   System.out.println("FPW.startTerm startFP=" + docTermStartFP);
+    //   System.out.println("FPW.startTerm startFP=" + docStartFP);
     // }
     skipWriter.resetSkip();
   }
@@ -348,48 +402,29 @@ public final class Lucene41PostingsWrite
     }
   }
 
-  private static class PendingTerm {
-    public final long docStartFP;
-    public final long posStartFP;
-    public final long payStartFP;
-    public final long skipOffset;
-    public final long lastPosBlockOffset;
-    public final int singletonDocID;
-
-    public PendingTerm(long docStartFP, long posStartFP, long payStartFP, long skipOffset, long lastPosBlockOffset, int singletonDocID) {
-      this.docStartFP = docStartFP;
-      this.posStartFP = posStartFP;
-      this.payStartFP = payStartFP;
-      this.skipOffset = skipOffset;
-      this.lastPosBlockOffset = lastPosBlockOffset;
-      this.singletonDocID = singletonDocID;
-    }
-  }
-
-  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
-
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(TermStats stats) throws IOException {
-    assert stats.docFreq > 0;
+  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 stats.docFreq == docCount: stats.docFreq + " vs " + docCount;
+    assert state.docFreq == docCount: state.docFreq + " vs " + docCount;
 
     // if (DEBUG) {
-    //   System.out.println("FPW.finishTerm docFreq=" + stats.docFreq);
+    //   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() + " docTermStartFP=" + docTermStartFP);
+    //     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 (stats.docFreq == 1) {
+    if (state.docFreq == 1) {
       // pulse the singleton docid into the term dictionary, freq is implicitly totalTermFreq
       singletonDocID = docDeltaBuffer[0];
     } else {
@@ -414,16 +449,16 @@ public final class Lucene41PostingsWrite
     if (fieldHasPositions) {
       // if (DEBUG) {
       //   if (posBufferUpto > 0) {
-      //     System.out.println("  write pos vInt block (count=" + posBufferUpto + ") at fp=" + posOut.getFilePointer() + " posTermStartFP=" + posTermStartFP + " hasPayloads=" + fieldHasPayloads + " hasOffsets=" + fieldHasOffsets);
+      //     System.out.println("  write pos vInt block (count=" + posBufferUpto + ") at fp=" + posOut.getFilePointer() + " posStartFP=" + posStartFP + " hasPayloads=" + fieldHasPayloads + " hasOffsets=" + fieldHasOffsets);
       //   }
       // }
 
       // totalTermFreq is just total number of positions(or payloads, or offsets)
       // associated with current term.
-      assert stats.totalTermFreq != -1;
-      if (stats.totalTermFreq > BLOCK_SIZE) {
+      assert state.totalTermFreq != -1;
+      if (state.totalTermFreq > BLOCK_SIZE) {
         // record file offset for last pos in last block
-        lastPosBlockOffset = posOut.getFilePointer() - posTermStartFP;
+        lastPosBlockOffset = posOut.getFilePointer() - posStartFP;
       } else {
         lastPosBlockOffset = -1;
       }
@@ -486,7 +521,7 @@ public final class Lucene41PostingsWrite
         }
       }
       // if (DEBUG) {
-      //   System.out.println("  totalTermFreq=" + stats.totalTermFreq + " lastPosBlockOffset=" + lastPosBlockOffset);
+      //   System.out.println("  totalTermFreq=" + state.totalTermFreq + " lastPosBlockOffset=" + lastPosBlockOffset);
       // }
     } else {
       lastPosBlockOffset = -1;
@@ -494,10 +529,10 @@ public final class Lucene41PostingsWrite
 
     long skipOffset;
     if (docCount > BLOCK_SIZE) {
-      skipOffset = skipWriter.writeSkip(docOut) - docTermStartFP;
+      skipOffset = skipWriter.writeSkip(docOut) - docStartFP;
       
       // if (DEBUG) {
-      //   System.out.println("skip packet " + (docOut.getFilePointer() - (docTermStartFP + skipOffset)) + " bytes");
+      //   System.out.println("skip packet " + (docOut.getFilePointer() - (docStartFP + skipOffset)) + " bytes");
       // }
     } else {
       skipOffset = -1;
@@ -505,76 +540,46 @@ public final class Lucene41PostingsWrite
       //   System.out.println("  no skip: docCount=" + docCount);
       // }
     }
-
-    long payStartFP;
-    if (stats.totalTermFreq >= BLOCK_SIZE) {
-      payStartFP = payTermStartFP;
-    } else {
-      payStartFP = -1;
-    }
-
     // if (DEBUG) {
     //   System.out.println("  payStartFP=" + payStartFP);
     // }
-
-    pendingTerms.add(new PendingTerm(docTermStartFP, posTermStartFP, payStartFP, skipOffset, lastPosBlockOffset, singletonDocID));
+    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;
   }
-
-  private final RAMOutputStream bytesWriter = new RAMOutputStream();
-
+  
   @Override
-  public void flushTermsBlock(int start, int count) throws IOException {
-
-    if (count == 0) {
-      termsOut.writeByte((byte) 0);
-      return;
+  public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    IntBlockTermState state = (IntBlockTermState)_state;
+    if (absolute) {
+      lastState = emptyState;
     }
-
-    assert start <= pendingTerms.size();
-    assert count <= start;
-
-    final int limit = pendingTerms.size() - start + count;
-
-    long lastDocStartFP = 0;
-    long lastPosStartFP = 0;
-    long lastPayStartFP = 0;
-    for(int idx=limit-count; idx<limit; idx++) {
-      PendingTerm term = pendingTerms.get(idx);
-
-      if (term.singletonDocID == -1) {
-        bytesWriter.writeVLong(term.docStartFP - lastDocStartFP);
-        lastDocStartFP = term.docStartFP;
-      } else {
-        bytesWriter.writeVInt(term.singletonDocID);
-      }
-
-      if (fieldHasPositions) {
-        bytesWriter.writeVLong(term.posStartFP - lastPosStartFP);
-        lastPosStartFP = term.posStartFP;
-        if (term.lastPosBlockOffset != -1) {
-          bytesWriter.writeVLong(term.lastPosBlockOffset);
-        }
-        if ((fieldHasPayloads || fieldHasOffsets) && term.payStartFP != -1) {
-          bytesWriter.writeVLong(term.payStartFP - lastPayStartFP);
-          lastPayStartFP = term.payStartFP;
-        }
+    longs[0] = state.docStartFP - lastState.docStartFP;
+    if (fieldHasPositions) {
+      longs[1] = state.posStartFP - lastState.posStartFP;
+      if (fieldHasPayloads || fieldHasOffsets) {
+        longs[2] = state.payStartFP - lastState.payStartFP;
       }
-
-      if (term.skipOffset != -1) {
-        bytesWriter.writeVLong(term.skipOffset);
+    }
+    if (state.singletonDocID != -1) {
+      out.writeVInt(state.singletonDocID);
+    }
+    if (fieldHasPositions) {
+      if (state.lastPosBlockOffset != -1) {
+        out.writeVLong(state.lastPosBlockOffset);
       }
     }
-
-    termsOut.writeVInt((int) bytesWriter.getFilePointer());
-    bytesWriter.writeTo(termsOut);
-    bytesWriter.reset();
-
-    // Remove the terms we just wrote:
-    pendingTerms.subList(limit-count, limit).clear();
+    if (state.skipOffset != -1) {
+      out.writeVLong(state.skipOffset);
+    }
+    lastState = state;
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/store/RAMOutputStream.java Wed Jan 29 16:21:48 2014
@@ -51,7 +51,7 @@ public class RAMOutputStream extends Ind
   }
 
   /** Copy the current contents of this buffer to the named output. */
-  public void writeTo(IndexOutput out) throws IOException {
+  public void writeTo(DataOutput out) throws IOException {
     flush();
     final long end = file.length;
     long pos = 0;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Wed Jan 29 16:21:48 2014
@@ -345,4 +345,24 @@ public class CompiledAutomaton {
       }
     }
   }
+  
+  public String toDot() {
+    StringBuilder b = new StringBuilder("digraph CompiledAutomaton {\n");
+    b.append("  rankdir = LR;\n");
+    int initial = runAutomaton.getInitialState();
+    for (int i = 0; i < sortedTransitions.length; i++) {
+      b.append("  ").append(i);
+      if (runAutomaton.isAccept(i)) b.append(" [shape=doublecircle,label=\"\"];\n");
+      else b.append(" [shape=circle,label=\"\"];\n");
+      if (i == initial) {
+        b.append("  initial [shape=plaintext,label=\"\"];\n");
+        b.append("  initial -> ").append(i).append("\n");
+      }
+      for (int j = 0; j < sortedTransitions[i].length; j++) {
+        b.append("  ").append(i);
+        sortedTransitions[i][j].appendDot(b);
+      }
+    }
+    return b.append("}\n").toString();
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/FST.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/FST.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/FST.java Wed Jan 29 16:21:48 2014
@@ -233,16 +233,19 @@ public final class FST<T> {
       StringBuilder b = new StringBuilder();
       b.append("node=" + node);
       b.append(" target=" + target);
-      b.append(" label=" + label);
-      if (flag(BIT_LAST_ARC)) {
-        b.append(" last");
-      }
+      b.append(" label=0x" + Integer.toHexString(label));
       if (flag(BIT_FINAL_ARC)) {
         b.append(" final");
       }
+      if (flag(BIT_LAST_ARC)) {
+        b.append(" last");
+      }
       if (flag(BIT_TARGET_NEXT)) {
         b.append(" targetNext");
       }
+      if (flag(BIT_STOP_NODE)) {
+        b.append(" stop");
+      }
       if (flag(BIT_ARC_HAS_OUTPUT)) {
         b.append(" output=" + output);
       }
@@ -834,6 +837,9 @@ public final class FST<T> {
     if (emptyOutput != null) {
       arc.flags = BIT_FINAL_ARC | BIT_LAST_ARC;
       arc.nextFinalOutput = emptyOutput;
+      if (emptyOutput != NO_OUTPUT) {
+        arc.flags |= BIT_ARC_HAS_FINAL_OUTPUT;
+      }
     } else {
       arc.flags = BIT_LAST_ARC;
       arc.nextFinalOutput = NO_OUTPUT;

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Wed Jan 29 16:21:48 2014
@@ -68,7 +68,7 @@ final class NodeHash<T> {
   }
 
   // hash code for an unfrozen node.  This must be identical
-  // to the un-frozen case (below)!!
+  // to the frozen case (below)!!
   private long hash(Builder.UnCompiledNode<T> node) {
     final int PRIME = 31;
     //System.out.println("hash unfrozen");

Modified: lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Outputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Outputs.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Outputs.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/java/org/apache/lucene/util/fst/Outputs.java Wed Jan 29 16:21:48 2014
@@ -40,7 +40,7 @@ public abstract class Outputs<T> {
   // (new object per byte/char/int) if eg used during
   // analysis
 
-  /** Eg common("foo", "foobar") -> "foo" */
+  /** Eg common("foobar", "food") -> "foo" */
   public abstract T common(T output1, T output2);
 
   /** Eg subtract("foobar", "foo") -> "bar" */

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Wed Jan 29 16:21:48 2014
@@ -164,7 +164,9 @@ public class TestBackwardsCompatibility 
                                     "42.cfs",
                                     "42.nocfs",
                                     "45.cfs",
-                                    "45.nocfs"
+                                    "45.nocfs",
+                                    "461.cfs",
+                                    "461.nocfs",
   };
   
   final String[] unsupportedNames = {"19.cfs",

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterReader.java Wed Jan 29 16:21:48 2014
@@ -1024,7 +1024,13 @@ public class TestIndexWriterReader exten
     // Don't proceed if picked Codec is in the list of illegal ones.
     final String format = _TestUtil.getPostingsFormat("f");
     assumeFalse("Format: " + format + " does not support ReaderTermsIndexDivisor!",
-                (format.equals("SimpleText") || format.equals("Memory") || format.equals("Direct")));
+                (format.equals("FSTPulsing41") ||
+                 format.equals("FSTOrdPulsing41") ||
+                 format.equals("FST41") ||
+                 format.equals("FSTOrd41") ||
+                 format.equals("SimpleText") ||
+                 format.equals("Memory") ||
+                 format.equals("Direct")));
 
     Directory dir = newDirectory();
     IndexWriter w = new IndexWriter(dir, conf);

Modified: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java (original)
+++ lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/TestTermsEnum.java Wed Jan 29 16:21:48 2014
@@ -340,7 +340,6 @@ public class TestTermsEnum extends Lucen
             loc++;
           } while (loc < termsArray.length && !acceptTermsSet.contains(termsArray[loc]));
         }
-
         assertNull(te.next());
       }
     }
@@ -771,4 +770,116 @@ public class TestTermsEnum extends Lucen
     r.close();
     dir.close();
   }
+  public void testIntersectStartTerm() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMergePolicy(new LogDocMergePolicy());
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+    Document doc = new Document();
+    doc.add(newStringField("field", "abc", Field.Store.NO));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(newStringField("field", "abd", Field.Store.NO));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(newStringField("field", "acd", Field.Store.NO));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(newStringField("field", "bcd", Field.Store.NO));
+    w.addDocument(doc);
+
+    w.forceMerge(1);
+    DirectoryReader r = w.getReader();
+    w.close();
+    AtomicReader sub = getOnlySegmentReader(r);
+    Terms terms = sub.fields().terms("field");
+
+    Automaton automaton = new RegExp(".*d", RegExp.NONE).toAutomaton();
+    CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);    
+    TermsEnum te;
+    
+    // should seek to startTerm
+    te = terms.intersect(ca, new BytesRef("aad"));
+    assertEquals("abd", te.next().utf8ToString());
+    assertEquals(1, te.docs(null, null, DocsEnum.FLAG_NONE).nextDoc());
+    assertEquals("acd", te.next().utf8ToString());
+    assertEquals(2, te.docs(null, null, DocsEnum.FLAG_NONE).nextDoc());
+    assertEquals("bcd", te.next().utf8ToString());
+    assertEquals(3, te.docs(null, null, DocsEnum.FLAG_NONE).nextDoc());
+    assertNull(te.next());
+
+    // should fail to find ceil label on second arc, rewind 
+    te = terms.intersect(ca, new BytesRef("add"));
+    assertEquals("bcd", te.next().utf8ToString());
+    assertEquals(3, te.docs(null, null, DocsEnum.FLAG_NONE).nextDoc());
+    assertNull(te.next());
+
+    // should reach end
+    te = terms.intersect(ca, new BytesRef("bcd"));
+    assertNull(te.next());
+    te = terms.intersect(ca, new BytesRef("ddd"));
+    assertNull(te.next());
+
+    r.close();
+    dir.close();
+  }
+
+  public void testIntersectEmptyString() throws Exception {
+    Directory dir = newDirectory();
+    IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    iwc.setMergePolicy(new LogDocMergePolicy());
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
+    Document doc = new Document();
+    doc.add(newStringField("field", "", Field.Store.NO));
+    doc.add(newStringField("field", "abc", Field.Store.NO));
+    w.addDocument(doc);
+
+    doc = new Document();
+    // add empty string to both documents, so that singletonDocID == -1.
+    // For a FST-based term dict, we'll expect to see the first arc is 
+    // flaged with HAS_FINAL_OUTPUT
+    doc.add(newStringField("field", "abc", Field.Store.NO));
+    doc.add(newStringField("field", "", Field.Store.NO));
+    w.addDocument(doc);
+
+    w.forceMerge(1);
+    DirectoryReader r = w.getReader();
+    w.close();
+    AtomicReader sub = getOnlySegmentReader(r);
+    Terms terms = sub.fields().terms("field");
+
+    Automaton automaton = new RegExp(".*", RegExp.NONE).toAutomaton();  // accept ALL
+    CompiledAutomaton ca = new CompiledAutomaton(automaton, false, false);    
+
+    TermsEnum te = terms.intersect(ca, null);
+    DocsEnum de;
+
+    assertEquals("", te.next().utf8ToString());
+    de = te.docs(null, null, DocsEnum.FLAG_NONE);
+    assertEquals(0, de.nextDoc());
+    assertEquals(1, de.nextDoc());
+
+    assertEquals("abc", te.next().utf8ToString());
+    de = te.docs(null, null, DocsEnum.FLAG_NONE);
+    assertEquals(0, de.nextDoc());
+    assertEquals(1, de.nextDoc());
+
+    assertNull(te.next());
+
+    // pass empty string
+    te = terms.intersect(ca, new BytesRef(""));
+
+    assertEquals("abc", te.next().utf8ToString());
+    de = te.docs(null, null, DocsEnum.FLAG_NONE);
+    assertEquals(0, de.nextDoc());
+    assertEquals(1, de.nextDoc());
+
+    assertNull(te.next());
+
+    r.close();
+    dir.close();
+  }
 }

Added: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/index.461.cfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/index.461.cfs.zip?rev=1562497&view=auto
==============================================================================
Binary file - no diff available.

Added: lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/index.461.nocfs.zip
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/core/src/test/org/apache/lucene/index/index.461.nocfs.zip?rev=1562497&view=auto
==============================================================================
Binary file - no diff available.

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/appending/AppendingTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/appending/AppendingTermsWriter.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/appending/AppendingTermsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/appending/AppendingTermsWriter.java Wed Jan 29 16:21:48 2014
@@ -36,12 +36,12 @@ public class AppendingTermsWriter extend
 
   @Override
   protected void writeHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, AppendingTermsReader.APPENDING_TERMS_CODEC_NAME, AppendingTermsReader.APPENDING_TERMS_VERSION_CURRENT);
+    CodecUtil.writeHeader(out, AppendingTermsReader.APPENDING_TERMS_CODEC_NAME, BlockTreeTermsWriter.TERMS_VERSION_CURRENT);
   }
 
   @Override
   protected void writeIndexHeader(IndexOutput out) throws IOException {
-    CodecUtil.writeHeader(out, AppendingTermsReader.APPENDING_TERMS_INDEX_CODEC_NAME, AppendingTermsReader.APPENDING_TERMS_INDEX_VERSION_CURRENT);
+    CodecUtil.writeHeader(out, AppendingTermsReader.APPENDING_TERMS_INDEX_CODEC_NAME, BlockTreeTermsWriter.TERMS_INDEX_VERSION_CURRENT);
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/lucene40/Lucene40PostingsWriter.java Wed Jan 29 16:21:48 2014
@@ -24,6 +24,7 @@ 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;
@@ -33,6 +34,7 @@ import org.apache.lucene.index.FieldInfo
 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;
@@ -67,7 +69,6 @@ public final class Lucene40PostingsWrite
    */
   final int maxSkipLevels = 10;
   final int totalNumDocs;
-  IndexOutput termsOut;
 
   IndexOptions indexOptions;
   boolean storePayloads;
@@ -81,6 +82,9 @@ public final class Lucene40PostingsWrite
   int lastPosition;
   int lastOffset;
 
+  final static StandardTermState emptyState = new StandardTermState();
+  StandardTermState lastState;
+
   // private String segment;
 
   /** Creates a {@link Lucene40PostingsWriter}, with the
@@ -134,8 +138,7 @@ public final class Lucene40PostingsWrite
   }
 
   @Override
-  public void start(IndexOutput termsOut) throws IOException {
-    this.termsOut = termsOut;
+  public void init(IndexOutput termsOut) throws IOException {
     CodecUtil.writeHeader(termsOut, Lucene40PostingsReader.TERMS_CODEC, Lucene40PostingsReader.VERSION_CURRENT);
     termsOut.writeInt(skipInterval);                // write skipInterval
     termsOut.writeInt(maxSkipLevels);               // write maxSkipLevels
@@ -143,6 +146,12 @@ public final class Lucene40PostingsWrite
   }
 
   @Override
+  public BlockTermState newTermState() {
+    return new StandardTermState();
+  }
+
+
+  @Override
   public void startTerm() {
     freqStart = freqOut.getFilePointer();
     //if (DEBUG) System.out.println("SPW: startTerm freqOut.fp=" + freqStart);
@@ -159,7 +168,7 @@ public final class Lucene40PostingsWrite
   // Currently, this instance is re-used across fields, so
   // our parent calls setField whenever the field changes
   @Override
-  public void setField(FieldInfo fieldInfo) {
+  public int setField(FieldInfo fieldInfo) {
     //System.out.println("SPW: setField");
     /*
     if (BlockTreeTermsWriter.DEBUG && fieldInfo.name.equals("id")) {
@@ -173,8 +182,10 @@ public final class Lucene40PostingsWrite
     
     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);
+    return 0;
   }
 
   int lastDocID;
@@ -265,94 +276,48 @@ public final class Lucene40PostingsWrite
   public void finishDoc() {
   }
 
-  private static class PendingTerm {
-    public final long freqStart;
-    public final long proxStart;
-    public final long skipOffset;
-
-    public PendingTerm(long freqStart, long proxStart, long skipOffset) {
-      this.freqStart = freqStart;
-      this.proxStart = proxStart;
-      this.skipOffset = skipOffset;
-    }
+  private static class StandardTermState extends BlockTermState {
+    public long freqStart;
+    public long proxStart;
+    public long skipOffset;
   }
 
-  private final List<PendingTerm> pendingTerms = new ArrayList<PendingTerm>();
-
   /** Called when we are done adding docs to this term */
   @Override
-  public void finishTerm(TermStats stats) throws IOException {
-
+  public void finishTerm(BlockTermState _state) throws IOException {
+    StandardTermState state = (StandardTermState)_state;
     // if (DEBUG) System.out.println("SPW: finishTerm seg=" + segment + " freqStart=" + freqStart);
-    assert stats.docFreq > 0;
+    assert state.docFreq > 0;
 
     // TODO: wasteful we are counting this (counting # docs
     // for this term) in two places?
-    assert stats.docFreq == df;
-
-    final long skipOffset;
+    assert state.docFreq == df;
+    state.freqStart = freqStart;
+    state.proxStart = proxStart;
     if (df >= skipMinimum) {
-      skipOffset = skipListWriter.writeSkip(freqOut)-freqStart;
+      state.skipOffset = skipListWriter.writeSkip(freqOut)-freqStart;
     } else {
-      skipOffset = -1;
+      state.skipOffset = -1;
     }
-
-    pendingTerms.add(new PendingTerm(freqStart, proxStart, skipOffset));
-
     lastDocID = 0;
     df = 0;
   }
 
-  private final RAMOutputStream bytesWriter = new RAMOutputStream();
-
   @Override
-  public void flushTermsBlock(int start, int count) throws IOException {
-    //if (DEBUG) System.out.println("SPW: flushTermsBlock start=" + start + " count=" + count + " left=" + (pendingTerms.size()-count) + " pendingTerms.size()=" + pendingTerms.size());
-
-    if (count == 0) {
-      termsOut.writeByte((byte) 0);
-      return;
-    }
-
-    assert start <= pendingTerms.size();
-    assert count <= start;
-
-    final int limit = pendingTerms.size() - start + count;
-    final PendingTerm firstTerm = pendingTerms.get(limit - count);
-    // First term in block is abs coded:
-    bytesWriter.writeVLong(firstTerm.freqStart);
-
-    if (firstTerm.skipOffset != -1) {
-      assert firstTerm.skipOffset > 0;
-      bytesWriter.writeVLong(firstTerm.skipOffset);
+  public void encodeTerm(long[] empty, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    StandardTermState state = (StandardTermState)_state;
+    if (absolute) {
+      lastState = emptyState;
+    }
+    out.writeVLong(state.freqStart - lastState.freqStart);
+    if (state.skipOffset != -1) {
+      assert state.skipOffset > 0;
+      out.writeVLong(state.skipOffset);
     }
     if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-      bytesWriter.writeVLong(firstTerm.proxStart);
+      out.writeVLong(state.proxStart - lastState.proxStart);
     }
-    long lastFreqStart = firstTerm.freqStart;
-    long lastProxStart = firstTerm.proxStart;
-    for(int idx=limit-count+1; idx<limit; idx++) {
-      final PendingTerm term = pendingTerms.get(idx);
-      //if (DEBUG) System.out.println("  write term freqStart=" + term.freqStart);
-      // The rest of the terms term are delta coded:
-      bytesWriter.writeVLong(term.freqStart - lastFreqStart);
-      lastFreqStart = term.freqStart;
-      if (term.skipOffset != -1) {
-        assert term.skipOffset > 0;
-        bytesWriter.writeVLong(term.skipOffset);
-      }
-      if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0) {
-        bytesWriter.writeVLong(term.proxStart - lastProxStart);
-        lastProxStart = term.proxStart;
-      }
-    }
-
-    termsOut.writeVInt((int) bytesWriter.getFilePointer());
-    bytesWriter.writeTo(termsOut);
-    bytesWriter.reset();
-
-    // Remove the terms we just wrote:
-    pendingTerms.subList(limit-count, limit).clear();
+    lastState = state;
   }
 
   @Override

Modified: lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java?rev=1562497&r1=1562496&r2=1562497&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java (original)
+++ lucene/dev/branches/branch_4x/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java Wed Jan 29 16:21:48 2014
@@ -50,6 +50,10 @@ 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;
@@ -183,12 +187,33 @@ public final class MockRandomPostingsFor
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: writing pulsing postings with totTFCutoff=" + totTFCutoff);
       }
-      postingsWriter = new PulsingPostingsWriter(totTFCutoff, postingsWriter);
+      postingsWriter = new PulsingPostingsWriter(state, totTFCutoff, postingsWriter);
     }
 
     final FieldsConsumer fields;
+    final int t1 = random.nextInt(4);
 
-    if (random.nextBoolean()) {
+    if (t1 == 0) {
+      boolean success = false;
+      try {
+        fields = new FSTTermsWriter(state, postingsWriter);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsWriter.close();
+        }
+      }
+    } else if (t1 == 1) {
+      boolean success = false;
+      try {
+        fields = new FSTOrdTermsWriter(state, postingsWriter);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsWriter.close();
+        }
+      }
+    } else if (t1 == 2) {
       // Use BlockTree terms dict
 
       if (LuceneTestCase.VERBOSE) {
@@ -322,12 +347,32 @@ public final class MockRandomPostingsFor
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading pulsing postings with totTFCutoff=" + totTFCutoff);
       }
-      postingsReader = new PulsingPostingsReader(postingsReader);
+      postingsReader = new PulsingPostingsReader(state, postingsReader);
     }
 
     final FieldsProducer fields;
-
-    if (random.nextBoolean()) {
+    final int t1 = random.nextInt(4);
+    if (t1 == 0) {
+      boolean success = false;
+      try {
+        fields = new FSTTermsReader(state, postingsReader);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsReader.close();
+        }
+      }
+    } else if (t1 == 1) {
+      boolean success = false;
+      try {
+        fields = new FSTOrdTermsReader(state, postingsReader);
+        success = true;
+      } finally {
+        if (!success) {
+          postingsReader.close();
+        }
+      }
+    } else if (t1 == 2) {
       // Use BlockTree terms dict
       if (LuceneTestCase.VERBOSE) {
         System.out.println("MockRandomCodec: reading BlockTree terms dict");