You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2015/02/06 17:39:49 UTC

svn commit: r1657890 [8/10] - in /lucene/dev/branches/branch_5x: ./ dev-tools/ dev-tools/idea/lucene/highlighter/ lucene/ lucene/analysis/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ lucene/analysis/common/src/test/org/apache/lucen...

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java Fri Feb  6 16:39:45 2015
@@ -25,7 +25,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -205,7 +205,7 @@ public final class TermsFilter extends F
     final BytesRef spare = new BytesRef(this.termsBytes);
     Terms terms = null;
     TermsEnum termsEnum = null;
-    DocsEnum docs = null;
+    PostingsEnum docs = null;
     for (TermsAndField termsAndField : this.termsAndFields) {
       if ((terms = fields.terms(termsAndField.field)) != null) {
         termsEnum = terms.iterator(termsEnum); // this won't return null
@@ -213,7 +213,7 @@ public final class TermsFilter extends F
           spare.offset = offsets[i];
           spare.length = offsets[i+1] - offsets[i];
           if (termsEnum.seekExact(spare)) {
-            docs = termsEnum.docs(acceptDocs, docs, DocsEnum.FLAG_NONE); // no freq since we don't need them
+            docs = termsEnum.postings(acceptDocs, docs, PostingsEnum.FLAG_NONE); // no freq since we don't need them
             builder.or(docs);
           }
         }

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java Fri Feb  6 16:39:45 2015
@@ -17,18 +17,24 @@ package org.apache.lucene.queries.functi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.search.*;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.ToStringUtils;
-
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Set;
 import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.ComplexExplanation;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.FilterScorer;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ToStringUtils;
 
 /**
  * Query that is boosted by a ValueSource
@@ -118,41 +124,24 @@ public class BoostedQuery extends Query
   }
 
 
-  private class CustomScorer extends Scorer {
+  private class CustomScorer extends FilterScorer {
     private final BoostedQuery.BoostedWeight weight;
     private final float qWeight;
-    private final Scorer scorer;
     private final FunctionValues vals;
     private final LeafReaderContext readerContext;
 
     private CustomScorer(LeafReaderContext readerContext, BoostedQuery.BoostedWeight w, float qWeight,
         Scorer scorer, ValueSource vs) throws IOException {
-      super(w);
+      super(scorer);
       this.weight = w;
       this.qWeight = qWeight;
-      this.scorer = scorer;
       this.readerContext = readerContext;
       this.vals = vs.getValues(weight.fcontext, readerContext);
     }
 
-    @Override
-    public int docID() {
-      return scorer.docID();
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return scorer.advance(target);
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      return scorer.nextDoc();
-    }
-
     @Override   
     public float score() throws IOException {
-      float score = qWeight * scorer.score() * vals.floatVal(scorer.docID());
+      float score = qWeight * in.score() * vals.floatVal(in.docID());
 
       // Current Lucene priority queues can't handle NaN and -Infinity, so
       // map to -Float.MAX_VALUE. This conditional handles both -infinity
@@ -161,13 +150,8 @@ public class BoostedQuery extends Query
     }
 
     @Override
-    public int freq() throws IOException {
-      return scorer.freq();
-    }
-
-    @Override
     public Collection<ChildScorer> getChildren() {
-      return Collections.singleton(new ChildScorer(scorer, "CUSTOM"));
+      return Collections.singleton(new ChildScorer(in, "CUSTOM"));
     }
 
     public Explanation explain(int doc) throws IOException {
@@ -183,10 +167,6 @@ public class BoostedQuery extends Query
       return res;
     }
 
-    @Override
-    public long cost() {
-      return scorer.cost();
-    }
   }
 
 

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java Fri Feb  6 16:39:45 2015
@@ -17,15 +17,21 @@ package org.apache.lucene.queries.functi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.ComplexExplanation;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
-
-import java.io.IOException;
-import java.util.Set;
-import java.util.Map;
+import org.apache.lucene.util.BytesRef;
 
 
 /**
@@ -162,6 +168,26 @@ public class FunctionQuery extends Query
       return 1;
     }
 
+    @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return null;
+    }
+
     public Explanation explain(int doc) throws IOException {
       float sc = qWeight * vals.floatVal(doc);
 
@@ -173,6 +199,7 @@ public class FunctionQuery extends Query
       result.addDetail(new Explanation(weight.queryNorm,"queryNorm"));
       return result;
     }
+
   }
 
 

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java Fri Feb  6 16:39:45 2015
@@ -17,12 +17,13 @@ package org.apache.lucene.queries.functi
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.util.Bits;
-
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * {@link Scorer} which returns the result of {@link FunctionValues#floatVal(int)} as
@@ -93,6 +94,26 @@ public class ValueSourceScorer extends S
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
+
+  @Override
   public long cost() {
     return maxDoc;
   }

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java Fri Feb  6 16:39:45 2015
@@ -17,19 +17,22 @@
 
 package org.apache.lucene.queries.function.valuesource;
 
+import java.io.IOException;
+import java.util.Map;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.docvalues.FloatDocValues;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueFloat;
 
-import java.io.IOException;
-import java.util.Map;
-
 /**
  * <code>QueryValueSource</code> returns the relevance score of the query
  */

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java Fri Feb  6 16:39:45 2015
@@ -17,7 +17,14 @@ package org.apache.lucene.queries.functi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.*;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.docvalues.FloatDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -25,9 +32,6 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.similarities.TFIDFSimilarity;
 import org.apache.lucene.util.BytesRef;
 
-import java.io.IOException;
-import java.util.Map;
-
 /** 
  * Function that returns {@link TFIDFSimilarity#tf(float)}
  * for every document.
@@ -56,7 +60,7 @@ public class TFValueSource extends TermF
     }
 
     return new FloatDocValues(this) {
-      DocsEnum docs ;
+      PostingsEnum docs ;
       int atDoc;
       int lastDocRequested = -1;
 
@@ -68,7 +72,7 @@ public class TFValueSource extends TermF
         if (terms != null) {
           final TermsEnum termsEnum = terms.iterator(null);
           if (termsEnum.seekExact(indexedBytes)) {
-            docs = termsEnum.docs(null, null);
+            docs = termsEnum.postings(null, null);
           } else {
             docs = null;
           }
@@ -77,13 +81,33 @@ public class TFValueSource extends TermF
         }
 
         if (docs == null) {
-          docs = new DocsEnum() {
+          docs = new PostingsEnum() {
             @Override
             public int freq() {
               return 0;
             }
 
             @Override
+            public int nextPosition() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public int startOffset() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public int endOffset() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public BytesRef getPayload() throws IOException {
+              return null;
+            }
+
+            @Override
             public int docID() {
               return DocIdSetIterator.NO_MORE_DOCS;
             }

Modified: lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java (original)
+++ lucene/dev/branches/branch_5x/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java Fri Feb  6 16:39:45 2015
@@ -17,17 +17,21 @@
 
 package org.apache.lucene.queries.function.valuesource;
 
-import org.apache.lucene.index.*;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.docvalues.IntDocValues;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BytesRef;
 
-import java.io.IOException;
-import java.util.Map;
-
 /**
- * Function that returns {@link DocsEnum#freq()} for the
+ * Function that returns {@link org.apache.lucene.index.PostingsEnum#freq()} for the
  * supplied term in every document.
  * <p>
  * If the term does not exist in the document, returns 0.
@@ -49,7 +53,7 @@ public class TermFreqValueSource extends
     final Terms terms = fields.terms(indexedField);
 
     return new IntDocValues(this) {
-      DocsEnum docs ;
+      PostingsEnum docs ;
       int atDoc;
       int lastDocRequested = -1;
 
@@ -61,7 +65,7 @@ public class TermFreqValueSource extends
         if (terms != null) {
           final TermsEnum termsEnum = terms.iterator(null);
           if (termsEnum.seekExact(indexedBytes)) {
-            docs = termsEnum.docs(null, null);
+            docs = termsEnum.postings(null, null);
           } else {
             docs = null;
           }
@@ -70,13 +74,33 @@ public class TermFreqValueSource extends
         }
 
         if (docs == null) {
-          docs = new DocsEnum() {
+          docs = new PostingsEnum() {
             @Override
             public int freq() {
               return 0;
             }
 
             @Override
+            public int nextPosition() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public int startOffset() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public int endOffset() throws IOException {
+              return -1;
+            }
+
+            @Override
+            public BytesRef getPayload() throws IOException {
+              throw new UnsupportedOperationException();
+            }
+
+            @Override
             public int docID() {
               return DocIdSetIterator.NO_MORE_DOCS;
             }

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java Fri Feb  6 16:39:45 2015
@@ -24,8 +24,7 @@ import java.util.Collections;
 import org.apache.lucene.codecs.BlockTermState;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.PostingsReaderBase;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.DataInput;
@@ -67,9 +66,22 @@ final class IDVersionPostingsReader exte
   }
 
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+  public PostingsEnum postings(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
     SingleDocsEnum docsEnum;
 
+    if (PostingsEnum.requiresPositions(flags)) {
+      SinglePostingsEnum posEnum;
+
+      if (reuse instanceof SinglePostingsEnum) {
+        posEnum = (SinglePostingsEnum) reuse;
+      } else {
+        posEnum = new SinglePostingsEnum();
+      }
+      IDVersionTermState _termState = (IDVersionTermState) termState;
+      posEnum.reset(_termState.docID, _termState.idVersion, liveDocs);
+      return posEnum;
+    }
+
     if (reuse instanceof SingleDocsEnum) {
       docsEnum = (SingleDocsEnum) reuse;
     } else {
@@ -81,21 +93,6 @@ final class IDVersionPostingsReader exte
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, int flags) {
-    SingleDocsAndPositionsEnum posEnum;
-
-    if (reuse instanceof SingleDocsAndPositionsEnum) {
-      posEnum = (SingleDocsAndPositionsEnum) reuse;
-    } else {
-      posEnum = new SingleDocsAndPositionsEnum();
-    }
-    IDVersionTermState termState = (IDVersionTermState) _termState;
-    posEnum.reset(termState.docID, termState.idVersion, liveDocs);
-    return posEnum;
-  }
-
-  @Override
   public long ramBytesUsed() {
     return 0;
   }

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java Fri Feb  6 16:39:45 2015
@@ -21,9 +21,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 
 import org.apache.lucene.codecs.BlockTermState;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;
@@ -997,7 +995,7 @@ public final class IDVersionSegmentTerms
   }
 
   @Override
-  public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException {
+  public PostingsEnum postings(Bits skipDocs, PostingsEnum reuse, int flags) throws IOException {
     assert !eof;
     //if (DEBUG) {
     //System.out.println("BTTR.docs seg=" + segment);
@@ -1006,19 +1004,7 @@ public final class IDVersionSegmentTerms
     //if (DEBUG) {
     //System.out.println("  state=" + currentFrame.state);
     //}
-    return fr.parent.postingsReader.docs(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags);
-  }
-
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-    if (fr.fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-      // Positions were not indexed:
-      return null;
-    }
-
-    assert !eof;
-    currentFrame.decodeMetaData();
-    return fr.parent.postingsReader.docsAndPositions(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags);
+    return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.state, skipDocs, reuse, flags);
   }
 
   @Override

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java Fri Feb  6 16:39:45 2015
@@ -17,10 +17,13 @@ package org.apache.lucene.codecs.idversi
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocsEnum;
+import java.io.IOException;
+
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 
-class SingleDocsEnum extends DocsEnum {
+class SingleDocsEnum extends PostingsEnum {
 
   private int doc;
   private int singleDocID;
@@ -68,4 +71,24 @@ class SingleDocsEnum extends DocsEnum {
   public int freq() {
     return 1;
   }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    throw new UnsupportedOperationException();
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java Fri Feb  6 16:39:45 2015
@@ -18,7 +18,7 @@ package org.apache.lucene.sandbox.querie
 
 import java.io.IOException;
 
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.SlowCompositeReaderWrapper;
@@ -100,13 +100,13 @@ public class DuplicateFilter extends Fil
 
     if (terms != null) {
       TermsEnum termsEnum = terms.iterator(null);
-      DocsEnum docs = null;
+      PostingsEnum docs = null;
       while (true) {
         BytesRef currTerm = termsEnum.next();
         if (currTerm == null) {
           break;
         } else {
-          docs = termsEnum.docs(acceptDocs, docs, DocsEnum.FLAG_NONE);
+          docs = termsEnum.postings(acceptDocs, docs, PostingsEnum.FLAG_NONE);
           int doc = docs.nextDoc();
           if (doc != DocIdSetIterator.NO_MORE_DOCS) {
             if (keepMode == KeepMode.KM_USE_FIRST_OCCURRENCE) {
@@ -136,7 +136,7 @@ public class DuplicateFilter extends Fil
 
     if (terms != null) {
       TermsEnum termsEnum = terms.iterator(null);
-      DocsEnum docs = null;
+      PostingsEnum docs = null;
       while (true) {
         BytesRef currTerm = termsEnum.next();
         if (currTerm == null) {
@@ -144,7 +144,7 @@ public class DuplicateFilter extends Fil
         } else {
           if (termsEnum.docFreq() > 1) {
             // unset potential duplicates
-            docs = termsEnum.docs(acceptDocs, docs, DocsEnum.FLAG_NONE);
+            docs = termsEnum.postings(acceptDocs, docs, PostingsEnum.FLAG_NONE);
             int doc = docs.nextDoc();
             if (doc != DocIdSetIterator.NO_MORE_DOCS) {
               if (keepMode == KeepMode.KM_USE_FIRST_OCCURRENCE) {

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java Fri Feb  6 16:39:45 2015
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.ReaderUtil;
@@ -325,7 +325,7 @@ public class TermAutomatonQuery extends
 
   static class EnumAndScorer {
     public final int termID;
-    public final DocsAndPositionsEnum posEnum;
+    public final PostingsEnum posEnum;
 
     // How many positions left in the current document:
     public int posLeft;
@@ -333,7 +333,7 @@ public class TermAutomatonQuery extends
     // Current position
     public int pos;
 
-    public EnumAndScorer(int termID, DocsAndPositionsEnum posEnum) {
+    public EnumAndScorer(int termID, PostingsEnum posEnum) {
       this.termID = termID;
       this.posEnum = posEnum;
     }
@@ -395,8 +395,7 @@ public class TermAutomatonQuery extends
 
           TermsEnum termsEnum = context.reader().terms(field).iterator(null);
           termsEnum.seekExact(term, state);
-          enums[ent.getKey()] = new EnumAndScorer(ent.getKey(),
-                                                  termsEnum.docsAndPositions(acceptDocs, null, 0));
+          enums[ent.getKey()] = new EnumAndScorer(ent.getKey(), termsEnum.postings(acceptDocs, null, PostingsEnum.FLAG_POSITIONS));
         }
       }
 

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java Fri Feb  6 16:39:45 2015
@@ -326,6 +326,26 @@ class TermAutomatonScorer extends Scorer
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return -1; // TODO can we get positional information out of this Scorer?
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
+
+  @Override
   public int docID() {
     return docID;
   }

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java Fri Feb  6 16:39:45 2015
@@ -30,7 +30,6 @@ import java.util.concurrent.CountDownLat
 import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.Analyzer.TokenStreamComponents;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenFilter;
 import org.apache.lucene.analysis.MockTokenizer;
@@ -39,7 +38,7 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.ConcurrentMergeScheduler;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
@@ -331,9 +330,9 @@ public class TestIDVersionPostingsFormat
           if (VERBOSE) {
             System.out.println("  found in seg=" + termsEnums[seg]);
           }
-          docsEnums[seg] = termsEnums[seg].docs(liveDocs[seg], docsEnums[seg], 0);
-          int docID = docsEnums[seg].nextDoc();
-          if (docID != DocsEnum.NO_MORE_DOCS) {
+          postingsEnums[seg] = termsEnums[seg].postings(liveDocs[seg], postingsEnums[seg], 0);
+          int docID = postingsEnums[seg].nextDoc();
+          if (docID != PostingsEnum.NO_MORE_DOCS) {
             lastVersion = ((IDVersionSegmentTermsEnum) termsEnums[seg]).getVersion();
             return docBases[seg] + docID;
           }

Modified: lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java (original)
+++ lucene/dev/branches/branch_5x/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java Fri Feb  6 16:39:45 2015
@@ -134,7 +134,7 @@ public class DuplicateFilterTest extends
     for (ScoreDoc hit : hits) {
       Document d = searcher.doc(hit.doc);
       String url = d.get(KEY_FIELD);
-      DocsEnum td = TestUtil.docs(random(), reader,
+      PostingsEnum td = TestUtil.docs(random(), reader,
           KEY_FIELD,
           new BytesRef(url),
           MultiFields.getLiveDocs(reader),
@@ -158,7 +158,7 @@ public class DuplicateFilterTest extends
     for (ScoreDoc hit : hits) {
       Document d = searcher.doc(hit.doc);
       String url = d.get(KEY_FIELD);
-      DocsEnum td = TestUtil.docs(random(), reader,
+      PostingsEnum td = TestUtil.docs(random(), reader,
           KEY_FIELD,
           new BytesRef(url),
           MultiFields.getLiveDocs(reader),

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java Fri Feb  6 16:39:45 2015
@@ -20,7 +20,7 @@ package org.apache.lucene.spatial.prefix
 import java.io.IOException;
 
 import com.spatial4j.core.shape.Shape;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Terms;
@@ -80,7 +80,7 @@ public abstract class AbstractPrefixTree
     protected final int maxDoc;
 
     protected TermsEnum termsEnum;//remember to check for null!
-    protected DocsEnum docsEnum;
+    protected PostingsEnum postingsEnum;
 
     public BaseTermsEnumTraverser(LeafReaderContext context, Bits acceptDocs) throws IOException {
       this.context = context;
@@ -94,8 +94,8 @@ public abstract class AbstractPrefixTree
 
     protected void collectDocs(BitSet bitSet) throws IOException {
       assert termsEnum != null;
-      docsEnum = termsEnum.docs(acceptDocs, docsEnum, DocsEnum.FLAG_NONE);
-      bitSet.or(docsEnum);
+      postingsEnum = termsEnum.postings(acceptDocs, postingsEnum, PostingsEnum.FLAG_NONE);
+      bitSet.or(postingsEnum);
     }
 
   }

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Fri Feb  6 16:39:45 2015
@@ -21,7 +21,7 @@ import com.spatial4j.core.shape.Shape;
 import com.spatial4j.core.shape.SpatialRelation;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.spatial.prefix.tree.Cell;
@@ -176,9 +176,9 @@ public class ContainsPrefixTreeFilter ex
     private SmallDocSet collectDocs(Bits acceptContains) throws IOException {
       SmallDocSet set = null;
 
-      docsEnum = termsEnum.docs(acceptContains, docsEnum, DocsEnum.FLAG_NONE);
+      postingsEnum = termsEnum.postings(acceptContains, postingsEnum, PostingsEnum.FLAG_NONE);
       int docid;
-      while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
         if (set == null) {
           int size = termsEnum.docFreq();
           if (size <= 0)

Modified: lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java (original)
+++ lucene/dev/branches/branch_5x/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheProvider.java Fri Feb  6 16:39:45 2015
@@ -61,7 +61,7 @@ public abstract class ShapeFieldCachePro
     log.fine("Building Cache [" + reader.maxDoc() + "]");
     idx = new ShapeFieldCache<>(reader.maxDoc(),defaultSize);
     int count = 0;
-    DocsEnum docs = null;
+    PostingsEnum docs = null;
     Terms terms = reader.terms(shapeField);
     TermsEnum te = null;
     if (terms != null) {
@@ -70,7 +70,7 @@ public abstract class ShapeFieldCachePro
       while (term != null) {
         T shape = readShape(term);
         if( shape != null ) {
-          docs = te.docs(null, docs, DocsEnum.FLAG_NONE);
+          docs = te.postings(null, docs, PostingsEnum.FLAG_NONE);
           Integer docid = docs.nextDoc();
           while (docid != DocIdSetIterator.NO_MORE_DOCS) {
             idx.add( docid, shape );

Modified: lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java (original)
+++ lucene/dev/branches/branch_5x/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/BlendedInfixSuggester.java Fri Feb  6 16:39:45 2015
@@ -29,7 +29,7 @@ import org.apache.lucene.analysis.Analyz
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.Terms;
@@ -281,7 +281,7 @@ public class BlendedInfixSuggester exten
 
       if (matchedTokens.contains(docTerm) || (prefixToken != null && docTerm.startsWith(prefixToken))) {
  
-        DocsAndPositionsEnum docPosEnum = it.docsAndPositions(null, null, DocsAndPositionsEnum.FLAG_OFFSETS);
+        PostingsEnum docPosEnum = it.postings(null, null, PostingsEnum.FLAG_OFFSETS);
         docPosEnum.nextDoc();
 
         // use the first occurrence of the term

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingPostingsFormat.java Fri Feb  6 16:39:45 2015
@@ -25,8 +25,7 @@ import org.apache.lucene.codecs.FieldsCo
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.index.AssertingLeafReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
@@ -160,8 +159,7 @@ public final class AssertingPostingsForm
 
         termsEnum = terms.iterator(termsEnum);
         BytesRefBuilder lastTerm = null;
-        DocsEnum docsEnum = null;
-        DocsAndPositionsEnum posEnum = null;
+        PostingsEnum postingsEnum = null;
 
         boolean hasFreqs = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0;
         boolean hasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
@@ -187,46 +185,46 @@ public final class AssertingPostingsForm
           int flags = 0;
           if (hasPositions == false) {
             if (hasFreqs) {
-              flags = flags | DocsEnum.FLAG_FREQS;
+              flags = flags | PostingsEnum.FLAG_FREQS;
             }
-            docsEnum = termsEnum.docs(null, docsEnum, flags);
+            postingsEnum = termsEnum.postings(null, postingsEnum, flags);
           } else {
+            flags = PostingsEnum.FLAG_POSITIONS;
             if (hasPayloads) {
-              flags |= DocsAndPositionsEnum.FLAG_PAYLOADS;
+              flags |= PostingsEnum.FLAG_PAYLOADS;
             }
             if (hasOffsets) {
-              flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS;
+              flags = flags | PostingsEnum.FLAG_OFFSETS;
             }
-            posEnum = termsEnum.docsAndPositions(null, posEnum, flags);
-            docsEnum = posEnum;
+            postingsEnum = termsEnum.postings(null, postingsEnum, flags);
           }
 
-          assert docsEnum != null : "termsEnum=" + termsEnum + " hasPositions=" + hasPositions;
+          assert postingsEnum != null : "termsEnum=" + termsEnum + " hasPositions=" + hasPositions;
 
           int lastDocID = -1;
 
           while(true) {
-            int docID = docsEnum.nextDoc();
-            if (docID == DocsEnum.NO_MORE_DOCS) {
+            int docID = postingsEnum.nextDoc();
+            if (docID == PostingsEnum.NO_MORE_DOCS) {
               break;
             }
             assert docID > lastDocID;
             lastDocID = docID;
             if (hasFreqs) {
-              int freq = docsEnum.freq();
+              int freq = postingsEnum.freq();
               assert freq > 0;
 
               if (hasPositions) {
                 int lastPos = -1;
                 int lastStartOffset = -1;
                 for(int i=0;i<freq;i++) {
-                  int pos = posEnum.nextPosition();
+                  int pos = postingsEnum.nextPosition();
                   assert pos >= lastPos: "pos=" + pos + " vs lastPos=" + lastPos + " i=" + i + " freq=" + freq;
                   lastPos = pos;
 
                   if (hasOffsets) {
-                    int startOffset = posEnum.startOffset();
-                    int endOffset = posEnum.endOffset();
+                    int startOffset = postingsEnum.startOffset();
+                    int endOffset = postingsEnum.endOffset();
                     assert endOffset >= startOffset;
                     assert startOffset >= lastStartOffset;
                     lastStartOffset = startOffset;

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/codecs/ramonly/RAMOnlyPostingsFormat.java Fri Feb  6 16:39:45 2015
@@ -34,8 +34,7 @@ import org.apache.lucene.codecs.FieldsCo
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.PostingsFormat;
 import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
@@ -267,8 +266,7 @@ public final class RAMOnlyPostingsFormat
         FixedBitSet docsSeen = new FixedBitSet(state.segmentInfo.getDocCount());
         long sumTotalTermFreq = 0;
         long sumDocFreq = 0;
-        DocsEnum docsEnum = null;
-        DocsAndPositionsEnum posEnum = null;
+        PostingsEnum postingsEnum = null;
         int enumFlags;
 
         IndexOptions indexOptions = fieldInfo.getIndexOptions();
@@ -280,18 +278,18 @@ public final class RAMOnlyPostingsFormat
         if (writeFreqs == false) {
           enumFlags = 0;
         } else if (writePositions == false) {
-          enumFlags = DocsEnum.FLAG_FREQS;
+          enumFlags = PostingsEnum.FLAG_FREQS;
         } else if (writeOffsets == false) {
           if (writePayloads) {
-            enumFlags = DocsAndPositionsEnum.FLAG_PAYLOADS;
+            enumFlags = PostingsEnum.FLAG_PAYLOADS;
           } else {
             enumFlags = 0;
           }
         } else {
           if (writePayloads) {
-            enumFlags = DocsAndPositionsEnum.FLAG_PAYLOADS | DocsAndPositionsEnum.FLAG_OFFSETS;
+            enumFlags = PostingsEnum.FLAG_PAYLOADS | PostingsEnum.FLAG_OFFSETS;
           } else {
-            enumFlags = DocsAndPositionsEnum.FLAG_OFFSETS;
+            enumFlags = PostingsEnum.FLAG_OFFSETS;
           }
         }
 
@@ -301,20 +299,13 @@ public final class RAMOnlyPostingsFormat
             break;
           }
           RAMPostingsWriterImpl postingsWriter = termsConsumer.startTerm(term);
-
-          if (writePositions) {
-            posEnum = termsEnum.docsAndPositions(null, posEnum, enumFlags);
-            docsEnum = posEnum;
-          } else {
-            docsEnum = termsEnum.docs(null, docsEnum, enumFlags);
-            posEnum = null;
-          }
+          postingsEnum = termsEnum.postings(null, postingsEnum, enumFlags);
 
           int docFreq = 0;
           long totalTermFreq = 0;
           while (true) {
-            int docID = docsEnum.nextDoc();
-            if (docID == DocsEnum.NO_MORE_DOCS) {
+            int docID = postingsEnum.nextDoc();
+            if (docID == PostingsEnum.NO_MORE_DOCS) {
               break;
             }
             docsSeen.set(docID);
@@ -322,7 +313,7 @@ public final class RAMOnlyPostingsFormat
 
             int freq;
             if (writeFreqs) {
-              freq = docsEnum.freq();
+              freq = postingsEnum.freq();
               totalTermFreq += freq;
             } else {
               freq = -1;
@@ -331,13 +322,13 @@ public final class RAMOnlyPostingsFormat
             postingsWriter.startDoc(docID, freq);
             if (writePositions) {
               for (int i=0;i<freq;i++) {
-                int pos = posEnum.nextPosition();
-                BytesRef payload = writePayloads ? posEnum.getPayload() : null;
+                int pos = postingsEnum.nextPosition();
+                BytesRef payload = writePayloads ? postingsEnum.getPayload() : null;
                 int startOffset;
                 int endOffset;
                 if (writeOffsets) {
-                  startOffset = posEnum.startOffset();
-                  endOffset = posEnum.endOffset();
+                  startOffset = postingsEnum.startOffset();
+                  endOffset = postingsEnum.endOffset();
                 } else {
                   startOffset = -1;
                   endOffset = -1;
@@ -494,17 +485,13 @@ public final class RAMOnlyPostingsFormat
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
       return new RAMDocsEnum(ramField.termToDocs.get(current), liveDocs);
     }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
-      return new RAMDocsAndPositionsEnum(ramField.termToDocs.get(current), liveDocs);
-    }
   }
 
-  private static class RAMDocsEnum extends DocsEnum {
+  private static class RAMDocsEnum extends PostingsEnum {
     private final RAMTerm ramTerm;
     private final Bits liveDocs;
     private RAMDoc current;
@@ -547,59 +534,10 @@ public final class RAMOnlyPostingsFormat
     public int docID() {
       return current.docID;
     }
-    
-    @Override
-    public long cost() {
-      return ramTerm.docs.size();
-    } 
-  }
-
-  private static class RAMDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    private final RAMTerm ramTerm;
-    private final Bits liveDocs;
-    private RAMDoc current;
-    int upto = -1;
-    int posUpto = 0;
-
-    public RAMDocsAndPositionsEnum(RAMTerm ramTerm, Bits liveDocs) {
-      this.ramTerm = ramTerm;
-      this.liveDocs = liveDocs;
-    }
-
-    @Override
-    public int advance(int targetDocID) throws IOException {
-      return slowAdvance(targetDocID);
-    }
-
-    // TODO: override bulk read, for better perf
-    @Override
-    public int nextDoc() {
-      while(true) {
-        upto++;
-        if (upto < ramTerm.docs.size()) {
-          current = ramTerm.docs.get(upto);
-          if (liveDocs == null || liveDocs.get(current.docID)) {
-            posUpto = 0;
-            return current.docID;
-          }
-        } else {
-          return NO_MORE_DOCS;
-        }
-      }
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return current.positions.length;
-    }
-
-    @Override
-    public int docID() {
-      return current.docID;
-    }
 
     @Override
     public int nextPosition() {
+      assert posUpto < current.positions.length;
       return current.positions[posUpto++];
     }
 

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/AssertingLeafReader.java Fri Feb  6 16:39:45 2015
@@ -139,31 +139,17 @@ public class AssertingLeafReader extends
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
       assertThread("Terms enums", creationThread);
       assert state == State.POSITIONED: "docs(...) called on unpositioned TermsEnum";
 
       // TODO: should we give this thing a random to be super-evil,
       // and randomly *not* unwrap?
-      if (reuse instanceof AssertingDocsEnum) {
-        reuse = ((AssertingDocsEnum) reuse).in;
+      if (reuse instanceof AssertingPostingsEnum) {
+        reuse = ((AssertingPostingsEnum) reuse).in;
       }
-      DocsEnum docs = super.docs(liveDocs, reuse, flags);
-      return docs == null ? null : new AssertingDocsEnum(docs);
-    }
-
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      assertThread("Terms enums", creationThread);
-      assert state == State.POSITIONED: "docsAndPositions(...) called on unpositioned TermsEnum";
-
-      // TODO: should we give this thing a random to be super-evil,
-      // and randomly *not* unwrap?
-      if (reuse instanceof AssertingDocsAndPositionsEnum) {
-        reuse = ((AssertingDocsAndPositionsEnum) reuse).in;
-      }
-      DocsAndPositionsEnum docs = super.docsAndPositions(liveDocs, reuse, flags);
-      return docs == null ? null : new AssertingDocsAndPositionsEnum(docs);
+      PostingsEnum docs = super.postings(liveDocs, reuse, flags);
+      return docs == null ? null : new AssertingPostingsEnum(docs);
     }
 
     // TODO: we should separately track if we are 'at the end' ?
@@ -274,12 +260,14 @@ public class AssertingLeafReader extends
   static enum DocsEnumState { START, ITERATING, FINISHED };
 
   /** Wraps a docsenum with additional checks */
-  public static class AssertingDocsEnum extends FilterDocsEnum {
+  public static class AssertingPostingsEnum extends FilterDocsEnum {
     private final Thread creationThread = Thread.currentThread();
     private DocsEnumState state = DocsEnumState.START;
+    int positionCount = 0;
+    int positionMax = 0;
     private int doc;
-    
-    public AssertingDocsEnum(DocsEnum in) {
+
+    public AssertingPostingsEnum(PostingsEnum in) {
       super(in);
       this.doc = in.docID();
     }
@@ -292,75 +280,12 @@ public class AssertingLeafReader extends
       assert nextDoc > doc : "backwards nextDoc from " + doc + " to " + nextDoc + " " + in;
       if (nextDoc == DocIdSetIterator.NO_MORE_DOCS) {
         state = DocsEnumState.FINISHED;
-      } else {
-        state = DocsEnumState.ITERATING;
-      }
-      assert super.docID() == nextDoc;
-      return doc = nextDoc;
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      assertThread("Docs enums", creationThread);
-      assert state != DocsEnumState.FINISHED : "advance() called after NO_MORE_DOCS";
-      assert target > doc : "target must be > docID(), got " + target + " <= " + doc;
-      int advanced = super.advance(target);
-      assert advanced >= target : "backwards advance from: " + target + " to: " + advanced;
-      if (advanced == DocIdSetIterator.NO_MORE_DOCS) {
-        state = DocsEnumState.FINISHED;
-      } else {
-        state = DocsEnumState.ITERATING;
-      }
-      assert super.docID() == advanced;
-      return doc = advanced;
-    }
-
-    @Override
-    public int docID() {
-      assertThread("Docs enums", creationThread);
-      assert doc == super.docID() : " invalid docID() in " + in.getClass() + " " + super.docID() + " instead of " + doc;
-      return doc;
-    }
-
-    @Override
-    public int freq() throws IOException {
-      assertThread("Docs enums", creationThread);
-      assert state != DocsEnumState.START : "freq() called before nextDoc()/advance()";
-      assert state != DocsEnumState.FINISHED : "freq() called after NO_MORE_DOCS";
-      int freq = super.freq();
-      assert freq > 0;
-      return freq;
-    }
-  }
-  
-  static class AssertingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
-    private final Thread creationThread = Thread.currentThread();
-    private DocsEnumState state = DocsEnumState.START;
-    private int positionMax = 0;
-    private int positionCount = 0;
-    private int doc;
-
-    public AssertingDocsAndPositionsEnum(DocsAndPositionsEnum in) {
-      super(in);
-      int docid = in.docID();
-      assert docid == -1 : "invalid initial doc id: " + docid;
-      doc = -1;
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      assertThread("Docs enums", creationThread);
-      assert state != DocsEnumState.FINISHED : "nextDoc() called after NO_MORE_DOCS";
-      int nextDoc = super.nextDoc();
-      assert nextDoc > doc : "backwards nextDoc from " + doc + " to " + nextDoc;
-      positionCount = 0;
-      if (nextDoc == DocIdSetIterator.NO_MORE_DOCS) {
-        state = DocsEnumState.FINISHED;
         positionMax = 0;
       } else {
         state = DocsEnumState.ITERATING;
         positionMax = super.freq();
       }
+      positionCount = 0;
       assert super.docID() == nextDoc;
       return doc = nextDoc;
     }
@@ -372,7 +297,6 @@ public class AssertingLeafReader extends
       assert target > doc : "target must be > docID(), got " + target + " <= " + doc;
       int advanced = super.advance(target);
       assert advanced >= target : "backwards advance from: " + target + " to: " + advanced;
-      positionCount = 0;
       if (advanced == DocIdSetIterator.NO_MORE_DOCS) {
         state = DocsEnumState.FINISHED;
         positionMax = 0;
@@ -380,6 +304,7 @@ public class AssertingLeafReader extends
         state = DocsEnumState.ITERATING;
         positionMax = super.freq();
       }
+      positionCount = 0;
       assert super.docID() == advanced;
       return doc = advanced;
     }
@@ -403,7 +328,6 @@ public class AssertingLeafReader extends
 
     @Override
     public int nextPosition() throws IOException {
-      assertThread("Docs enums", creationThread);
       assert state != DocsEnumState.START : "nextPosition() called before nextDoc()/advance()";
       assert state != DocsEnumState.FINISHED : "nextPosition() called after NO_MORE_DOCS";
       assert positionCount < positionMax : "nextPosition() called more than freq() times!";
@@ -415,7 +339,6 @@ public class AssertingLeafReader extends
 
     @Override
     public int startOffset() throws IOException {
-      assertThread("Docs enums", creationThread);
       assert state != DocsEnumState.START : "startOffset() called before nextDoc()/advance()";
       assert state != DocsEnumState.FINISHED : "startOffset() called after NO_MORE_DOCS";
       assert positionCount > 0 : "startOffset() called before nextPosition()!";
@@ -424,7 +347,6 @@ public class AssertingLeafReader extends
 
     @Override
     public int endOffset() throws IOException {
-      assertThread("Docs enums", creationThread);
       assert state != DocsEnumState.START : "endOffset() called before nextDoc()/advance()";
       assert state != DocsEnumState.FINISHED : "endOffset() called after NO_MORE_DOCS";
       assert positionCount > 0 : "endOffset() called before nextPosition()!";
@@ -433,16 +355,15 @@ public class AssertingLeafReader extends
 
     @Override
     public BytesRef getPayload() throws IOException {
-      assertThread("Docs enums", creationThread);
       assert state != DocsEnumState.START : "getPayload() called before nextDoc()/advance()";
       assert state != DocsEnumState.FINISHED : "getPayload() called after NO_MORE_DOCS";
       assert positionCount > 0 : "getPayload() called before nextPosition()!";
       BytesRef payload = super.getPayload();
-      assert payload == null || payload.isValid() && payload.length > 0 : "getPayload() returned payload with invalid length!";
+      assert payload == null || payload.length > 0 : "getPayload() returned payload with invalid length!";
       return payload;
     }
   }
-  
+
   /** Wraps a NumericDocValues but with additional asserts */
   public static class AssertingNumericDocValues extends NumericDocValues {
     private final Thread creationThread = Thread.currentThread();

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java Fri Feb  6 16:39:45 2015
@@ -17,8 +17,6 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
-
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.PrintStream;
@@ -63,6 +61,8 @@ import org.apache.lucene.util.BytesRefHa
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TestUtil;
 
+import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS;
+
 /**
  * Abstract class to do basic tests for a docvalues format.
  * NOTE: This test focuses on the docvalues impl, nothing else.
@@ -1156,8 +1156,8 @@ public abstract class BaseDocValuesForma
 
     for (Entry<String, String> entry : entrySet) {
       // pk lookup
-      DocsEnum termDocsEnum = slowR.termDocsEnum(new Term("id", entry.getKey()));
-      int docId = termDocsEnum.nextDoc();
+      PostingsEnum termPostingsEnum = slowR.termDocsEnum(new Term("id", entry.getKey()));
+      int docId = termPostingsEnum.nextDoc();
       expected = new BytesRef(entry.getValue());
       final BytesRef actual = docValues.get(docId);
       assertEquals(expected, actual);
@@ -2085,7 +2085,7 @@ public abstract class BaseDocValuesForma
       );
     }
   }
-  
+
   public void testSortedNumericsMultipleValuesVsStoredFields() throws Exception {
     assumeTrue("Codec does not support SORTED_NUMERIC", codecSupportsSortedNumeric());
     int numIterations = atLeast(1);

Modified: lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/test-framework/src/java/org/apache/lucene/index/BasePostingsFormatTestCase.java Fri Feb  6 16:39:45 2015
@@ -97,7 +97,7 @@ public abstract class BasePostingsFormat
     // Sometimes use .advance():
     SKIPPING,
 
-    // Sometimes reuse the Docs/AndPositionsEnum across terms:
+    // Sometimes reuse the PostingsEnum across terms:
     REUSE_ENUMS,
 
     // Sometimes pass non-null live docs:
@@ -121,7 +121,7 @@ public abstract class BasePostingsFormat
 
   /** Given the same random seed this always enumerates the
    *  same random postings */
-  private static class SeedPostings extends DocsAndPositionsEnum {
+  private static class SeedPostings extends PostingsEnum {
     // Used only to generate docIDs; this way if you pull w/
     // or w/o positions you get the same docID sequence:
     private final Random docRandom;
@@ -234,7 +234,7 @@ public abstract class BasePostingsFormat
         return 0;
       }
       assert posUpto < freq;
-
+      
       if (posUpto == 0 && random.nextBoolean()) {
         // Sometimes index pos = 0
       } else if (posSpacing == 1) {
@@ -270,7 +270,7 @@ public abstract class BasePostingsFormat
       posUpto++;
       return pos;
     }
-  
+
     @Override
     public int startOffset() {
       return startOffset;
@@ -414,10 +414,10 @@ public abstract class BasePostingsFormat
 
         // NOTE: sort of silly: we enum all the docs just to
         // get the maxDoc
-        DocsEnum docsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS, true);
+        PostingsEnum postingsEnum = getSeedPostings(term, termSeed, false, IndexOptions.DOCS, true);
         int doc;
         int lastDoc = 0;
-        while((doc = docsEnum.nextDoc()) != DocsEnum.NO_MORE_DOCS) {
+        while((doc = postingsEnum.nextDoc()) != PostingsEnum.NO_MORE_DOCS) {
           lastDoc = doc;
         }
         maxDoc = Math.max(lastDoc, maxDoc);
@@ -639,32 +639,27 @@ public abstract class BasePostingsFormat
     }
 
     @Override
-    public final DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+    public final PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
       if (liveDocs != null) {
         throw new IllegalArgumentException("liveDocs must be null");
       }
-      if ((flags & DocsEnum.FLAG_FREQS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) {
-        return null;
-      }
-      return getSeedPostings(current.getKey().utf8ToString(), current.getValue().seed, false, maxAllowed, allowPayloads);
-    }
-
-    @Override
-    public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      if (liveDocs != null) {
-        throw new IllegalArgumentException("liveDocs must be null");
-      }
-      if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-        return null;
-      }
-      if ((flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
-        return null;
+      if (PostingsEnum.requiresPositions(flags)) {
+        if (maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
+          return null;
+        }
+        if ((flags & PostingsEnum.FLAG_OFFSETS) == PostingsEnum.FLAG_OFFSETS && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) < 0) {
+          return null;
+        }
+        if ((flags & PostingsEnum.FLAG_PAYLOADS) == PostingsEnum.FLAG_PAYLOADS && allowPayloads == false) {
+          return null;
+        }
       }
-      if ((flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0 && allowPayloads == false) {
+      if ((flags & PostingsEnum.FLAG_FREQS) != 0 && maxAllowed.compareTo(IndexOptions.DOCS_AND_FREQS) < 0) {
         return null;
       }
       return getSeedPostings(current.getKey().utf8ToString(), current.getValue().seed, false, maxAllowed, allowPayloads);
     }
+
   }
 
   // TODO maybe instead of @BeforeClass just make a single test run: build postings & index & test it?
@@ -745,8 +740,7 @@ public abstract class BasePostingsFormat
 
   private static class ThreadState {
     // Only used with REUSE option:
-    public DocsEnum reuseDocsEnum;
-    public DocsAndPositionsEnum reuseDocsAndPositionsEnum;
+    public PostingsEnum reusePostingsEnum;
   }
 
   private void verifyEnum(ThreadState threadState,
@@ -809,78 +803,74 @@ public abstract class BasePostingsFormat
 
     boolean doCheckPayloads = options.contains(Option.PAYLOADS) && allowPositions && fieldInfo.hasPayloads() && (alwaysTestMax || random().nextInt(3) <= 2);
 
-    DocsEnum prevDocsEnum = null;
+    PostingsEnum prevPostingsEnum = null;
 
-    DocsEnum docsEnum;
-    DocsAndPositionsEnum docsAndPositionsEnum;
+    PostingsEnum postingsEnum;
 
     if (!doCheckPositions) {
       if (allowPositions && random().nextInt(10) == 7) {
         // 10% of the time, even though we will not check positions, pull a DocsAndPositions enum
         
         if (options.contains(Option.REUSE_ENUMS) && random().nextInt(10) < 9) {
-          prevDocsEnum = threadState.reuseDocsAndPositionsEnum;
+          prevPostingsEnum = threadState.reusePostingsEnum;
         }
 
-        int flags = 0;
+        int flags = PostingsEnum.FLAG_POSITIONS;
         if (alwaysTestMax || random().nextBoolean()) {
-          flags |= DocsAndPositionsEnum.FLAG_OFFSETS;
+          flags |= PostingsEnum.FLAG_OFFSETS;
         }
         if (alwaysTestMax || random().nextBoolean()) {
-          flags |= DocsAndPositionsEnum.FLAG_PAYLOADS;
+          flags |= PostingsEnum.FLAG_PAYLOADS;
         }
 
         if (VERBOSE) {
-          System.out.println("  get DocsAndPositionsEnum (but we won't check positions) flags=" + flags);
+          System.out.println("  get DocsEnum (but we won't check positions) flags=" + flags);
         }
 
-        threadState.reuseDocsAndPositionsEnum = termsEnum.docsAndPositions(liveDocs, (DocsAndPositionsEnum) prevDocsEnum, flags);
-        docsEnum = threadState.reuseDocsAndPositionsEnum;
-        docsAndPositionsEnum = threadState.reuseDocsAndPositionsEnum;
+        threadState.reusePostingsEnum = termsEnum.postings(liveDocs, prevPostingsEnum, flags);
+        postingsEnum = threadState.reusePostingsEnum;
       } else {
         if (VERBOSE) {
           System.out.println("  get DocsEnum");
         }
         if (options.contains(Option.REUSE_ENUMS) && random().nextInt(10) < 9) {
-          prevDocsEnum = threadState.reuseDocsEnum;
+          prevPostingsEnum = threadState.reusePostingsEnum;
         }
-        threadState.reuseDocsEnum = termsEnum.docs(liveDocs, prevDocsEnum, doCheckFreqs ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
-        docsEnum = threadState.reuseDocsEnum;
-        docsAndPositionsEnum = null;
+        threadState.reusePostingsEnum = termsEnum.postings(liveDocs, prevPostingsEnum, doCheckFreqs ? PostingsEnum.FLAG_FREQS : PostingsEnum.FLAG_NONE);
+        postingsEnum = threadState.reusePostingsEnum;
       }
     } else {
       if (options.contains(Option.REUSE_ENUMS) && random().nextInt(10) < 9) {
-        prevDocsEnum = threadState.reuseDocsAndPositionsEnum;
+        prevPostingsEnum = threadState.reusePostingsEnum;
       }
 
-      int flags = 0;
+      int flags = PostingsEnum.FLAG_POSITIONS;
       if (alwaysTestMax || doCheckOffsets || random().nextInt(3) == 1) {
-        flags |= DocsAndPositionsEnum.FLAG_OFFSETS;
+        flags |= PostingsEnum.FLAG_OFFSETS;
       }
       if (alwaysTestMax || doCheckPayloads|| random().nextInt(3) == 1) {
-        flags |= DocsAndPositionsEnum.FLAG_PAYLOADS;
+        flags |= PostingsEnum.FLAG_PAYLOADS;
       }
 
       if (VERBOSE) {
-        System.out.println("  get DocsAndPositionsEnum flags=" + flags);
+        System.out.println("  get DocsEnum flags=" + flags);
       }
 
-      threadState.reuseDocsAndPositionsEnum = termsEnum.docsAndPositions(liveDocs, (DocsAndPositionsEnum) prevDocsEnum, flags);
-      docsEnum = threadState.reuseDocsAndPositionsEnum;
-      docsAndPositionsEnum = threadState.reuseDocsAndPositionsEnum;
+      threadState.reusePostingsEnum = termsEnum.postings(liveDocs, prevPostingsEnum, flags);
+      postingsEnum = threadState.reusePostingsEnum;
     }
 
-    assertNotNull("null DocsEnum", docsEnum);
-    int initialDocID = docsEnum.docID();
-    assertEquals("inital docID should be -1" + docsEnum, -1, initialDocID);
+    assertNotNull("null DocsEnum", postingsEnum);
+    int initialDocID = postingsEnum.docID();
+    assertEquals("inital docID should be -1" + postingsEnum, -1, initialDocID);
 
     if (VERBOSE) {
-      if (prevDocsEnum == null) {
-        System.out.println("  got enum=" + docsEnum);
-      } else if (prevDocsEnum == docsEnum) {
-        System.out.println("  got reuse enum=" + docsEnum);
+      if (prevPostingsEnum == null) {
+        System.out.println("  got enum=" + postingsEnum);
+      } else if (prevPostingsEnum == postingsEnum) {
+        System.out.println("  got reuse enum=" + postingsEnum);
       } else {
-        System.out.println("  got enum=" + docsEnum + " (reuse of " + prevDocsEnum + " failed)");
+        System.out.println("  got enum=" + postingsEnum + " (reuse of " + prevPostingsEnum + " failed)");
       }
     }
 
@@ -930,10 +920,10 @@ public abstract class BasePostingsFormat
     while (expected.upto <= stopAt) {
       if (expected.upto == stopAt) {
         if (stopAt == expected.docFreq) {
-          assertEquals("DocsEnum should have ended but didn't", DocsEnum.NO_MORE_DOCS, docsEnum.nextDoc());
+          assertEquals("DocsEnum should have ended but didn't", PostingsEnum.NO_MORE_DOCS, postingsEnum.nextDoc());
 
           // Common bug is to forget to set this.doc=NO_MORE_DOCS in the enum!:
-          assertEquals("DocsEnum should have ended but didn't", DocsEnum.NO_MORE_DOCS, docsEnum.docID());
+          assertEquals("DocsEnum should have ended but didn't", PostingsEnum.NO_MORE_DOCS, postingsEnum.docID());
         }
         break;
       }
@@ -944,7 +934,7 @@ public abstract class BasePostingsFormat
           // Pick target we know exists:
           final int skipCount = TestUtil.nextInt(random(), 1, skipInc);
           for(int skip=0;skip<skipCount;skip++) {
-            if (expected.nextDoc() == DocsEnum.NO_MORE_DOCS) {
+            if (expected.nextDoc() == PostingsEnum.NO_MORE_DOCS) {
               break;
             }
           }
@@ -958,21 +948,21 @@ public abstract class BasePostingsFormat
         }
 
         if (expected.upto >= stopAt) {
-          int target = random().nextBoolean() ? maxDoc : DocsEnum.NO_MORE_DOCS;
+          int target = random().nextBoolean() ? maxDoc : PostingsEnum.NO_MORE_DOCS;
           if (VERBOSE) {
             System.out.println("  now advance to end (target=" + target + ")");
           }
-          assertEquals("DocsEnum should have ended but didn't", DocsEnum.NO_MORE_DOCS, docsEnum.advance(target));
+          assertEquals("DocsEnum should have ended but didn't", PostingsEnum.NO_MORE_DOCS, postingsEnum.advance(target));
           break;
         } else {
           if (VERBOSE) {
             if (targetDocID != -1) {
-              System.out.println("  now advance to random target=" + targetDocID + " (" + expected.upto + " of " + stopAt + ") current=" + docsEnum.docID());
+              System.out.println("  now advance to random target=" + targetDocID + " (" + expected.upto + " of " + stopAt + ") current=" + postingsEnum.docID());
             } else {
-              System.out.println("  now advance to known-exists target=" + expected.docID() + " (" + expected.upto + " of " + stopAt + ") current=" + docsEnum.docID());
+              System.out.println("  now advance to known-exists target=" + expected.docID() + " (" + expected.upto + " of " + stopAt + ") current=" + postingsEnum.docID());
             }
           }
-          int docID = docsEnum.advance(targetDocID != -1 ? targetDocID : expected.docID());
+          int docID = postingsEnum.advance(targetDocID != -1 ? targetDocID : expected.docID());
           assertEquals("docID is wrong", expected.docID(), docID);
         }
       } else {
@@ -980,9 +970,9 @@ public abstract class BasePostingsFormat
         if (VERBOSE) {
           System.out.println("  now nextDoc to " + expected.docID() + " (" + expected.upto + " of " + stopAt + ")");
         }
-        int docID = docsEnum.nextDoc();
+        int docID = postingsEnum.nextDoc();
         assertEquals("docID is wrong", expected.docID(), docID);
-        if (docID == DocsEnum.NO_MORE_DOCS) {
+        if (docID == PostingsEnum.NO_MORE_DOCS) {
           break;
         }
       }
@@ -991,12 +981,12 @@ public abstract class BasePostingsFormat
         if (VERBOSE) {
           System.out.println("    now freq()=" + expected.freq());
         }
-        int freq = docsEnum.freq();
+        int freq = postingsEnum.freq();
         assertEquals("freq is wrong", expected.freq(), freq);
       }
 
       if (doCheckPositions) {
-        int freq = docsEnum.freq();
+        int freq = postingsEnum.freq();
         int numPosToConsume;
         if (!alwaysTestMax && options.contains(Option.PARTIAL_POS_CONSUME) && random().nextInt(5) == 1) {
           numPosToConsume = random().nextInt(freq);
@@ -1009,7 +999,7 @@ public abstract class BasePostingsFormat
           if (VERBOSE) {
             System.out.println("    now nextPosition to " + pos);
           }
-          assertEquals("position is wrong", pos, docsAndPositionsEnum.nextPosition());
+          assertEquals("position is wrong", pos, postingsEnum.nextPosition());
 
           if (doCheckPayloads) {
             BytesRef expectedPayload = expected.getPayload();
@@ -1018,9 +1008,9 @@ public abstract class BasePostingsFormat
                 System.out.println("      now check expectedPayload length=" + (expectedPayload == null ? 0 : expectedPayload.length));
               }
               if (expectedPayload == null || expectedPayload.length == 0) {
-                assertNull("should not have payload", docsAndPositionsEnum.getPayload());
+                assertNull("should not have payload", postingsEnum.getPayload());
               } else {
-                BytesRef payload = docsAndPositionsEnum.getPayload();
+                BytesRef payload = postingsEnum.getPayload();
                 assertNotNull("should have payload but doesn't", payload);
 
                 assertEquals("payload length is wrong", expectedPayload.length, payload.length);
@@ -1032,7 +1022,7 @@ public abstract class BasePostingsFormat
                 
                 // make a deep copy
                 payload = BytesRef.deepCopyOf(payload);
-                assertEquals("2nd call to getPayload returns something different!", payload, docsAndPositionsEnum.getPayload());
+                assertEquals("2nd call to getPayload returns something different!", payload, postingsEnum.getPayload());
               }
             } else {
               if (VERBOSE) {
@@ -1046,8 +1036,8 @@ public abstract class BasePostingsFormat
               if (VERBOSE) {
                 System.out.println("      now check offsets: startOff=" + expected.startOffset() + " endOffset=" + expected.endOffset());
               }
-              assertEquals("startOffset is wrong", expected.startOffset(), docsAndPositionsEnum.startOffset());
-              assertEquals("endOffset is wrong", expected.endOffset(), docsAndPositionsEnum.endOffset());
+              assertEquals("startOffset is wrong", expected.startOffset(), postingsEnum.startOffset());
+              assertEquals("endOffset is wrong", expected.endOffset(), postingsEnum.endOffset());
             } else {
               if (VERBOSE) {
                 System.out.println("      skip check offsets");
@@ -1057,8 +1047,8 @@ public abstract class BasePostingsFormat
             if (VERBOSE) {
               System.out.println("      now check offsets are -1");
             }
-            assertEquals("startOffset isn't -1", -1, docsAndPositionsEnum.startOffset());
-            assertEquals("endOffset isn't -1", -1, docsAndPositionsEnum.endOffset());
+            assertEquals("startOffset isn't -1", -1, postingsEnum.startOffset());
+            assertEquals("endOffset isn't -1", -1, postingsEnum.endOffset());
           }
         }
       }
@@ -1406,6 +1396,41 @@ public abstract class BasePostingsFormat
       IOUtils.rm(path);
     }
   }
+
+  protected boolean isPostingsEnumReuseImplemented() {
+    return true;
+  }
+
+  public void testPostingsEnumReuse() throws Exception {
+
+    Path path = createTempDir("testPostingsEnumReuse");
+    Directory dir = newFSDirectory(path);
+
+    FieldsProducer fieldsProducer = buildIndex(dir, IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, random().nextBoolean(), true);
+    Collections.shuffle(allTerms, random());
+    FieldAndTerm fieldAndTerm = allTerms.get(0);
+
+    Terms terms = fieldsProducer.terms(fieldAndTerm.field);
+    TermsEnum te = terms.iterator(null);
+
+    te.seekExact(fieldAndTerm.term);
+    checkReuse(te, PostingsEnum.FLAG_FREQS, PostingsEnum.FLAG_ALL, false);
+    if (isPostingsEnumReuseImplemented())
+      checkReuse(te, PostingsEnum.FLAG_ALL, PostingsEnum.FLAG_ALL, true);
+
+    fieldsProducer.close();
+    dir.close();
+    IOUtils.rm(path);
+  }
+
+  protected static void checkReuse(TermsEnum termsEnum, int firstFlags, int secondFlags, boolean shouldReuse) throws IOException {
+    PostingsEnum postings1 = termsEnum.postings(null, null, firstFlags);
+    PostingsEnum postings2 = termsEnum.postings(null, postings1, secondFlags);
+    if (shouldReuse)
+      assertSame("Expected PostingsEnum " + postings1.getClass().getName() + " to be reused", postings1, postings2);
+    else
+      assertNotSame("Expected PostingsEnum " + postings1.getClass().getName() + " to not be reused", postings1, postings2);
+  }
   
   public void testJustEmptyField() throws Exception {
     Directory dir = newDirectory();
@@ -1470,7 +1495,7 @@ public abstract class BasePostingsFormat
     LeafReader ar = getOnlySegmentReader(ir);
     TermsEnum termsEnum = ar.terms("field").iterator(null);
     assertTrue(termsEnum.seekExact(new BytesRef("value")));
-    DocsEnum docsEnum = termsEnum.docs(null, null, DocsEnum.FLAG_NONE);
+    PostingsEnum docsEnum = termsEnum.postings(null, null, DocsEnum.FLAG_NONE);
     assertEquals(0, docsEnum.nextDoc());
     assertEquals(1, docsEnum.freq());
     assertEquals(1, docsEnum.nextDoc());
@@ -1506,8 +1531,8 @@ public abstract class BasePostingsFormat
       TermsEnum termsEnum = terms.iterator(null);
       BytesRef term = termsEnum.next();
       if (term != null) {
-        DocsEnum docsEnum = termsEnum.docs(null, null);
-        assertTrue(docsEnum.nextDoc() == DocsEnum.NO_MORE_DOCS);
+        PostingsEnum postingsEnum = termsEnum.postings(null, null);
+        assertTrue(postingsEnum.nextDoc() == PostingsEnum.NO_MORE_DOCS);
       }
     }
     ir.close();
@@ -1592,27 +1617,24 @@ public abstract class BasePostingsFormat
                     assert terms != null;
 
                     TermsEnum termsEnum = terms.iterator(null);
-                    DocsEnum docs = null;
+                    PostingsEnum docs = null;
                     while(termsEnum.next() != null) {
                       BytesRef term = termsEnum.term();
-
-                      if (random().nextBoolean()) {
-                        docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
-                      } else if (docs instanceof DocsAndPositionsEnum) {
-                        docs = termsEnum.docsAndPositions(null, (DocsAndPositionsEnum) docs, 0);
+                      boolean noPositions = random().nextBoolean();
+                      if (noPositions) {
+                        docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_FREQS);
                       } else {
-                        docs = termsEnum.docsAndPositions(null, null, 0);
+                        docs = termsEnum.postings(null, null, PostingsEnum.FLAG_POSITIONS);
                       }
                       int docFreq = 0;
                       long totalTermFreq = 0;
-                      while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+                      while (docs.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
                         docFreq++;
                         totalTermFreq += docs.freq();
-                        if (docs instanceof DocsAndPositionsEnum) {
-                          DocsAndPositionsEnum posEnum = (DocsAndPositionsEnum) docs;
-                          int limit = TestUtil.nextInt(random(), 1, docs.freq());
-                          for(int i=0;i<limit;i++) {
-                            posEnum.nextPosition();
+                        int limit = TestUtil.nextInt(random(), 1, docs.freq());
+                        if (!noPositions) {
+                          for (int i = 0; i < limit; i++) {
+                            docs.nextPosition();
                           }
                         }
                       }
@@ -1646,24 +1668,22 @@ public abstract class BasePostingsFormat
                     // Also test seeking the TermsEnum:
                     for(String term : termFreqs.keySet()) {
                       if (termsEnum.seekExact(new BytesRef(term))) {
-                        if (random().nextBoolean()) {
-                          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_FREQS);
-                        } else if (docs instanceof DocsAndPositionsEnum) {
-                          docs = termsEnum.docsAndPositions(null, (DocsAndPositionsEnum) docs, 0);
+                        boolean noPositions = random().nextBoolean();
+                        if (noPositions) {
+                          docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_FREQS);
                         } else {
-                          docs = termsEnum.docsAndPositions(null, null, 0);
+                          docs = termsEnum.postings(null, null, PostingsEnum.FLAG_POSITIONS);
                         }
 
                         int docFreq = 0;
                         long totalTermFreq = 0;
-                        while (docs.nextDoc() != DocsEnum.NO_MORE_DOCS) {
+                        while (docs.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
                           docFreq++;
                           totalTermFreq += docs.freq();
-                          if (docs instanceof DocsAndPositionsEnum) {
-                            DocsAndPositionsEnum posEnum = (DocsAndPositionsEnum) docs;
-                            int limit = TestUtil.nextInt(random(), 1, docs.freq());
-                            for(int i=0;i<limit;i++) {
-                              posEnum.nextPosition();
+                          int limit = TestUtil.nextInt(random(), 1, docs.freq());
+                          if (!noPositions) {
+                            for (int i = 0; i < limit; i++) {
+                              docs.nextPosition();
                             }
                           }
                         }