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

svn commit: r1534320 [8/39] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/idea/.idea/ dev-tools/idea/lucene/expressions/ dev-tools/idea/solr/contrib/velocity/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/expressions/ ...

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/sep/SepPostingsWriter.java Mon Oct 21 18:58:24 2013
@@ -18,20 +18,18 @@ package org.apache.lucene.codecs.sep;
  */
 
 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.FieldInfo;
 import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
@@ -39,7 +37,7 @@ import org.apache.lucene.util.IOUtils;
  *  to .pyl, skip data to .skp
  *
  * @lucene.experimental */
-public final class SepPostingsWriter extends PostingsWriterBase {
+public final class SepPostingsWriter extends PushPostingsWriterBase {
   final static String CODEC = "SepPostingsWriter";
 
   final static String DOC_EXTENSION = "doc";
@@ -64,7 +62,6 @@ public final class SepPostingsWriter ext
   IndexOutput payloadOut;
 
   IndexOutput skipOut;
-  IndexOutput termsOut;
 
   final SepSkipListWriter skipListWriter;
   /** Expert: The fraction of TermDocs entries stored in skip tables,
@@ -87,19 +84,17 @@ public final class SepPostingsWriter ext
 
   final int totalNumDocs;
 
-  boolean storePayloads;
   IndexOptions indexOptions;
 
-  FieldInfo fieldInfo;
-
   int lastPayloadLength;
   int lastPosition;
   long payloadStart;
   int lastDocID;
   int df;
 
-  // Holds pending byte[] blob for the current terms block
-  private final RAMOutputStream indexBytesWriter = new RAMOutputStream();
+  SepTermState lastState;
+  long lastPayloadFP;
+  long lastSkipFP;
 
   public SepPostingsWriter(SegmentWriteState state, IntStreamFactory factory) throws IOException {
     this(state, factory, DEFAULT_SKIP_INTERVAL);
@@ -116,9 +111,10 @@ public final class SepPostingsWriter ext
       this.skipInterval = skipInterval;
       this.skipMinimum = skipInterval; /* set to the same for now */
       final String docFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, DOC_EXTENSION);
+
       docOut = factory.createOutput(state.directory, docFileName, state.context);
       docIndex = docOut.index();
-      
+
       if (state.fieldInfos.hasFreq()) {
         final String frqFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, FREQ_EXTENSION);
         freqOut = factory.createOutput(state.directory, frqFileName, state.context);
@@ -134,7 +130,7 @@ public final class SepPostingsWriter ext
         final String payloadFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, PAYLOAD_EXTENSION);
         payloadOut = state.directory.createOutput(payloadFileName, state.context);
       }
-      
+
       final String skipFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SKIP_EXTENSION);
       skipOut = state.directory.createOutput(skipFileName, state.context);
       
@@ -155,8 +151,7 @@ public final class SepPostingsWriter ext
   }
 
   @Override
-  public void start(IndexOutput termsOut) throws IOException {
-    this.termsOut = termsOut;
+  public void init(IndexOutput termsOut) throws IOException {
     CodecUtil.writeHeader(termsOut, CODEC, VERSION_CURRENT);
     // TODO: -- just ask skipper to "start" here
     termsOut.writeInt(skipInterval);                // write skipInterval
@@ -165,6 +160,11 @@ public final class SepPostingsWriter ext
   }
 
   @Override
+  public BlockTermState newTermState() {
+    return new SepTermState();
+  }
+
+  @Override
   public void startTerm() throws IOException {
     docIndex.mark();
     //System.out.println("SEPW: startTerm docIndex=" + docIndex);
@@ -185,14 +185,31 @@ public final class SepPostingsWriter ext
   // Currently, this instance is re-used across fields, so
   // our parent calls setField whenever the field changes
   @Override
-  public void setField(FieldInfo fieldInfo) {
-    this.fieldInfo = fieldInfo;
+  public int setField(FieldInfo fieldInfo) {
+    super.setField(fieldInfo);
     this.indexOptions = fieldInfo.getIndexOptions();
     if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0) {
       throw new UnsupportedOperationException("this codec cannot index offsets");
     }
     skipListWriter.setIndexOptions(indexOptions);
-    storePayloads = indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS && fieldInfo.hasPayloads();
+    lastPayloadFP = 0;
+    lastSkipFP = 0;
+    lastState = setEmptyState();
+    return 0;
+  }
+
+  private SepTermState setEmptyState() {
+    SepTermState emptyState = new SepTermState();
+    emptyState.docIndex = docOut.index();
+    if (indexOptions != IndexOptions.DOCS_ONLY) {
+      emptyState.freqIndex = freqOut.index();
+      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        emptyState.posIndex = posOut.index();
+      }
+    }
+    emptyState.payloadFP = 0;
+    emptyState.skipFP = 0;
+    return emptyState;
   }
 
   /** Adds a new doc in this term.  If this returns null
@@ -211,7 +228,7 @@ public final class SepPostingsWriter ext
       // TODO: -- awkward we have to make these two
       // separate calls to skipper
       //System.out.println("    buffer skip lastDocID=" + lastDocID);
-      skipListWriter.setSkipData(lastDocID, storePayloads, lastPayloadLength);
+      skipListWriter.setSkipData(lastDocID, writePayloads, lastPayloadLength);
       skipListWriter.bufferSkip(df);
     }
 
@@ -232,7 +249,7 @@ public final class SepPostingsWriter ext
     assert delta >= 0: "position=" + position + " lastPosition=" + lastPosition;            // not quite right (if pos=0 is repeated twice we don't catch it)
     lastPosition = position;
 
-    if (storePayloads) {
+    if (writePayloads) {
       final int payloadLength = payload == null ? 0 : payload.length;
       if (payloadLength != lastPayloadLength) {
         lastPayloadLength = payloadLength;
@@ -260,132 +277,86 @@ public final class SepPostingsWriter ext
     lastPosition = 0;
   }
 
-  private static class PendingTerm {
-    public final IntIndexOutput.Index docIndex;
-    public final IntIndexOutput.Index freqIndex;
-    public final IntIndexOutput.Index posIndex;
-    public final long payloadFP;
-    public final long skipFP;
-
-    public PendingTerm(IntIndexOutput.Index docIndex, IntIndexOutput.Index freqIndex, IntIndexOutput.Index posIndex, long payloadFP, long skipFP) {
-      this.docIndex = docIndex;
-      this.freqIndex = freqIndex;
-      this.posIndex = posIndex;
-      this.payloadFP = payloadFP;
-      this.skipFP = skipFP;
-    }
+  private static class SepTermState extends BlockTermState {
+    public IntIndexOutput.Index docIndex;
+    public IntIndexOutput.Index freqIndex;
+    public IntIndexOutput.Index posIndex;
+    public long payloadFP;
+    public long skipFP;
   }
 
-  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 {
+    SepTermState state = (SepTermState)_state;
     // TODO: -- wasteful we are counting this in two places?
-    assert stats.docFreq > 0;
-    assert stats.docFreq == df;
+    assert state.docFreq > 0;
+    assert state.docFreq == df;
 
-    final IntIndexOutput.Index docIndexCopy = docOut.index();
-    docIndexCopy.copyFrom(docIndex, false);
-
-    final IntIndexOutput.Index freqIndexCopy;
-    final IntIndexOutput.Index posIndexCopy;
+    state.docIndex = docOut.index();
+    state.docIndex.copyFrom(docIndex, false);
     if (indexOptions != IndexOptions.DOCS_ONLY) {
-      freqIndexCopy = freqOut.index();
-      freqIndexCopy.copyFrom(freqIndex, false);
+      state.freqIndex = freqOut.index();
+      state.freqIndex.copyFrom(freqIndex, false);
       if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-        posIndexCopy = posOut.index();
-        posIndexCopy.copyFrom(posIndex, false);
+        state.posIndex = posOut.index();
+        state.posIndex.copyFrom(posIndex, false);
       } else {
-        posIndexCopy = null;
+        state.posIndex = null;
       }
     } else {
-      freqIndexCopy = null;
-      posIndexCopy = null;
+      state.freqIndex = null;
+      state.posIndex = null;
     }
 
-    final long skipFP;
     if (df >= skipMinimum) {
-      skipFP = skipOut.getFilePointer();
+      state.skipFP = skipOut.getFilePointer();
       //System.out.println("  skipFP=" + skipFP);
       skipListWriter.writeSkip(skipOut);
       //System.out.println("    numBytes=" + (skipOut.getFilePointer()-skipFP));
     } else {
-      skipFP = -1;
+      state.skipFP = -1;
     }
+    state.payloadFP = payloadStart;
 
     lastDocID = 0;
     df = 0;
-
-    pendingTerms.add(new PendingTerm(docIndexCopy,
-                                     freqIndexCopy,
-                                     posIndexCopy,
-                                     payloadStart,
-                                     skipFP));
   }
 
   @Override
-  public void flushTermsBlock(int start, int count) throws IOException {
-    //System.out.println("SEPW: flushTermsBlock: start=" + start + " count=" + count + " pendingTerms.size()=" + pendingTerms.size() + " termsOut.fp=" + termsOut.getFilePointer());
-    assert indexBytesWriter.getFilePointer() == 0;
-    final int absStart = pendingTerms.size() - start;
-    final List<PendingTerm> slice = pendingTerms.subList(absStart, absStart+count);
-
-    long lastPayloadFP = 0;
-    long lastSkipFP = 0;
-
-    if (count == 0) {
-      termsOut.writeByte((byte) 0);
-      return;
-    }
-
-    final PendingTerm firstTerm = slice.get(0);
-    final IntIndexOutput.Index docIndexFlush = firstTerm.docIndex;
-    final IntIndexOutput.Index freqIndexFlush = firstTerm.freqIndex;
-    final IntIndexOutput.Index posIndexFlush = firstTerm.posIndex;
-
-    for(int idx=0;idx<slice.size();idx++) {
-      final boolean isFirstTerm = idx == 0;
-      final PendingTerm t = slice.get(idx);
-      //System.out.println("  write idx=" + idx + " docIndex=" + t.docIndex);
-      docIndexFlush.copyFrom(t.docIndex, false);
-      docIndexFlush.write(indexBytesWriter, isFirstTerm);
-      if (indexOptions != IndexOptions.DOCS_ONLY) {
-        freqIndexFlush.copyFrom(t.freqIndex, false);
-        freqIndexFlush.write(indexBytesWriter, isFirstTerm);
-        //System.out.println("    freqIndex=" + t.freqIndex);
-        if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
-          posIndexFlush.copyFrom(t.posIndex, false);
-          posIndexFlush.write(indexBytesWriter, isFirstTerm);
-          //System.out.println("    posIndex=" + t.posIndex);
-          if (storePayloads) {
-            //System.out.println("    payloadFP=" + t.payloadFP);
-            if (isFirstTerm) {
-              indexBytesWriter.writeVLong(t.payloadFP);
-            } else {
-              indexBytesWriter.writeVLong(t.payloadFP - lastPayloadFP);
-            }
-            lastPayloadFP = t.payloadFP;
+  public void encodeTerm(long[] longs, DataOutput out, FieldInfo fieldInfo, BlockTermState _state, boolean absolute) throws IOException {
+    SepTermState state = (SepTermState)_state;
+    if (absolute) {
+      lastSkipFP = 0;
+      lastPayloadFP = 0;
+      lastState = state;
+    }
+    lastState.docIndex.copyFrom(state.docIndex, false);
+    lastState.docIndex.write(out, absolute);
+    if (indexOptions != IndexOptions.DOCS_ONLY) {
+      lastState.freqIndex.copyFrom(state.freqIndex, false);
+      lastState.freqIndex.write(out, absolute);
+      if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) {
+        lastState.posIndex.copyFrom(state.posIndex, false);
+        lastState.posIndex.write(out, absolute);
+        if (writePayloads) {
+          if (absolute) {
+            out.writeVLong(state.payloadFP);
+          } else {
+            out.writeVLong(state.payloadFP - lastPayloadFP);
           }
+          lastPayloadFP = state.payloadFP;
         }
       }
-
-      if (t.skipFP != -1) {
-        if (isFirstTerm) {
-          indexBytesWriter.writeVLong(t.skipFP);
-        } else {
-          indexBytesWriter.writeVLong(t.skipFP - lastSkipFP);
-        }
-        lastSkipFP = t.skipFP;
-        //System.out.println("    skipFP=" + t.skipFP);
+    }
+    if (state.skipFP != -1) {
+      if (absolute) {
+        out.writeVLong(state.skipFP);
+      } else {
+        out.writeVLong(state.skipFP - lastSkipFP);
       }
+      lastSkipFP = state.skipFP;
     }
-
-    //System.out.println("  numBytes=" + indexBytesWriter.getFilePointer());
-    termsOut.writeVLong((int) indexBytesWriter.getFilePointer());
-    indexBytesWriter.writeTo(termsOut);
-    indexBytesWriter.reset();
-    slice.clear();
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesFormat.java Mon Oct 21 18:58:24 2013
@@ -38,12 +38,16 @@ import org.apache.lucene.index.SegmentWr
  *    minvalue 0
  *    pattern 000
  *  005
+ *  T
  *  234
+ *  T
  *  123
+ *  T
  *  ...
  *  </pre>
  *  so a document's value (delta encoded from minvalue) can be retrieved by 
- *  seeking to startOffset + (1+pattern.length())*docid. The extra 1 is the newline.
+ *  seeking to startOffset + (1+pattern.length()+2)*docid. The extra 1 is the newline. 
+ *  The extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
  *  
  *  for bytes this is also a "fixed-width" file, for example:
  *  <pre>
@@ -53,12 +57,15 @@ import org.apache.lucene.index.SegmentWr
  *    pattern 0
  *  length 6
  *  foobar[space][space]
+ *  T
  *  length 3
  *  baz[space][space][space][space][space]
+ *  T
  *  ...
  *  </pre>
- *  so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*doc
+ *  so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength+2)*doc
  *  the extra 9 is 2 newlines, plus "length " itself.
+ *  the extra 2 is another newline and 'T' or 'F': true if the value is real, false if missing.
  *  
  *  for sorted bytes this is a fixed-width file, for example:
  *  <pre>

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesReader.java Mon Oct 21 18:58:24 2013
@@ -17,6 +17,16 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
+import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
+
 import java.io.IOException;
 import java.math.BigDecimal;
 import java.math.BigInteger;
@@ -31,26 +41,17 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.FieldInfo.DocValuesType;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.StringHelper;
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
-import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
-
 class SimpleTextDocValuesReader extends DocValuesProducer {
 
   static class OneField {
@@ -61,7 +62,7 @@ class SimpleTextDocValuesReader extends 
     boolean fixedLength;
     long minValue;
     long numValues;
-  };
+  }
 
   final int maxDoc;
   final IndexInput data;
@@ -69,7 +70,7 @@ class SimpleTextDocValuesReader extends 
   final Map<String,OneField> fields = new HashMap<String,OneField>();
   
   public SimpleTextDocValuesReader(SegmentReadState state, String ext) throws IOException {
-    //System.out.println("dir=" + state.directory + " seg=" + state.segmentInfo.name + " ext=" + ext);
+    // System.out.println("dir=" + state.directory + " seg=" + state.segmentInfo.name + " file=" + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext));
     data = state.directory.openInput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
     maxDoc = state.segmentInfo.getDocCount();
     while(true) {
@@ -81,8 +82,6 @@ class SimpleTextDocValuesReader extends 
       assert startsWith(FIELD) : scratch.utf8ToString();
       String fieldName = stripPrefix(FIELD);
       //System.out.println("  field=" + fieldName);
-      FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldName);
-      assert fieldInfo != null;
 
       OneField field = new OneField();
       fields.put(fieldName, field);
@@ -100,7 +99,7 @@ class SimpleTextDocValuesReader extends 
         assert startsWith(PATTERN);
         field.pattern = stripPrefix(PATTERN);
         field.dataStartFilePointer = data.getFilePointer();
-        data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
+        data.seek(data.getFilePointer() + (1+field.pattern.length()+2) * maxDoc);
       } else if (dvType == DocValuesType.BINARY) {
         readLine();
         assert startsWith(MAXLENGTH);
@@ -109,7 +108,7 @@ class SimpleTextDocValuesReader extends 
         assert startsWith(PATTERN);
         field.pattern = stripPrefix(PATTERN);
         field.dataStartFilePointer = data.getFilePointer();
-        data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
+        data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength+2) * maxDoc);
       } else if (dvType == DocValuesType.SORTED || dvType == DocValuesType.SORTED_SET) {
         readLine();
         assert startsWith(NUMVALUES);
@@ -158,7 +157,7 @@ class SimpleTextDocValuesReader extends 
           if (docID < 0 || docID >= maxDoc) {
             throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
           }
-          in.seek(field.dataStartFilePointer + (1+field.pattern.length())*docID);
+          in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*docID);
           SimpleTextUtil.readLine(in, scratch);
           //System.out.println("parsing delta: " + scratch.utf8ToString());
           BigDecimal bd;
@@ -169,6 +168,7 @@ class SimpleTextDocValuesReader extends 
             e.initCause(pe);
             throw e;
           }
+          SimpleTextUtil.readLine(in, scratch); // read the line telling us if its real or not
           return BigInteger.valueOf(field.minValue).add(bd.toBigIntegerExact()).longValue();
         } catch (IOException ioe) {
           throw new RuntimeException(ioe);
@@ -176,6 +176,30 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  private Bits getNumericDocsWithField(FieldInfo fieldInfo) throws IOException {
+    final OneField field = fields.get(fieldInfo.name);
+    final IndexInput in = data.clone();
+    final BytesRef scratch = new BytesRef();
+    return new Bits() {
+      @Override
+      public boolean get(int index) {
+        try {
+          in.seek(field.dataStartFilePointer + (1+field.pattern.length()+2)*index);
+          SimpleTextUtil.readLine(in, scratch); // data
+          SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
+          return scratch.bytes[scratch.offset] == (byte) 'T';
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+
+      @Override
+      public int length() {
+        return maxDoc;
+      }
+    };
+  }
 
   @Override
   public BinaryDocValues getBinary(FieldInfo fieldInfo) throws IOException {
@@ -196,7 +220,7 @@ class SimpleTextDocValuesReader extends 
           if (docID < 0 || docID >= maxDoc) {
             throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
           }
-          in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength)*docID);
+          in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*docID);
           SimpleTextUtil.readLine(in, scratch);
           assert StringHelper.startsWith(scratch, LENGTH);
           int len;
@@ -217,6 +241,45 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  private Bits getBinaryDocsWithField(FieldInfo fieldInfo) throws IOException {
+    final OneField field = fields.get(fieldInfo.name);
+    final IndexInput in = data.clone();
+    final BytesRef scratch = new BytesRef();
+    final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
+
+    return new Bits() {
+      @Override
+      public boolean get(int index) {
+        try {
+          in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength+2)*index);
+          SimpleTextUtil.readLine(in, scratch);
+          assert StringHelper.startsWith(scratch, LENGTH);
+          int len;
+          try {
+            len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
+          } catch (ParseException pe) {
+            CorruptIndexException e = new CorruptIndexException("failed to parse int length (resource=" + in + ")");
+            e.initCause(pe);
+            throw e;
+          }
+          // skip past bytes
+          byte bytes[] = new byte[len];
+          in.readBytes(bytes, 0, len);
+          SimpleTextUtil.readLine(in, scratch); // newline
+          SimpleTextUtil.readLine(in, scratch); // 'T' or 'F'
+          return scratch.bytes[scratch.offset] == (byte) 'T';
+        } catch (IOException ioe) {
+          throw new RuntimeException(ioe);
+        }
+      }
+
+      @Override
+      public int length() {
+        return maxDoc;
+      }
+    };
+  }
 
   @Override
   public SortedDocValues getSorted(FieldInfo fieldInfo) throws IOException {
@@ -241,7 +304,7 @@ class SimpleTextDocValuesReader extends 
           in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
           SimpleTextUtil.readLine(in, scratch);
           try {
-            return ordDecoder.parse(scratch.utf8ToString()).intValue();
+            return (int) ordDecoder.parse(scratch.utf8ToString()).longValue()-1;
           } catch (ParseException pe) {
             CorruptIndexException e = new CorruptIndexException("failed to parse ord (resource=" + in + ")");
             e.initCause(pe);
@@ -362,6 +425,22 @@ class SimpleTextDocValuesReader extends 
       }
     };
   }
+  
+  @Override
+  public Bits getDocsWithField(FieldInfo field) throws IOException {
+    switch (field.getDocValuesType()) {
+      case SORTED_SET:
+        return new SortedSetDocsWithField(getSortedSet(field), maxDoc);
+      case SORTED:
+        return new SortedDocsWithField(getSorted(field), maxDoc);
+      case BINARY:
+        return getBinaryDocsWithField(field);
+      case NUMERIC:
+        return getNumericDocsWithField(field);
+      default:
+        throw new AssertionError();
+    }
+  }
 
   @Override
   public void close() throws IOException {
@@ -383,4 +462,9 @@ class SimpleTextDocValuesReader extends 
   private String stripPrefix(BytesRef prefix) throws IOException {
     return new String(scratch.bytes, scratch.offset + prefix.length, scratch.length - prefix.length, "UTF-8");
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextDocValuesWriter.java Mon Oct 21 18:58:24 2013
@@ -55,7 +55,7 @@ class SimpleTextDocValuesWriter extends 
   private final Set<String> fieldsSeen = new HashSet<String>(); // for asserting
   
   public SimpleTextDocValuesWriter(SegmentWriteState state, String ext) throws IOException {
-    //System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
+    // System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
     data = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
     numDocs = state.segmentInfo.getDocCount();
   }
@@ -78,7 +78,7 @@ class SimpleTextDocValuesWriter extends 
     long minValue = Long.MAX_VALUE;
     long maxValue = Long.MIN_VALUE;
     for(Number n : values) {
-      long v = n.longValue();
+      long v = n == null ? 0 : n.longValue();
       minValue = Math.min(minValue, v);
       maxValue = Math.max(maxValue, v);
     }
@@ -112,13 +112,19 @@ class SimpleTextDocValuesWriter extends 
 
     // second pass to write the values
     for(Number n : values) {
-      long value = n.longValue();
+      long value = n == null ? 0 : n.longValue();
       assert value >= minValue;
       Number delta = BigInteger.valueOf(value).subtract(BigInteger.valueOf(minValue));
       String s = encoder.format(delta);
       assert s.length() == patternString.length();
       SimpleTextUtil.write(data, s, scratch);
       SimpleTextUtil.writeNewline(data);
+      if (n == null) {
+        SimpleTextUtil.write(data, "F", scratch);
+      } else {
+        SimpleTextUtil.write(data, "T", scratch);
+      }
+      SimpleTextUtil.writeNewline(data);
       numDocsWritten++;
       assert numDocsWritten <= numDocs;
     }
@@ -132,7 +138,8 @@ class SimpleTextDocValuesWriter extends 
     assert field.getDocValuesType() == DocValuesType.BINARY;
     int maxLength = 0;
     for(BytesRef value : values) {
-      maxLength = Math.max(maxLength, value.length);
+      final int length = value == null ? 0 : value.length;
+      maxLength = Math.max(maxLength, length);
     }
     writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
 
@@ -155,19 +162,28 @@ class SimpleTextDocValuesWriter extends 
     int numDocsWritten = 0;
     for(BytesRef value : values) {
       // write length
+      final int length = value == null ? 0 : value.length;
       SimpleTextUtil.write(data, LENGTH);
-      SimpleTextUtil.write(data, encoder.format(value.length), scratch);
+      SimpleTextUtil.write(data, encoder.format(length), scratch);
       SimpleTextUtil.writeNewline(data);
         
       // write bytes -- don't use SimpleText.write
       // because it escapes:
-      data.writeBytes(value.bytes, value.offset, value.length);
+      if (value != null) {
+        data.writeBytes(value.bytes, value.offset, value.length);
+      }
 
       // pad to fit
-      for (int i = value.length; i < maxLength; i++) {
+      for (int i = length; i < maxLength; i++) {
         data.writeByte((byte)' ');
       }
       SimpleTextUtil.writeNewline(data);
+      if (value == null) {
+        SimpleTextUtil.write(data, "F", scratch);
+      } else {
+        SimpleTextUtil.write(data, "T", scratch);
+      }
+      SimpleTextUtil.writeNewline(data);
       numDocsWritten++;
     }
 
@@ -209,7 +225,7 @@ class SimpleTextDocValuesWriter extends 
     SimpleTextUtil.writeNewline(data);
     final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
     
-    int maxOrdBytes = Integer.toString(valueCount).length();
+    int maxOrdBytes = Long.toString(valueCount+1L).length();
     sb.setLength(0);
     for (int i = 0; i < maxOrdBytes; i++) {
       sb.append('0');
@@ -246,7 +262,7 @@ class SimpleTextDocValuesWriter extends 
     assert valuesSeen == valueCount;
 
     for(Number ord : docToOrd) {
-      SimpleTextUtil.write(data, ordEncoder.format(ord.intValue()), scratch);
+      SimpleTextUtil.write(data, ordEncoder.format(ord.longValue()+1), scratch);
       SimpleTextUtil.writeNewline(data);
     }
   }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosReader.java Mon Oct 21 18:58:24 2013
@@ -47,8 +47,8 @@ import static org.apache.lucene.codecs.s
 public class SimpleTextFieldInfosReader extends FieldInfosReader {
 
   @Override
-  public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
+  public FieldInfos read(Directory directory, String segmentName, String segmentSuffix, IOContext iocontext) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
     IndexInput input = directory.openInput(fileName, iocontext);
     BytesRef scratch = new BytesRef();
     
@@ -105,6 +105,10 @@ public class SimpleTextFieldInfosReader 
         final DocValuesType docValuesType = docValuesType(dvType);
         
         SimpleTextUtil.readLine(input, scratch);
+        assert StringHelper.startsWith(scratch, DOCVALUES_GEN);
+        final long dvGen = Long.parseLong(readString(DOCVALUES_GEN.length, scratch));
+        
+        SimpleTextUtil.readLine(input, scratch);
         assert StringHelper.startsWith(scratch, NUM_ATTS);
         int numAtts = Integer.parseInt(readString(NUM_ATTS.length, scratch));
         Map<String,String> atts = new HashMap<String,String>();
@@ -122,6 +126,7 @@ public class SimpleTextFieldInfosReader 
 
         infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, 
           omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(atts));
+        infos[i].setDocValuesGen(dvGen);
       }
 
       if (input.getFilePointer() != input.length()) {

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldInfosWriter.java Mon Oct 21 18:58:24 2013
@@ -53,14 +53,15 @@ public class SimpleTextFieldInfosWriter 
   static final BytesRef NORMS           =  new BytesRef("  norms ");
   static final BytesRef NORMS_TYPE      =  new BytesRef("  norms type ");
   static final BytesRef DOCVALUES       =  new BytesRef("  doc values ");
+  static final BytesRef DOCVALUES_GEN   =  new BytesRef("  doc values gen ");
   static final BytesRef INDEXOPTIONS    =  new BytesRef("  index options ");
   static final BytesRef NUM_ATTS        =  new BytesRef("  attributes ");
   final static BytesRef ATT_KEY         =  new BytesRef("    key ");
   final static BytesRef ATT_VALUE       =  new BytesRef("    value ");
   
   @Override
-  public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
-    final String fileName = IndexFileNames.segmentFileName(segmentName, "", FIELD_INFOS_EXTENSION);
+  public void write(Directory directory, String segmentName, String segmentSuffix, FieldInfos infos, IOContext context) throws IOException {
+    final String fileName = IndexFileNames.segmentFileName(segmentName, segmentSuffix, FIELD_INFOS_EXTENSION);
     IndexOutput out = directory.createOutput(fileName, context);
     BytesRef scratch = new BytesRef();
     boolean success = false;
@@ -108,6 +109,10 @@ public class SimpleTextFieldInfosWriter 
         SimpleTextUtil.write(out, DOCVALUES);
         SimpleTextUtil.write(out, getDocValuesType(fi.getDocValuesType()), scratch);
         SimpleTextUtil.writeNewline(out);
+        
+        SimpleTextUtil.write(out, DOCVALUES_GEN);
+        SimpleTextUtil.write(out, Long.toString(fi.getDocValuesGen()), scratch);
+        SimpleTextUtil.writeNewline(out);
                
         Map<String,String> atts = fi.attributes();
         int numAtts = atts == null ? 0 : atts.size();

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Mon Oct 21 18:58:24 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -218,11 +217,6 @@ class SimpleTextFieldsReader extends Fie
       } 
       return docsAndPositionsEnum.reset(docsStart, liveDocs, indexOptions, docFreq);
     }
-    
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
   }
 
   private class SimpleTextDocsEnum extends DocsEnum {
@@ -574,6 +568,11 @@ class SimpleTextFieldsReader extends Fie
       */
       //System.out.println("FST " + fst.sizeInBytes());
     }
+    
+    /** Returns approximate RAM bytes used */
+    public long ramBytesUsed() {
+      return (fst!=null) ? fst.sizeInBytes() : 0;
+    }
 
     @Override
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
@@ -585,11 +584,6 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
-
-    @Override
     public long size() {
       return (long) termCount;
     }
@@ -610,6 +604,11 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
+    public boolean hasFreqs() {
+      return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
+    }
+
+    @Override
     public boolean hasOffsets() {
       return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
     }
@@ -630,7 +629,7 @@ class SimpleTextFieldsReader extends Fie
     return Collections.unmodifiableSet(fields.keySet()).iterator();
   }
 
-  private final Map<String,Terms> termsCache = new HashMap<String,Terms>();
+  private final Map<String,SimpleTextTerms> termsCache = new HashMap<String,SimpleTextTerms>();
 
   @Override
   synchronized public Terms terms(String field) throws IOException {
@@ -641,7 +640,7 @@ class SimpleTextFieldsReader extends Fie
         return null;
       } else {
         terms = new SimpleTextTerms(field, fp);
-        termsCache.put(field, terms);
+        termsCache.put(field, (SimpleTextTerms) terms);
       }
     }
     return terms;
@@ -656,4 +655,13 @@ class SimpleTextFieldsReader extends Fie
   public void close() throws IOException {
     in.close();
   }
+
+  @Override
+  public long ramBytesUsed() {
+    long sizeInBytes = 0;
+    for(SimpleTextTerms simpleTextTerms : termsCache.values()) {
+      sizeInBytes += (simpleTextTerms!=null) ? simpleTextTerms.ramBytesUsed() : 0;
+    }
+    return sizeInBytes;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Mon Oct 21 18:58:24 2013
@@ -17,23 +17,27 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
-import org.apache.lucene.util.BytesRef;
+import java.io.Closeable;
+import java.io.IOException;
+
 import org.apache.lucene.codecs.FieldsConsumer;
-import org.apache.lucene.codecs.PostingsConsumer;
-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;
-import org.apache.lucene.index.FieldInfo.IndexOptions;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 
-import java.io.IOException;
-import java.util.Comparator;
-
-class SimpleTextFieldsWriter extends FieldsConsumer {
+class SimpleTextFieldsWriter extends FieldsConsumer implements Closeable {
   
   private final IndexOutput out;
   private final BytesRef scratch = new BytesRef(10);
+  private final SegmentWriteState writeState;
 
   final static BytesRef END          = new BytesRef("END");
   final static BytesRef FIELD        = new BytesRef("field ");
@@ -45,134 +49,168 @@ class SimpleTextFieldsWriter extends Fie
   final static BytesRef END_OFFSET   = new BytesRef("      endOffset ");
   final static BytesRef PAYLOAD      = new BytesRef("        payload ");
 
-  public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
-    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix);
-    out = state.directory.createOutput(fileName, state.context);
-  }
-
-  private void write(String s) throws IOException {
-    SimpleTextUtil.write(out, s, scratch);
-  }
-
-  private void write(BytesRef b) throws IOException {
-    SimpleTextUtil.write(out, b);
-  }
-
-  private void newline() throws IOException {
-    SimpleTextUtil.writeNewline(out);
+  public SimpleTextFieldsWriter(SegmentWriteState writeState) throws IOException {
+    final String fileName = SimpleTextPostingsFormat.getPostingsFileName(writeState.segmentInfo.name, writeState.segmentSuffix);
+    out = writeState.directory.createOutput(fileName, writeState.context);
+    this.writeState = writeState;
   }
 
   @Override
-  public TermsConsumer addField(FieldInfo field) throws IOException {
-    write(FIELD);
-    write(field.name);
-    newline();
-    return new SimpleTextTermsWriter(field);
-  }
-
-  private class SimpleTextTermsWriter extends TermsConsumer {
-    private final SimpleTextPostingsWriter postingsWriter;
-    
-    public SimpleTextTermsWriter(FieldInfo field) {
-      postingsWriter = new SimpleTextPostingsWriter(field);
-    }
-
-    @Override
-    public PostingsConsumer startTerm(BytesRef term) throws IOException {
-      return postingsWriter.reset(term);
-    }
-
-    @Override
-    public void finishTerm(BytesRef term, TermStats stats) throws IOException {
-    }
-
-    @Override
-    public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
-    }
-
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
+  public void write(Fields fields) throws IOException {
+    boolean success = false;
+    try {
+      write(writeState.fieldInfos, fields);
+      success = true;
+    } finally {
+      if (success) {
+        IOUtils.close(this);
+      } else {
+        IOUtils.closeWhileHandlingException(this);
+      }
     }
   }
 
-  private class SimpleTextPostingsWriter extends PostingsConsumer {
-    private BytesRef term;
-    private boolean wroteTerm;
-    private final IndexOptions indexOptions;
-    private final boolean writePositions;
-    private final boolean writeOffsets;
-
-    // for assert:
-    private int lastStartOffset = 0;
-
-    public SimpleTextPostingsWriter(FieldInfo field) {
-      this.indexOptions = field.getIndexOptions();
-      writePositions = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
-      writeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-      //System.out.println("writeOffsets=" + writeOffsets);
-      //System.out.println("writePos=" + writePositions);
-    }
+  public void write(FieldInfos fieldInfos, Fields fields) throws IOException {
 
-    @Override
-    public void startDoc(int docID, int termDocFreq) throws IOException {
-      if (!wroteTerm) {
-        // we lazily do this, in case the term had zero docs
-        write(TERM);
-        write(term);
-        newline();
-        wroteTerm = true;
+    // for each field
+    for(String field : fields) {
+      Terms terms = fields.terms(field);
+      if (terms == null) {
+        // Annoyingly, this can happen!
+        continue;
       }
+      FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
 
-      write(DOC);
-      write(Integer.toString(docID));
-      newline();
-      if (indexOptions != IndexOptions.DOCS_ONLY) {
-        write(FREQ);
-        write(Integer.toString(termDocFreq));
-        newline();
-      }
-
-      lastStartOffset = 0;
-    }
-    
-    public PostingsConsumer reset(BytesRef term) {
-      this.term = term;
-      wroteTerm = false;
-      return this;
-    }
-
-    @Override
-    public void addPosition(int position, BytesRef payload, int startOffset, int endOffset) throws IOException {
-      if (writePositions) {
-        write(POS);
-        write(Integer.toString(position));
-        newline();
-      }
+      boolean wroteField = false;
 
-      if (writeOffsets) {
-        assert endOffset >= startOffset;
-        assert startOffset >= lastStartOffset: "startOffset=" + startOffset + " lastStartOffset=" + lastStartOffset;
-        lastStartOffset = startOffset;
-        write(START_OFFSET);
-        write(Integer.toString(startOffset));
-        newline();
-        write(END_OFFSET);
-        write(Integer.toString(endOffset));
-        newline();
+      boolean hasPositions = terms.hasPositions();
+      boolean hasFreqs = terms.hasFreqs();
+      boolean hasPayloads = fieldInfo.hasPayloads();
+      boolean hasOffsets = terms.hasOffsets();
+
+      int flags = 0;
+      if (hasPositions) {
+        
+        if (hasPayloads) {
+          flags = flags | DocsAndPositionsEnum.FLAG_PAYLOADS;
+        }
+        if (hasOffsets) {
+          flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS;
+        }
+      } else {
+        if (hasFreqs) {
+          flags = flags | DocsEnum.FLAG_FREQS;
+        }
       }
 
-      if (payload != null && payload.length > 0) {
-        assert payload.length != 0;
-        write(PAYLOAD);
-        write(payload);
-        newline();
+      TermsEnum termsEnum = terms.iterator(null);
+      DocsAndPositionsEnum posEnum = null;
+      DocsEnum docsEnum = null;
+
+      // for each term in field
+      while(true) {
+        BytesRef term = termsEnum.next();
+        if (term == null) {
+          break;
+        }
+
+        if (hasPositions) {
+          posEnum = termsEnum.docsAndPositions(null, posEnum, flags);
+          docsEnum = posEnum;
+        } else {
+          docsEnum = termsEnum.docs(null, docsEnum, flags);
+        }
+        assert docsEnum != null: "termsEnum=" + termsEnum + " hasPos=" + hasPositions + " flags=" + flags;
+
+        boolean wroteTerm = false;
+
+        // for each doc in field+term
+        while(true) {
+          int doc = docsEnum.nextDoc();
+          if (doc == DocsEnum.NO_MORE_DOCS) {
+            break;
+          }
+
+          if (!wroteTerm) {
+
+            if (!wroteField) {
+              // we lazily do this, in case the field had
+              // no terms              
+              write(FIELD);
+              write(field);
+              newline();
+              wroteField = true;
+            }
+
+            // we lazily do this, in case the term had
+            // zero docs
+            write(TERM);
+            write(term);
+            newline();
+            wroteTerm = true;
+          }
+
+          write(DOC);
+          write(Integer.toString(doc));
+          newline();
+          if (hasFreqs) {
+            int freq = docsEnum.freq();
+            write(FREQ);
+            write(Integer.toString(freq));
+            newline();
+
+            if (hasPositions) {
+              // for assert:
+              int lastStartOffset = 0;
+
+              // for each pos in field+term+doc
+              for(int i=0;i<freq;i++) {
+                int position = posEnum.nextPosition();
+
+                write(POS);
+                write(Integer.toString(position));
+                newline();
+
+                if (hasOffsets) {
+                  int startOffset = posEnum.startOffset();
+                  int endOffset = posEnum.endOffset();
+                  assert endOffset >= startOffset;
+                  assert startOffset >= lastStartOffset: "startOffset=" + startOffset + " lastStartOffset=" + lastStartOffset;
+                  lastStartOffset = startOffset;
+                  write(START_OFFSET);
+                  write(Integer.toString(startOffset));
+                  newline();
+                  write(END_OFFSET);
+                  write(Integer.toString(endOffset));
+                  newline();
+                }
+
+                BytesRef payload = posEnum.getPayload();
+
+                if (payload != null && payload.length > 0) {
+                  assert payload.length != 0;
+                  write(PAYLOAD);
+                  write(payload);
+                  newline();
+                }
+              }
+            }
+          }
+        }
       }
     }
+  }
 
-    @Override
-    public void finishDoc() {
-    }
+  private void write(String s) throws IOException {
+    SimpleTextUtil.write(out, s, scratch);
+  }
+
+  private void write(BytesRef b) throws IOException {
+    SimpleTextUtil.write(out, b);
+  }
+
+  private void newline() throws IOException {
+    SimpleTextUtil.writeNewline(out);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextPostingsFormat.java Mon Oct 21 18:58:24 2013
@@ -22,9 +22,9 @@ import java.io.IOException;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
-import org.apache.lucene.index.SegmentWriteState;
-import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.SegmentReadState;
+import org.apache.lucene.index.SegmentWriteState;
 
 /** For debugging, curiosity, transparency only!!  Do not
  *  use this codec in production.

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoReader.java Mon Oct 21 18:58:24 2013
@@ -17,8 +17,16 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_DIAG_KEY;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_DIAG_VALUE;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_DOCCOUNT;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_FILE;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_NUM_DIAG;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_NUM_FILES;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_USECOMPOUND;
+import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.SI_VERSION;
+
 import java.io.IOException;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -34,8 +42,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.StringHelper;
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextSegmentInfoWriter.*;
-
 /**
  * reads plaintext segments files
  * <p>
@@ -80,22 +86,6 @@ public class SimpleTextSegmentInfoReader
       }
       
       SimpleTextUtil.readLine(input, scratch);
-      assert StringHelper.startsWith(scratch, SI_NUM_ATTS);
-      int numAtts = Integer.parseInt(readString(SI_NUM_ATTS.length, scratch));
-      Map<String,String> attributes = new HashMap<String,String>();
-
-      for (int i = 0; i < numAtts; i++) {
-        SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, SI_ATT_KEY);
-        String key = readString(SI_ATT_KEY.length, scratch);
-      
-        SimpleTextUtil.readLine(input, scratch);
-        assert StringHelper.startsWith(scratch, SI_ATT_VALUE);
-        String value = readString(SI_ATT_VALUE.length, scratch);
-        attributes.put(key, value);
-      }
-
-      SimpleTextUtil.readLine(input, scratch);
       assert StringHelper.startsWith(scratch, SI_NUM_FILES);
       int numFiles = Integer.parseInt(readString(SI_NUM_FILES.length, scratch));
       Set<String> files = new HashSet<String>();
@@ -108,7 +98,7 @@ public class SimpleTextSegmentInfoReader
       }
 
       SegmentInfo info = new SegmentInfo(directory, version, segmentName, docCount, 
-                                         isCompoundFile, null, diagnostics, Collections.unmodifiableMap(attributes));
+                                         isCompoundFile, null, diagnostics);
       info.setFiles(files);
       success = true;
       return info;

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextSegmentInfoWriter.java Mon Oct 21 18:58:24 2013
@@ -45,9 +45,6 @@ public class SimpleTextSegmentInfoWriter
   final static BytesRef SI_NUM_DIAG         = new BytesRef("    diagnostics ");
   final static BytesRef SI_DIAG_KEY         = new BytesRef("      key ");
   final static BytesRef SI_DIAG_VALUE       = new BytesRef("      value ");
-  final static BytesRef SI_NUM_ATTS         = new BytesRef("    attributes ");
-  final static BytesRef SI_ATT_KEY          = new BytesRef("      key ");
-  final static BytesRef SI_ATT_VALUE        = new BytesRef("      value ");
   final static BytesRef SI_NUM_FILES        = new BytesRef("    files ");
   final static BytesRef SI_FILE             = new BytesRef("      file ");
   
@@ -93,24 +90,6 @@ public class SimpleTextSegmentInfoWriter
         }
       }
       
-      Map<String,String> atts = si.attributes();
-      int numAtts = atts == null ? 0 : atts.size();
-      SimpleTextUtil.write(output, SI_NUM_ATTS);
-      SimpleTextUtil.write(output, Integer.toString(numAtts), scratch);
-      SimpleTextUtil.writeNewline(output);
-    
-      if (numAtts > 0) {
-        for (Map.Entry<String,String> entry : atts.entrySet()) {
-          SimpleTextUtil.write(output, SI_ATT_KEY);
-          SimpleTextUtil.write(output, entry.getKey(), scratch);
-          SimpleTextUtil.writeNewline(output);
-        
-          SimpleTextUtil.write(output, SI_ATT_VALUE);
-          SimpleTextUtil.write(output, entry.getValue(), scratch);
-          SimpleTextUtil.writeNewline(output);
-        }
-      }
-
       Set<String> files = si.files();
       int numFiles = files == null ? 0 : files.size();
       SimpleTextUtil.write(output, SI_NUM_FILES);

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsReader.java Mon Oct 21 18:58:24 2013
@@ -192,4 +192,9 @@ public class SimpleTextStoredFieldsReade
     return a.length == b.length - bOffset && 
         ArrayUtil.equals(a.bytes, a.offset, b.bytes, b.offset + bOffset, b.length - bOffset);
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Mon Oct 21 18:58:24 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.codecs.simplet
 
 import java.io.IOException;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.SortedMap;
@@ -272,11 +271,6 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
-
-    @Override
     public long size() throws IOException {
       return terms.size();
     }
@@ -297,6 +291,11 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
+    public boolean hasFreqs() {
+      return true;
+    }
+
+    @Override
     public boolean hasOffsets() {
       return hasOffsets;
     }
@@ -394,11 +393,6 @@ public class SimpleTextTermVectorsReader
       e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
       return e;
     }
-
-    @Override
-    public Comparator<BytesRef> getComparator() {
-      return BytesRef.getUTF8SortedAsUnicodeComparator();
-    }
   }
   
   // note: these two enum classes are exactly like the Default impl...
@@ -538,4 +532,9 @@ public class SimpleTextTermVectorsReader
       return 1;
     }
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return 0;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java Mon Oct 21 18:58:24 2013
@@ -18,7 +18,6 @@ package org.apache.lucene.codecs.simplet
  */
 
 import java.io.IOException;
-import java.util.Comparator;
 
 import org.apache.lucene.codecs.TermVectorsWriter;
 import org.apache.lucene.index.FieldInfo;
@@ -189,11 +188,6 @@ public class SimpleTextTermVectorsWriter
     }
   }
   
-  @Override
-  public Comparator<BytesRef> getComparator() throws IOException {
-    return BytesRef.getUTF8SortedAsUnicodeComparator();
-  }
-  
   private void write(String s) throws IOException {
     SimpleTextUtil.write(out, s, scratch);
   }

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.DocValuesFormat Mon Oct 21 18:58:24 2013
@@ -14,4 +14,5 @@
 #  limitations under the License.
 
 org.apache.lucene.codecs.diskdv.DiskDocValuesFormat
+org.apache.lucene.codecs.memory.MemoryDocValuesFormat
 org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat

Modified: lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat (original)
+++ lucene/dev/branches/lucene4956/lucene/codecs/src/resources/META-INF/services/org.apache.lucene.codecs.PostingsFormat Mon Oct 21 18:58:24 2013
@@ -18,3 +18,7 @@ org.apache.lucene.codecs.simpletext.Simp
 org.apache.lucene.codecs.memory.MemoryPostingsFormat
 org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat
 org.apache.lucene.codecs.memory.DirectPostingsFormat
+org.apache.lucene.codecs.memory.FSTPulsing41PostingsFormat
+org.apache.lucene.codecs.memory.FSTOrdPulsing41PostingsFormat
+org.apache.lucene.codecs.memory.FSTPostingsFormat
+org.apache.lucene.codecs.memory.FSTOrdPostingsFormat

Modified: lucene/dev/branches/lucene4956/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/common-build.xml?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/common-build.xml (original)
+++ lucene/dev/branches/lucene4956/lucene/common-build.xml Mon Oct 21 18:58:24 2013
@@ -360,7 +360,7 @@
   <property name="ivy_install_path" location="${user.home}/.ant/lib" />
   <property name="ivy_bootstrap_url1" value="http://repo1.maven.org/maven2"/>
   <!-- you might need to tweak this from china so it works -->
-  <property name="ivy_bootstrap_url2" value="http://mirror.netcologne.de/maven2"/>
+  <property name="ivy_bootstrap_url2" value="http://uk.maven.org/maven2"/>
   <property name="ivy_checksum_sha1" value="c5ebf1c253ad4959a29f4acfe696ee48cdd9f473"/>
 
   <target name="ivy-availability-check" unless="ivy.available">
@@ -490,12 +490,41 @@
     <jarify/>
   </target>
 
+  <property name="lucene.tgz.file" location="${common.dir}/dist/lucene-${version}.tgz"/>
+  <available file="${lucene.tgz.file}" property="lucene.tgz.exists"/>
+  <property name="lucene.tgz.unpack.dir" location="${common.build.dir}/lucene.tgz.unpacked"/>
+  <patternset id="patternset.lucene.solr.jars">
+    <include name="**/lucene-*.jar"/>
+    <include name="**/solr-*.jar"/>
+  </patternset>
+  <available type="dir" file="${lucene.tgz.unpack.dir}" property="lucene.tgz.unpack.dir.exists"/>
+  <target name="-ensure-lucene-tgz-exists" unless="lucene.tgz.exists">
+    <ant dir="${common.dir}" target="package-tgz" inheritall="false"/>
+  </target>
+  <target name="-unpack-lucene-tgz" unless="lucene.tgz.unpack.dir.exists">
+    <antcall target="-ensure-lucene-tgz-exists" inheritall="false"/>
+    <mkdir dir="${lucene.tgz.unpack.dir}"/>
+    <untar compression="gzip" src="${lucene.tgz.file}" dest="${lucene.tgz.unpack.dir}">
+      <patternset refid="patternset.lucene.solr.jars"/>
+    </untar>
+  </target>
+  <property name="dist.jar.dir.prefix" value="${lucene.tgz.unpack.dir}/lucene"/>
+  <pathconvert property="dist.jar.dir.suffix">
+    <mapper>
+      <chainedmapper>
+        <globmapper from="${common.dir}*" to="*"/>
+        <globmapper from="*build.xml" to="*"/>
+      </chainedmapper>
+    </mapper>
+    <path location="${ant.file}"/>
+  </pathconvert>
+
   <macrodef name="m2-deploy" description="Builds a Maven artifact">
   	<element name="artifact-attachments" optional="yes"/>
     <element name="parent-poms" optional="yes"/>
     <element name="credentials" optional="yes"/>
     <attribute name="pom.xml"/>
-    <attribute name="jar.file" default="${build.dir}/${final.name}.jar"/>
+    <attribute name="jar.file" default="${dist.jar.dir.prefix}-${version}/${dist.jar.dir.suffix}/${final.name}.jar"/>
     <sequential>
       <artifact:install-provider artifactId="wagon-ssh" version="1.0-beta-7"/>
       <parent-poms/>
@@ -1235,7 +1264,7 @@ ${tests-output}/junit4-*.suites     - pe
    -->
   <target name="clover" depends="-clover.disable,-clover.load,-clover.classpath,-clover.setup" description="Instrument the Unit tests using Clover. To use, specify -Drun.clover=true on the command line."/>
   
-  <target name="-clover.load" if="run.clover" unless="clover.loaded">
+  <target name="-clover.load" depends="ivy-availability-check,ivy-configure" if="run.clover" unless="clover.loaded">
     <available file="${clover.license.path}" property="clover.license.available" />
     <fail unless="clover.license.available"><![CDATA[.
 
@@ -1256,8 +1285,8 @@ ${tests-output}/junit4-*.suites     - pe
   
 ]]></fail>
     <echo>Code coverage with Atlassian Clover enabled.</echo>
-    <ivy:cachepath organisation="com.cenqua.clover" module="clover" revision="3.1.10"
-      inline="true" conf="master" type="jar" pathid="clover.classpath"/>
+    <ivy:cachepath organisation="com.cenqua.clover" module="clover" revision="3.2.0-SNAPSHOT"
+      inline="true" conf="master" pathid="clover.classpath"/>
     <taskdef resource="cloverlib.xml" classpathref="clover.classpath" />
     <mkdir dir="${clover.db.dir}"/>
     <!-- This is a hack, instead of setting "clover.loaded" to "true", we set it
@@ -1353,9 +1382,9 @@ ${tests-output}/junit4-*.suites     - pe
   </target>
 
   <target name="dist-maven"
-          depends="filter-pom-templates, install-maven-tasks, m2-deploy-lucene-parent-pom, dist-maven-common"/>
+          depends="filter-pom-templates, install-maven-tasks, m2-deploy-lucene-parent-pom, -unpack-lucene-tgz, dist-maven-common"/>
   <target name="dist-maven-common"
-          depends="jar-core, jar-src, javadocs, install-maven-tasks, filter-pom-templates">
+          depends="jar-src, javadocs, install-maven-tasks, filter-pom-templates">
     <sequential>
       <property name="top.level.dir" location="${common.dir}/.."/>
       <pathconvert property="pom.xml">
@@ -1379,9 +1408,9 @@ ${tests-output}/junit4-*.suites     - pe
   </target>
 
   <target name="dist-maven-src-java"
-          depends="filter-pom-templates, install-maven-tasks, m2-deploy-lucene-parent-pom, dist-maven-common-src-java"/>
+          depends="filter-pom-templates, install-maven-tasks, m2-deploy-lucene-parent-pom, -unpack-lucene-tgz, dist-maven-common-src-java"/>
   <target name="dist-maven-common-src-java"
-          depends="jar-core, jar-src, javadocs, install-maven-tasks, filter-pom-templates">
+          depends="-unpack-lucene-tgz, jar-src, javadocs, install-maven-tasks, filter-pom-templates">
     <sequential>
       <property name="top.level.dir" location="${common.dir}/.."/>
       <pathconvert property="pom.xml">
@@ -1498,7 +1527,7 @@ ${tests-output}/junit4-*.suites     - pe
   </target>
 
   <target name="rat-sources-typedef" unless="rat.loaded">
-    <ivy:cachepath organisation="org.apache.rat" module="apache-rat" revision="0.9" transitive="false" inline="true" conf="master" type="jar" pathid="rat.classpath"/>
+    <ivy:cachepath organisation="org.apache.rat" module="apache-rat" revision="0.10" transitive="false" inline="true" conf="master" type="jar" pathid="rat.classpath"/>
     <typedef resource="org/apache/rat/anttasks/antlib.xml" uri="antlib:org.apache.rat.anttasks" classpathref="rat.classpath"/>
     <property name="rat.loaded" value="true"/>
   </target>
@@ -1567,6 +1596,8 @@ ${tests-output}/junit4-*.suites     - pe
         <pattern substring="This file was generated automatically by the Snowball to Java compiler"/>
       <!-- uima tests generated by JCasGen -->
         <pattern substring="First created by JCasGen"/>
+      <!-- parsers generated by antlr -->
+        <pattern substring="ANTLR GENERATED CODE"/>
       </rat:substringMatcher>
 
       <!-- built in approved licenses -->
@@ -2064,6 +2095,9 @@ ${tests-output}/junit4-*.suites     - pe
   <target name="resolve-groovy" unless="groovy.loaded" depends="ivy-availability-check,ivy-configure">
     <ivy:cachepath organisation="org.codehaus.groovy" module="groovy-all" revision="2.1.5"
       inline="true" conf="default" type="jar" transitive="true" pathid="groovy.classpath"/>
+    <taskdef name="groovy"
+      classname="org.codehaus.groovy.ant.Groovy"
+      classpathref="groovy.classpath"/>
     <property name="groovy.loaded" value="true"/>
   </target>
   

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/Analyzer.java Mon Oct 21 18:58:24 2013
@@ -72,12 +72,15 @@ public abstract class Analyzer implement
 
   private final ReuseStrategy reuseStrategy;
 
+  // non final as it gets nulled if closed; pkg private for access by ReuseStrategy's final helper methods:
+  CloseableThreadLocal<Object> storedValue = new CloseableThreadLocal<Object>();
+
   /**
    * Create a new Analyzer, reusing the same set of components per-thread
    * across calls to {@link #tokenStream(String, Reader)}. 
    */
   public Analyzer() {
-    this(new GlobalReuseStrategy());
+    this(GLOBAL_REUSE_STRATEGY);
   }
 
   /**
@@ -133,11 +136,11 @@ public abstract class Analyzer implement
    */
   public final TokenStream tokenStream(final String fieldName,
                                        final Reader reader) throws IOException {
-    TokenStreamComponents components = reuseStrategy.getReusableComponents(fieldName);
+    TokenStreamComponents components = reuseStrategy.getReusableComponents(this, fieldName);
     final Reader r = initReader(fieldName, reader);
     if (components == null) {
       components = createComponents(fieldName, r);
-      reuseStrategy.setReusableComponents(fieldName, components);
+      reuseStrategy.setReusableComponents(this, fieldName, components);
     } else {
       components.setReader(r);
     }
@@ -167,7 +170,7 @@ public abstract class Analyzer implement
    * @see #tokenStream(String, Reader)
    */
   public final TokenStream tokenStream(final String fieldName, final String text) throws IOException {
-    TokenStreamComponents components = reuseStrategy.getReusableComponents(fieldName);
+    TokenStreamComponents components = reuseStrategy.getReusableComponents(this, fieldName);
     @SuppressWarnings("resource") final ReusableStringReader strReader = 
         (components == null || components.reusableStringReader == null) ?
         new ReusableStringReader() : components.reusableStringReader;
@@ -175,7 +178,7 @@ public abstract class Analyzer implement
     final Reader r = initReader(fieldName, strReader);
     if (components == null) {
       components = createComponents(fieldName, r);
-      reuseStrategy.setReusableComponents(fieldName, components);
+      reuseStrategy.setReusableComponents(this, fieldName, components);
     } else {
       components.setReader(r);
     }
@@ -229,10 +232,20 @@ public abstract class Analyzer implement
     return 1;
   }
 
+  /**
+   * Returns the used {@link ReuseStrategy}.
+   */
+  public final ReuseStrategy getReuseStrategy() {
+    return reuseStrategy;
+  }
+
   /** Frees persistent resources used by this Analyzer */
   @Override
   public void close() {
-    reuseStrategy.close();
+    if (storedValue != null) {
+      storedValue.close();
+      storedValue = null;
+    }
   }
 
   /**
@@ -317,126 +330,101 @@ public abstract class Analyzer implement
    * Strategy defining how TokenStreamComponents are reused per call to
    * {@link Analyzer#tokenStream(String, java.io.Reader)}.
    */
-  public static abstract class ReuseStrategy implements Closeable {
-
-    private CloseableThreadLocal<Object> storedValue = new CloseableThreadLocal<Object>();
+  public static abstract class ReuseStrategy {
 
     /** Sole constructor. (For invocation by subclass constructors, typically implicit.) */
     public ReuseStrategy() {}
 
     /**
-     * Gets the reusable TokenStreamComponents for the field with the given name
+     * Gets the reusable TokenStreamComponents for the field with the given name.
      *
+     * @param analyzer Analyzer from which to get the reused components. Use
+     *        {@link #getStoredValue(Analyzer)} and {@link #setStoredValue(Analyzer, Object)}
+     *        to access the data on the Analyzer.
      * @param fieldName Name of the field whose reusable TokenStreamComponents
      *        are to be retrieved
      * @return Reusable TokenStreamComponents for the field, or {@code null}
      *         if there was no previous components for the field
      */
-    public abstract TokenStreamComponents getReusableComponents(String fieldName);
+    public abstract TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName);
 
     /**
      * Stores the given TokenStreamComponents as the reusable components for the
-     * field with the give name
+     * field with the give name.
      *
      * @param fieldName Name of the field whose TokenStreamComponents are being set
      * @param components TokenStreamComponents which are to be reused for the field
      */
-    public abstract void setReusableComponents(String fieldName, TokenStreamComponents components);
+    public abstract void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents components);
 
     /**
-     * Returns the currently stored value
+     * Returns the currently stored value.
      *
      * @return Currently stored value or {@code null} if no value is stored
-     * @throws AlreadyClosedException if the ReuseStrategy is closed.
+     * @throws AlreadyClosedException if the Analyzer is closed.
      */
-    protected final Object getStoredValue() {
-      try {
-        return storedValue.get();
-      } catch (NullPointerException npe) {
-        if (storedValue == null) {
-          throw new AlreadyClosedException("this Analyzer is closed");
-        } else {
-          throw npe;
-        }
+    protected final Object getStoredValue(Analyzer analyzer) {
+      if (analyzer.storedValue == null) {
+        throw new AlreadyClosedException("this Analyzer is closed");
       }
+      return analyzer.storedValue.get();
     }
 
     /**
-     * Sets the stored value
+     * Sets the stored value.
      *
      * @param storedValue Value to store
-     * @throws AlreadyClosedException if the ReuseStrategy is closed.
+     * @throws AlreadyClosedException if the Analyzer is closed.
      */
-    protected final void setStoredValue(Object storedValue) {
-      try {
-        this.storedValue.set(storedValue);
-      } catch (NullPointerException npe) {
-        if (storedValue == null) {
-          throw new AlreadyClosedException("this Analyzer is closed");
-        } else {
-          throw npe;
-        }
+    protected final void setStoredValue(Analyzer analyzer, Object storedValue) {
+      if (analyzer.storedValue == null) {
+        throw new AlreadyClosedException("this Analyzer is closed");
       }
+      analyzer.storedValue.set(storedValue);
     }
 
-    /**
-     * Closes the ReuseStrategy, freeing any resources
-     */
-    @Override
-    public void close() {
-      if (storedValue != null) {
-        storedValue.close();
-        storedValue = null;
-      }
-    }
   }
 
   /**
-   * Implementation of {@link ReuseStrategy} that reuses the same components for
+   * A predefined {@link ReuseStrategy}  that reuses the same components for
    * every field.
    */
-  public final static class GlobalReuseStrategy extends ReuseStrategy {
-    
-    /** Creates a new instance, with empty per-thread values */
-    public GlobalReuseStrategy() {}
+  public static final ReuseStrategy GLOBAL_REUSE_STRATEGY = new ReuseStrategy() {
 
     @Override
-    public TokenStreamComponents getReusableComponents(String fieldName) {
-      return (TokenStreamComponents) getStoredValue();
+    public TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName) {
+      return (TokenStreamComponents) getStoredValue(analyzer);
     }
 
     @Override
-    public void setReusableComponents(String fieldName, TokenStreamComponents components) {
-      setStoredValue(components);
+    public void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents components) {
+      setStoredValue(analyzer, components);
     }
-  }
+  };
 
   /**
-   * Implementation of {@link ReuseStrategy} that reuses components per-field by
+   * A predefined {@link ReuseStrategy} that reuses components per-field by
    * maintaining a Map of TokenStreamComponent per field name.
    */
-  public static class PerFieldReuseStrategy extends ReuseStrategy {
-
-    /** Creates a new instance, with empty per-thread-per-field values */
-    public PerFieldReuseStrategy() {}
+  public static final ReuseStrategy PER_FIELD_REUSE_STRATEGY = new ReuseStrategy() {
 
     @SuppressWarnings("unchecked")
     @Override
-    public TokenStreamComponents getReusableComponents(String fieldName) {
-      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue();
+    public TokenStreamComponents getReusableComponents(Analyzer analyzer, String fieldName) {
+      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue(analyzer);
       return componentsPerField != null ? componentsPerField.get(fieldName) : null;
     }
 
     @SuppressWarnings("unchecked")
     @Override
-    public void setReusableComponents(String fieldName, TokenStreamComponents components) {
-      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue();
+    public void setReusableComponents(Analyzer analyzer, String fieldName, TokenStreamComponents components) {
+      Map<String, TokenStreamComponents> componentsPerField = (Map<String, TokenStreamComponents>) getStoredValue(analyzer);
       if (componentsPerField == null) {
         componentsPerField = new HashMap<String, TokenStreamComponents>();
-        setStoredValue(componentsPerField);
+        setStoredValue(analyzer, componentsPerField);
       }
       componentsPerField.put(fieldName, components);
     }
-  }
+  };
 
 }

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/AnalyzerWrapper.java Mon Oct 21 18:58:24 2013
@@ -34,10 +34,27 @@ public abstract class AnalyzerWrapper ex
 
   /**
    * Creates a new AnalyzerWrapper.  Since the {@link Analyzer.ReuseStrategy} of
-   * the wrapped Analyzers are unknown, {@link Analyzer.PerFieldReuseStrategy} is assumed
+   * the wrapped Analyzers are unknown, {@link #PER_FIELD_REUSE_STRATEGY} is assumed.
+   * @deprecated Use {@link #AnalyzerWrapper(Analyzer.ReuseStrategy)}
+   * and specify a valid {@link Analyzer.ReuseStrategy}, probably retrieved from the
+   * wrapped analyzer using {@link #getReuseStrategy()}.
    */
+  @Deprecated
   protected AnalyzerWrapper() {
-    super(new PerFieldReuseStrategy());
+    this(PER_FIELD_REUSE_STRATEGY);
+  }
+
+  /**
+   * Creates a new AnalyzerWrapper with the given reuse strategy.
+   * <p>If you want to wrap a single delegate Analyzer you can probably
+   * reuse its strategy when instantiating this subclass:
+   * {@code super(delegate.getReuseStrategy());}.
+   * <p>If you choose different analyzers per field, use
+   * {@link #PER_FIELD_REUSE_STRATEGY}.
+   * @see #getReuseStrategy()
+   */
+  protected AnalyzerWrapper(ReuseStrategy reuseStrategy) {
+    super(reuseStrategy);
   }
 
   /**

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/NumericTokenStream.java Mon Oct 21 18:58:24 2013
@@ -160,16 +160,10 @@ public final class NumericTokenStream ex
     
     @Override
     public int fillBytesRef() {
-      try {
-        assert valueSize == 64 || valueSize == 32;
-        return (valueSize == 64) ? 
-          NumericUtils.longToPrefixCoded(value, shift, bytes) :
-          NumericUtils.intToPrefixCoded((int) value, shift, bytes);
-      } catch (IllegalArgumentException iae) {
-        // return empty token before first or after last
-        bytes.length = 0;
-        return 0;
-      }
+      assert valueSize == 64 || valueSize == 32;
+      return (valueSize == 64) ? 
+        NumericUtils.longToPrefixCoded(value, shift, bytes) :
+        NumericUtils.intToPrefixCoded((int) value, shift, bytes);
     }
 
     @Override

Modified: lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java?rev=1534320&r1=1534319&r2=1534320&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java (original)
+++ lucene/dev/branches/lucene4956/lucene/core/src/java/org/apache/lucene/analysis/TokenStream.java Mon Oct 21 18:58:24 2013
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.Closeable;
 import java.lang.reflect.Modifier;
 
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriter;
@@ -159,23 +160,41 @@ public abstract class TokenStream extend
    * setting the final offset of a stream. The final offset of a stream might
    * differ from the offset of the last token eg in case one or more whitespaces
    * followed after the last token, but a WhitespaceTokenizer was used.
+   * <p>
+   * Additionally any skipped positions (such as those removed by a stopfilter)
+   * can be applied to the position increment, or any adjustment of other
+   * attributes where the end-of-stream value may be important.
+   * <p>
+   * If you override this method, always call {@code super.end()}.
    * 
    * @throws IOException If an I/O error occurs
    */
   public void end() throws IOException {
-    // do nothing by default
+    clearAttributes(); // LUCENE-3849: don't consume dirty atts
+    if (hasAttribute(PositionIncrementAttribute.class)) {
+      getAttribute(PositionIncrementAttribute.class).setPositionIncrement(0);
+    }
   }
 
   /**
    * This method is called by a consumer before it begins consumption using
    * {@link #incrementToken()}.
-   * <p/>
+   * <p>
    * Resets this stream to a clean state. Stateful implementations must implement
    * this method so that they can be reused, just as if they had been created fresh.
+   * <p>
+   * If you override this method, always call {@code super.reset()}, otherwise
+   * some internal state will not be correctly reset (e.g., {@link Tokenizer} will
+   * throw {@link IllegalStateException} on further usage).
    */
   public void reset() throws IOException {}
   
-  /** Releases resources associated with this stream. */
+  /** Releases resources associated with this stream.
+   * <p>
+   * If you override this method, always call {@code super.close()}, otherwise
+   * some internal state will not be correctly reset (e.g., {@link Tokenizer} will
+   * throw {@link IllegalStateException} on reuse).
+   */
   @Override
   public void close() throws IOException {}