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 14:25:25 UTC

svn commit: r1657800 [7/9] - in /lucene/dev/trunk: dev-tools/idea/lucene/highlighter/ lucene/ lucene/analysis/common/src/test/org/apache/lucene/analysis/core/ lucene/analysis/common/src/test/org/apache/lucene/analysis/sinks/ lucene/analysis/common/src/...

Modified: lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java (original)
+++ lucene/dev/trunk/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java Fri Feb  6 13:25:22 2015
@@ -39,7 +39,7 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
@@ -733,17 +733,17 @@ public class TestJoinUtil extends Lucene
         LeafReader slowCompositeReader = SlowCompositeReaderWrapper.wrap(toSearcher.getIndexReader());
         Terms terms = slowCompositeReader.terms(toField);
         if (terms != null) {
-          DocsEnum docsEnum = null;
+          PostingsEnum postingsEnum = null;
           TermsEnum termsEnum = null;
           SortedSet<BytesRef> joinValues = new TreeSet<>(BytesRef.getUTF8SortedAsUnicodeComparator());
           joinValues.addAll(joinValueToJoinScores.keySet());
           for (BytesRef joinValue : joinValues) {
             termsEnum = terms.iterator(termsEnum);
             if (termsEnum.seekExact(joinValue)) {
-              docsEnum = termsEnum.docs(slowCompositeReader.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
+              postingsEnum = termsEnum.postings(slowCompositeReader.getLiveDocs(), postingsEnum, PostingsEnum.FLAG_NONE);
               JoinScore joinScore = joinValueToJoinScores.get(joinValue);
 
-              for (int doc = docsEnum.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = docsEnum.nextDoc()) {
+              for (int doc = postingsEnum.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = postingsEnum.nextDoc()) {
                 // First encountered join value determines the score.
                 // Something to keep in mind for many-to-many relations.
                 if (!docToJoinScore.containsKey(doc)) {
@@ -853,9 +853,9 @@ public class TestJoinUtil extends Lucene
         }
 
         for (RandomDoc otherSideDoc : otherMatchingDocs) {
-          DocsEnum docsEnum = MultiFields.getTermDocsEnum(topLevelReader, MultiFields.getLiveDocs(topLevelReader), "id", new BytesRef(otherSideDoc.id), 0);
-          assert docsEnum != null;
-          int doc = docsEnum.nextDoc();
+          PostingsEnum postingsEnum = MultiFields.getTermDocsEnum(topLevelReader, MultiFields.getLiveDocs(topLevelReader), "id", new BytesRef(otherSideDoc.id), 0);
+          assert postingsEnum != null;
+          int doc = postingsEnum.nextDoc();
           expectedResult.set(doc);
         }
       }

Modified: lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java (original)
+++ lucene/dev/trunk/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java Fri Feb  6 13:25:22 2015
@@ -34,8 +34,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValuesType;
-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.FieldInfos;
 import org.apache.lucene.index.FieldInvertState;
@@ -983,20 +982,12 @@ public class MemoryIndex {
       }
 
       @Override
-      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
-        if (reuse == null || !(reuse instanceof MemoryDocsEnum)) {
-          reuse = new MemoryDocsEnum();
-        }
-        return ((MemoryDocsEnum) reuse).reset(liveDocs, info.sliceArray.freq[info.sortedTerms[termUpto]]);
-      }
-
-      @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
-        if (reuse == null || !(reuse instanceof MemoryDocsAndPositionsEnum)) {
-          reuse = new MemoryDocsAndPositionsEnum();
+      public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
+        if (reuse == null || !(reuse instanceof MemoryPostingsEnum)) {
+          reuse = new MemoryPostingsEnum();
         }
         final int ord = info.sortedTerms[termUpto];
-        return ((MemoryDocsAndPositionsEnum) reuse).reset(liveDocs, info.sliceArray.start[ord], info.sliceArray.end[ord], info.sliceArray.freq[ord]);
+        return ((MemoryPostingsEnum) reuse).reset(liveDocs, info.sliceArray.start[ord], info.sliceArray.end[ord], info.sliceArray.freq[ord]);
       }
 
       @Override
@@ -1013,69 +1004,26 @@ public class MemoryIndex {
       }
     }
     
-    private class MemoryDocsEnum extends DocsEnum {
-      private boolean hasNext;
-      private Bits liveDocs;
-      private int doc = -1;
-      private int freq;
+    private class MemoryPostingsEnum extends PostingsEnum {
 
-      public DocsEnum reset(Bits liveDocs, int freq) {
-        this.liveDocs = liveDocs;
-        hasNext = true;
-        doc = -1;
-        this.freq = freq;
-        return this;
-      }
-
-      @Override
-      public int docID() {
-        return doc;
-      }
-
-      @Override
-      public int nextDoc() {
-        if (hasNext && (liveDocs == null || liveDocs.get(0))) {
-          hasNext = false;
-          return doc = 0;
-        } else {
-          return doc = NO_MORE_DOCS;
-        }
-      }
-
-      @Override
-      public int advance(int target) throws IOException {
-        return slowAdvance(target);
-      }
-
-      @Override
-      public int freq() throws IOException {
-        return freq;
-      }
-
-      @Override
-      public long cost() {
-        return 1;
-      }
-    }
-    
-    private class MemoryDocsAndPositionsEnum extends DocsAndPositionsEnum {
       private final SliceReader sliceReader;
       private int posUpto; // for assert
       private boolean hasNext;
       private Bits liveDocs;
       private int doc = -1;
       private int freq;
+      private int pos;
       private int startOffset;
       private int endOffset;
       private int payloadIndex;
       private final BytesRefBuilder payloadBuilder;//only non-null when storePayloads
 
-      public MemoryDocsAndPositionsEnum() {
+      public MemoryPostingsEnum() {
         this.sliceReader = new SliceReader(intBlockPool);
         this.payloadBuilder = storePayloads ? new BytesRefBuilder() : null;
       }
 
-      public DocsAndPositionsEnum reset(Bits liveDocs, int start, int end, int freq) {
+      public PostingsEnum reset(Bits liveDocs, int start, int end, int freq) {
         this.liveDocs = liveDocs;
         this.sliceReader.reset(start, end);
         posUpto = 0; // for assert
@@ -1093,6 +1041,7 @@ public class MemoryIndex {
 
       @Override
       public int nextDoc() {
+        pos = -1;
         if (hasNext && (liveDocs == null || liveDocs.get(0))) {
           hasNext = false;
           return doc = 0;
@@ -1113,10 +1062,12 @@ public class MemoryIndex {
 
       @Override
       public int nextPosition() {
-        assert posUpto++ < freq;
+        posUpto++;
+        assert posUpto <= freq;
         assert !sliceReader.endOfSlice() : " stores offsets : " + startOffset;
         int pos = sliceReader.readInt();
         if (storeOffsets) {
+          //pos = sliceReader.readInt();
           startOffset = sliceReader.readInt();
           endOffset = sliceReader.readInt();
         }

Modified: lucene/dev/trunk/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java (original)
+++ lucene/dev/trunk/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java Fri Feb  6 13:25:22 2015
@@ -42,8 +42,7 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.CompositeReader;
 import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
@@ -199,9 +198,9 @@ public class TestMemoryIndexAgainstRAMDi
           while(iwTermsIter.next() != null) {
             assertNotNull(memTermsIter.next());
             assertEquals(iwTermsIter.term(), memTermsIter.term());
-            DocsAndPositionsEnum iwDocsAndPos = iwTermsIter.docsAndPositions(null, null);
-            DocsAndPositionsEnum memDocsAndPos = memTermsIter.docsAndPositions(null, null);
-            while(iwDocsAndPos.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS) {
+            PostingsEnum iwDocsAndPos = iwTermsIter.postings(null, null, PostingsEnum.FLAG_ALL);
+            PostingsEnum memDocsAndPos = memTermsIter.postings(null, null, PostingsEnum.FLAG_ALL);
+            while(iwDocsAndPos.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
               assertEquals(iwDocsAndPos.docID(), memDocsAndPos.nextDoc());
               assertEquals(iwDocsAndPos.freq(), memDocsAndPos.freq());
               for (int i = 0; i < iwDocsAndPos.freq(); i++) {
@@ -222,9 +221,9 @@ public class TestMemoryIndexAgainstRAMDi
         } else {
           while(iwTermsIter.next() != null) {
             assertEquals(iwTermsIter.term(), memTermsIter.term());
-            DocsEnum iwDocsAndPos = iwTermsIter.docs(null, null);
-            DocsEnum memDocsAndPos = memTermsIter.docs(null, null);
-            while(iwDocsAndPos.nextDoc() != DocsAndPositionsEnum.NO_MORE_DOCS) {
+            PostingsEnum iwDocsAndPos = iwTermsIter.postings(null, null);
+            PostingsEnum memDocsAndPos = memTermsIter.postings(null, null);
+            while(iwDocsAndPos.nextDoc() != PostingsEnum.NO_MORE_DOCS) {
               assertEquals(iwDocsAndPos.docID(), memDocsAndPos.nextDoc());
               assertEquals(iwDocsAndPos.freq(), memDocsAndPos.freq());
             }
@@ -319,7 +318,7 @@ public class TestMemoryIndexAgainstRAMDi
     MemoryIndex memory = new MemoryIndex(random().nextBoolean(), false, random().nextInt(50) * 1024 * 1024);
     memory.addField("foo", "bar", analyzer);
     LeafReader reader = (LeafReader) memory.createSearcher().getIndexReader();
-    DocsEnum disi = TestUtil.docs(random(), reader, "foo", new BytesRef("bar"), null, null, DocsEnum.FLAG_NONE);
+    PostingsEnum disi = TestUtil.docs(random(), reader, "foo", new BytesRef("bar"), null, null, PostingsEnum.FLAG_NONE);
     int docid = disi.docID();
     assertEquals(-1, docid);
     assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@@ -327,7 +326,7 @@ public class TestMemoryIndexAgainstRAMDi
     // now reuse and check again
     TermsEnum te = reader.terms("foo").iterator(null);
     assertTrue(te.seekExact(new BytesRef("bar")));
-    disi = te.docs(null, disi, DocsEnum.FLAG_NONE);
+    disi = te.postings(null, disi, PostingsEnum.FLAG_NONE);
     docid = disi.docID();
     assertEquals(-1, docid);
     assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@@ -354,7 +353,7 @@ public class TestMemoryIndexAgainstRAMDi
       memory.addField("foo", "bar", analyzer);
       LeafReader reader = (LeafReader) memory.createSearcher().getIndexReader();
       assertEquals(1, reader.terms("foo").getSumTotalTermFreq());
-      DocsAndPositionsEnum disi = reader.termPositionsEnum(new Term("foo", "bar"));
+      PostingsEnum disi = reader.termDocsEnum(new Term("foo", "bar"), PostingsEnum.FLAG_ALL);
       int docid = disi.docID();
       assertEquals(-1, docid);
       assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@@ -365,7 +364,7 @@ public class TestMemoryIndexAgainstRAMDi
       // now reuse and check again
       TermsEnum te = reader.terms("foo").iterator(null);
       assertTrue(te.seekExact(new BytesRef("bar")));
-      disi = te.docsAndPositions(null, disi);
+      disi = te.postings(null, disi);
       docid = disi.docID();
       assertEquals(-1, docid);
       assertTrue(disi.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
@@ -426,7 +425,7 @@ public class TestMemoryIndexAgainstRAMDi
     assertNull(reader.getNumericDocValues("not-in-index"));
     assertNull(reader.getNormValues("not-in-index"));
     assertNull(reader.termDocsEnum(new Term("not-in-index", "foo")));
-    assertNull(reader.termPositionsEnum(new Term("not-in-index", "foo")));
+    assertNull(reader.termDocsEnum(new Term("not-in-index", "foo"), PostingsEnum.FLAG_ALL));
     assertNull(reader.terms("not-in-index"));
   }
 
@@ -526,8 +525,8 @@ public class TestMemoryIndexAgainstRAMDi
       assertNotNull(memTermEnum.next());
       assertThat(termEnum.totalTermFreq(), equalTo(memTermEnum.totalTermFreq()));
 
-      DocsAndPositionsEnum docsPosEnum = termEnum.docsAndPositions(null, null, 0);
-      DocsAndPositionsEnum memDocsPosEnum = memTermEnum.docsAndPositions(null, null, 0);
+      PostingsEnum docsPosEnum = termEnum.postings(null, null, PostingsEnum.FLAG_POSITIONS);
+      PostingsEnum memDocsPosEnum = memTermEnum.postings(null, null, PostingsEnum.FLAG_POSITIONS);
       String currentTerm = termEnum.term().utf8ToString();
 
       assertThat("Token mismatch for field: " + field_name, currentTerm, equalTo(memTermEnum.term().utf8ToString()));

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/Sorter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/Sorter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/Sorter.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/Sorter.java Fri Feb  6 13:25:22 2015
@@ -24,6 +24,7 @@ import org.apache.lucene.search.LeafFiel
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.TimSorter;
 import org.apache.lucene.util.packed.PackedInts;
 import org.apache.lucene.util.packed.PackedLongValues;
@@ -259,24 +260,60 @@ final class Sorter {
   }
   
   static final Scorer FAKESCORER = new Scorer(null) {
-    
+
+    float score;
+    int doc = -1;
+    int freq = 1;
+
     @Override
-    public float score() throws IOException { throw new UnsupportedOperationException(); }
-    
+    public int docID() {
+      return doc;
+    }
+
     @Override
-    public int freq() throws IOException { throw new UnsupportedOperationException(); }
+    public int nextDoc() throws IOException {
+      throw new UnsupportedOperationException();
+    }
 
     @Override
-    public int docID() { throw new UnsupportedOperationException(); }
+    public int advance(int target) throws IOException {
+      throw new UnsupportedOperationException();
+    }
 
     @Override
-    public int nextDoc() throws IOException { throw new UnsupportedOperationException(); }
+    public long cost() {
+      throw new UnsupportedOperationException();
+    }
 
     @Override
-    public int advance(int target) throws IOException { throw new UnsupportedOperationException(); }
+    public int freq() throws IOException {
+      return freq;
+    }
 
     @Override
-    public long cost() { throw new UnsupportedOperationException(); }
+    public int nextPosition() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
   };
   
 }

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/index/SortingLeafReader.java Fri Feb  6 13:25:22 2015
@@ -20,21 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.index.FilterLeafReader;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.StoredFieldVisitor;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.Sorter.DocMap;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Sort;
@@ -52,7 +37,7 @@ import org.apache.lucene.util.automaton.
 /**
  * An {@link org.apache.lucene.index.LeafReader} which supports sorting documents by a given
  * {@link Sort}. You can use this class to sort an index as follows:
- * 
+ *
  * <pre class="prettyprint">
  * IndexWriter writer; // writer to which the sorted index will be added
  * DirectoryReader reader; // reader on the input index
@@ -62,7 +47,7 @@ import org.apache.lucene.util.automaton.
  * writer.close();
  * reader.close();
  * </pre>
- * 
+ *
  * @lucene.experimental
  */
 public class SortingLeafReader extends FilterLeafReader {
@@ -94,7 +79,7 @@ public class SortingLeafReader extends F
 
     private final Sorter.DocMap docMap;
     private final IndexOptions indexOptions;
-    
+
     public SortingTerms(final Terms in, IndexOptions indexOptions, final Sorter.DocMap docMap) {
       super(in);
       this.docMap = docMap;
@@ -118,7 +103,7 @@ public class SortingLeafReader extends F
 
     final Sorter.DocMap docMap; // pkg-protected to avoid synthetic accessor methods
     private final IndexOptions indexOptions;
-    
+
     public SortingTermsEnum(final TermsEnum in, Sorter.DocMap docMap, IndexOptions indexOptions) {
       super(in);
       this.docMap = docMap;
@@ -145,8 +130,35 @@ public class SortingLeafReader extends F
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, final int flags) throws IOException {
-      final DocsEnum inReuse;
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, final int flags) throws IOException {
+
+      if (PostingsEnum.requiresPositions(flags)) {
+        final PostingsEnum inReuse;
+        final SortingPostingsEnum wrapReuse;
+        if (reuse != null && reuse instanceof SortingPostingsEnum) {
+          // if we're asked to reuse the given DocsEnum and it is Sorting, return
+          // the wrapped one, since some Codecs expect it.
+          wrapReuse = (SortingPostingsEnum) reuse;
+          inReuse = wrapReuse.getWrapped();
+        } else {
+          wrapReuse = null;
+          inReuse = reuse;
+        }
+
+        final PostingsEnum inDocsAndPositions = in.postings(newToOld(liveDocs), inReuse, flags);
+        if (inDocsAndPositions == null) {
+          return null;
+        }
+
+        // we ignore the fact that offsets may be stored but not asked for,
+        // since this code is expected to be used during addIndexes which will
+        // ask for everything. if that assumption changes in the future, we can
+        // factor in whether 'flags' says offsets are not required.
+        final boolean storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
+        return new SortingPostingsEnum(docMap.size(), wrapReuse, inDocsAndPositions, docMap, storeOffsets);
+      }
+
+      final PostingsEnum inReuse;
       final SortingDocsEnum wrapReuse;
       if (reuse != null && reuse instanceof SortingDocsEnum) {
         // if we're asked to reuse the given DocsEnum and it is Sorting, return
@@ -158,45 +170,18 @@ public class SortingLeafReader extends F
         inReuse = reuse;
       }
 
-      final DocsEnum inDocs = in.docs(newToOld(liveDocs), inReuse, flags);
-      final boolean withFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >=0 && (flags & DocsEnum.FLAG_FREQS) != 0;
+      final PostingsEnum inDocs = in.postings(newToOld(liveDocs), inReuse, flags);
+      final boolean withFreqs = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS) >=0 && (flags & PostingsEnum.FLAG_FREQS) != 0;
       return new SortingDocsEnum(docMap.size(), wrapReuse, inDocs, withFreqs, docMap);
     }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, final int flags) throws IOException {
-      final DocsAndPositionsEnum inReuse;
-      final SortingDocsAndPositionsEnum wrapReuse;
-      if (reuse != null && reuse instanceof SortingDocsAndPositionsEnum) {
-        // if we're asked to reuse the given DocsEnum and it is Sorting, return
-        // the wrapped one, since some Codecs expect it.
-        wrapReuse = (SortingDocsAndPositionsEnum) reuse;
-        inReuse = wrapReuse.getWrapped();
-      } else {
-        wrapReuse = null;
-        inReuse = reuse;
-      }
-
-      final DocsAndPositionsEnum inDocsAndPositions = in.docsAndPositions(newToOld(liveDocs), inReuse, flags);
-      if (inDocsAndPositions == null) {
-        return null;
-      }
-
-      // we ignore the fact that offsets may be stored but not asked for,
-      // since this code is expected to be used during addIndexes which will
-      // ask for everything. if that assumption changes in the future, we can
-      // factor in whether 'flags' says offsets are not required.
-      final boolean storeOffsets = indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
-      return new SortingDocsAndPositionsEnum(docMap.size(), wrapReuse, inDocsAndPositions, docMap, storeOffsets);
-    }
-
   }
 
   private static class SortingBinaryDocValues extends BinaryDocValues {
-    
+
     private final BinaryDocValues in;
     private final Sorter.DocMap docMap;
-    
+
     SortingBinaryDocValues(BinaryDocValues in, Sorter.DocMap docMap) {
       this.in = in;
       this.docMap = docMap;
@@ -207,7 +192,7 @@ public class SortingLeafReader extends F
       return in.get(docMap.newToOld(docID));
     }
   }
-  
+
   private static class SortingNumericDocValues extends NumericDocValues {
 
     private final NumericDocValues in;
@@ -223,33 +208,33 @@ public class SortingLeafReader extends F
       return in.get(docMap.newToOld(docID));
     }
   }
-  
+
   private static class SortingSortedNumericDocValues extends SortedNumericDocValues {
-    
+
     private final SortedNumericDocValues in;
     private final Sorter.DocMap docMap;
-    
+
     SortingSortedNumericDocValues(SortedNumericDocValues in, DocMap docMap) {
       this.in = in;
       this.docMap = docMap;
     }
-    
+
     @Override
     public int count() {
       return in.count();
     }
-    
+
     @Override
     public void setDocument(int doc) {
       in.setDocument(docMap.newToOld(doc));
     }
-    
+
     @Override
     public long valueAt(int index) {
       return in.valueAt(index);
     }
   }
-  
+
   private static class SortingBits implements Bits {
 
     private final Bits in;
@@ -270,12 +255,12 @@ public class SortingLeafReader extends F
       return in.length();
     }
   }
-  
+
   private static class SortingSortedDocValues extends SortedDocValues {
-    
+
     private final SortedDocValues in;
     private final Sorter.DocMap docMap;
-    
+
     SortingSortedDocValues(SortedDocValues in, Sorter.DocMap docMap) {
       this.in = in;
       this.docMap = docMap;
@@ -306,12 +291,12 @@ public class SortingLeafReader extends F
       return in.lookupTerm(key);
     }
   }
-  
+
   private static class SortingSortedSetDocValues extends SortedSetDocValues {
-    
+
     private final SortedSetDocValues in;
     private final Sorter.DocMap docMap;
-    
+
     SortingSortedSetDocValues(SortedSetDocValues in, Sorter.DocMap docMap) {
       this.in = in;
       this.docMap = docMap;
@@ -344,14 +329,14 @@ public class SortingLeafReader extends F
   }
 
   static class SortingDocsEnum extends FilterDocsEnum {
-    
+
     private static final class DocFreqSorter extends TimSorter {
-      
+
       private int[] docs;
       private int[] freqs;
       private final int[] tmpDocs;
       private int[] tmpFreqs;
-      
+
       public DocFreqSorter(int maxDoc) {
         super(maxDoc / 64);
         this.tmpDocs = new int[maxDoc / 64];
@@ -369,13 +354,13 @@ public class SortingLeafReader extends F
       protected int compare(int i, int j) {
         return docs[i] - docs[j];
       }
-      
+
       @Override
       protected void swap(int i, int j) {
         int tmpDoc = docs[i];
         docs[i] = docs[j];
         docs[j] = tmpDoc;
-        
+
         if (freqs != null) {
           int tmpFreq = freqs[i];
           freqs[i] = freqs[j];
@@ -421,7 +406,7 @@ public class SortingLeafReader extends F
     private final int upto;
     private final boolean withFreqs;
 
-    SortingDocsEnum(int maxDoc, SortingDocsEnum reuse, final DocsEnum in, boolean withFreqs, final Sorter.DocMap docMap) throws IOException {
+    SortingDocsEnum(int maxDoc, SortingDocsEnum reuse, final PostingsEnum in, boolean withFreqs, final Sorter.DocMap docMap) throws IOException {
       super(in);
       this.maxDoc = maxDoc;
       this.withFreqs = withFreqs;
@@ -470,7 +455,7 @@ public class SortingLeafReader extends F
     }
 
     // for testing
-    boolean reused(DocsEnum other) {
+    boolean reused(PostingsEnum other) {
       if (other == null || !(other instanceof SortingDocsEnum)) {
         return false;
       }
@@ -483,43 +468,43 @@ public class SortingLeafReader extends F
       // don't bother to implement efficiently for now.
       return slowAdvance(target);
     }
-    
+
     @Override
     public int docID() {
       return docIt < 0 ? -1 : docIt >= upto ? NO_MORE_DOCS : docs[docIt];
     }
-    
+
     @Override
     public int freq() throws IOException {
       return withFreqs && docIt < upto ? freqs[docIt] : 1;
     }
-    
+
     @Override
     public int nextDoc() throws IOException {
       if (++docIt >= upto) return NO_MORE_DOCS;
       return docs[docIt];
     }
-    
-    /** Returns the wrapped {@link DocsEnum}. */
-    DocsEnum getWrapped() {
+
+    /** Returns the wrapped {@link PostingsEnum}. */
+    PostingsEnum getWrapped() {
       return in;
     }
   }
-  
-  static class SortingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
-    
+
+  static class SortingPostingsEnum extends FilterDocsEnum {
+
     /**
      * A {@link TimSorter} which sorts two parallel arrays of doc IDs and
      * offsets in one go. Everytime a doc ID is 'swapped', its corresponding offset
      * is swapped too.
      */
     private static final class DocOffsetSorter extends TimSorter {
-      
+
       private int[] docs;
       private long[] offsets;
       private final int[] tmpDocs;
       private final long[] tmpOffsets;
-      
+
       public DocOffsetSorter(int maxDoc) {
         super(maxDoc / 64);
         this.tmpDocs = new int[maxDoc / 64];
@@ -535,13 +520,13 @@ public class SortingLeafReader extends F
       protected int compare(int i, int j) {
         return docs[i] - docs[j];
       }
-      
+
       @Override
       protected void swap(int i, int j) {
         int tmpDoc = docs[i];
         docs[i] = docs[j];
         docs[j] = tmpDoc;
-        
+
         long tmpOffset = offsets[i];
         offsets[i] = offsets[j];
         offsets[j] = tmpOffset;
@@ -570,16 +555,16 @@ public class SortingLeafReader extends F
         return tmpDocs[i] - docs[j];
       }
     }
-    
+
     private final int maxDoc;
     private final DocOffsetSorter sorter;
     private int[] docs;
     private long[] offsets;
     private final int upto;
-    
+
     private final IndexInput postingInput;
     private final boolean storeOffsets;
-    
+
     private int docIt = -1;
     private int pos;
     private int startOffset = -1;
@@ -589,7 +574,7 @@ public class SortingLeafReader extends F
 
     private final RAMFile file;
 
-    SortingDocsAndPositionsEnum(int maxDoc, SortingDocsAndPositionsEnum reuse, final DocsAndPositionsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
+    SortingPostingsEnum(int maxDoc, SortingPostingsEnum reuse, final PostingsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
       super(in);
       this.maxDoc = maxDoc;
       this.storeOffsets = storeOffsets;
@@ -632,14 +617,14 @@ public class SortingLeafReader extends F
     }
 
     // for testing
-    boolean reused(DocsAndPositionsEnum other) {
-      if (other == null || !(other instanceof SortingDocsAndPositionsEnum)) {
+    boolean reused(PostingsEnum other) {
+      if (other == null || !(other instanceof SortingPostingsEnum)) {
         return false;
       }
-      return docs == ((SortingDocsAndPositionsEnum) other).docs;
+      return docs == ((SortingPostingsEnum) other).docs;
     }
 
-    private void addPositions(final DocsAndPositionsEnum in, final IndexOutput out) throws IOException {
+    private void addPositions(final PostingsEnum in, final IndexOutput out) throws IOException {
       int freq = in.freq();
       out.writeVInt(freq);
       int previousPosition = 0;
@@ -648,7 +633,7 @@ public class SortingLeafReader extends F
         final int pos = in.nextPosition();
         final BytesRef payload = in.getPayload();
         // The low-order bit of token is set only if there is a payload, the
-        // previous bits are the delta-encoded position. 
+        // previous bits are the delta-encoded position.
         final int token = (pos - previousPosition) << 1 | (payload == null ? 0 : 1);
         out.writeVInt(token);
         previousPosition = pos;
@@ -665,34 +650,34 @@ public class SortingLeafReader extends F
         }
       }
     }
-    
+
     @Override
     public int advance(final int target) throws IOException {
       // need to support it for checkIndex, but in practice it won't be called, so
       // don't bother to implement efficiently for now.
       return slowAdvance(target);
     }
-    
+
     @Override
     public int docID() {
       return docIt < 0 ? -1 : docIt >= upto ? NO_MORE_DOCS : docs[docIt];
     }
-    
+
     @Override
     public int endOffset() throws IOException {
       return endOffset;
     }
-    
+
     @Override
     public int freq() throws IOException {
       return currFreq;
     }
-    
+
     @Override
     public BytesRef getPayload() throws IOException {
       return payload.length == 0 ? null : payload;
     }
-    
+
     @Override
     public int nextDoc() throws IOException {
       if (++docIt >= upto) return DocIdSetIterator.NO_MORE_DOCS;
@@ -703,7 +688,7 @@ public class SortingLeafReader extends F
       endOffset = 0;
       return docs[docIt];
     }
-    
+
     @Override
     public int nextPosition() throws IOException {
       final int token = postingInput.readVInt();
@@ -724,14 +709,14 @@ public class SortingLeafReader extends F
       }
       return pos;
     }
-    
+
     @Override
     public int startOffset() throws IOException {
       return startOffset;
     }
 
-    /** Returns the wrapped {@link DocsAndPositionsEnum}. */
-    DocsAndPositionsEnum getWrapped() {
+    /** Returns the wrapped {@link PostingsEnum}. */
+    PostingsEnum getWrapped() {
       return in;
     }
   }
@@ -767,12 +752,12 @@ public class SortingLeafReader extends F
   public void document(final int docID, final StoredFieldVisitor visitor) throws IOException {
     in.document(docMap.newToOld(docID), visitor);
   }
-  
+
   @Override
   public Fields fields() throws IOException {
     return new SortingFields(in.fields(), in.getFieldInfos(), docMap);
   }
-  
+
   @Override
   public BinaryDocValues getBinaryDocValues(String field) throws IOException {
     BinaryDocValues oldDocValues = in.getBinaryDocValues(field);
@@ -782,7 +767,7 @@ public class SortingLeafReader extends F
       return new SortingBinaryDocValues(oldDocValues, docMap);
     }
   }
-  
+
   @Override
   public Bits getLiveDocs() {
     final Bits inLiveDocs = in.getLiveDocs();
@@ -792,7 +777,7 @@ public class SortingLeafReader extends F
       return new SortingBits(inLiveDocs, docMap);
     }
   }
-  
+
   @Override
   public NumericDocValues getNormValues(String field) throws IOException {
     final NumericDocValues norm = in.getNormValues(field);
@@ -809,7 +794,7 @@ public class SortingLeafReader extends F
     if (oldDocValues == null) return null;
     return new SortingNumericDocValues(oldDocValues, docMap);
   }
-  
+
   @Override
   public SortedNumericDocValues getSortedNumericDocValues(String field)
       throws IOException {
@@ -830,7 +815,7 @@ public class SortingLeafReader extends F
       return new SortingSortedDocValues(sortedDV, docMap);
     }
   }
-  
+
   @Override
   public SortedSetDocValues getSortedSetDocValues(String field) throws IOException {
     SortedSetDocValues sortedSetDV = in.getSortedSetDocValues(field);
@@ -838,7 +823,7 @@ public class SortingLeafReader extends F
       return null;
     } else {
       return new SortingSortedSetDocValues(sortedSetDV, docMap);
-    }  
+    }
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/DocTermOrds.java Fri Feb  6 13:25:22 2015
@@ -25,8 +25,7 @@ import java.util.List;
 import org.apache.lucene.codecs.PostingsFormat; // javadocs
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
-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.LeafReader;
 import org.apache.lucene.index.SortedSetDocValues;
@@ -165,7 +164,7 @@ public class DocTermOrds implements Acco
   protected int ordBase;
 
   /** Used while uninverting. */
-  protected DocsEnum docsEnum;
+  protected PostingsEnum postingsEnum;
 
   /** Returns total bytes used. */
   public long ramBytesUsed() {
@@ -326,7 +325,7 @@ public class DocTermOrds implements Acco
     // frequent terms ahead of time.
 
     int termNum = 0;
-    docsEnum = null;
+    postingsEnum = null;
 
     // Loop begins with te positioned to first term (we call
     // seek above):
@@ -366,13 +365,13 @@ public class DocTermOrds implements Acco
       final int df = te.docFreq();
       if (df <= maxTermDocFreq) {
 
-        docsEnum = te.docs(liveDocs, docsEnum, DocsEnum.FLAG_NONE);
+        postingsEnum = te.postings(liveDocs, postingsEnum, PostingsEnum.FLAG_NONE);
 
         // dF, but takes deletions into account
         int actualDF = 0;
 
         for (;;) {
-          int doc = docsEnum.nextDoc();
+          int doc = postingsEnum.nextDoc();
           if (doc == DocIdSetIterator.NO_MORE_DOCS) {
             break;
           }
@@ -613,13 +612,8 @@ public class DocTermOrds implements Acco
     }
 
     @Override    
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-      return termsEnum.docs(liveDocs, reuse, flags);
-    }
-
-    @Override    
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      return termsEnum.docsAndPositions(liveDocs, reuse, flags);
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+      return termsEnum.postings(liveDocs, reuse, flags);
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java (original)
+++ lucene/dev/trunk/lucene/misc/src/java/org/apache/lucene/uninverting/FieldCacheImpl.java Fri Feb  6 13:25:22 2015
@@ -30,7 +30,7 @@ import java.util.WeakHashMap;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.DocValuesType;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.LeafReader;
@@ -280,7 +280,7 @@ class FieldCacheImpl implements FieldCac
 
         final TermsEnum termsEnum = termsEnum(terms);
 
-        DocsEnum docs = null;
+        PostingsEnum docs = null;
         FixedBitSet docsWithField = null;
         while(true) {
           final BytesRef term = termsEnum.next();
@@ -288,7 +288,7 @@ class FieldCacheImpl implements FieldCac
             break;
           }
           visitTerm(term);
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
+          docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_NONE);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -408,7 +408,7 @@ class FieldCacheImpl implements FieldCac
           return new BitsEntry(new Bits.MatchAllBits(maxDoc));
         }
         final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
+        PostingsEnum docs = null;
         while(true) {
           final BytesRef term = termsEnum.next();
           if (term == null) {
@@ -419,7 +419,7 @@ class FieldCacheImpl implements FieldCac
             res = new FixedBitSet(maxDoc);
           }
 
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
+          docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_NONE);
           // TODO: use bulk API
           while (true) {
             final int docID = docs.nextDoc();
@@ -686,7 +686,7 @@ class FieldCacheImpl implements FieldCac
 
       if (terms != null) {
         final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
+        PostingsEnum docs = null;
 
         while(true) {
           final BytesRef term = termsEnum.next();
@@ -698,7 +698,7 @@ class FieldCacheImpl implements FieldCac
           }
 
           termOrdToBytesOffset.add(bytes.copyUsingLengthPrefix(term));
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
+          docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_NONE);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
@@ -836,7 +836,7 @@ class FieldCacheImpl implements FieldCac
       if (terms != null) {
         int termCount = 0;
         final TermsEnum termsEnum = terms.iterator(null);
-        DocsEnum docs = null;
+        PostingsEnum docs = null;
         while(true) {
           if (termCount++ == termCountHardLimit) {
             // app is misusing the API (there is more than
@@ -850,7 +850,7 @@ class FieldCacheImpl implements FieldCac
             break;
           }
           final long pointer = bytes.copyUsingLengthPrefix(term);
-          docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
+          docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_NONE);
           while (true) {
             final int docID = docs.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {

Modified: lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java (original)
+++ lucene/dev/trunk/lucene/misc/src/test/org/apache/lucene/index/SorterTestBase.java Fri Feb  6 13:25:22 2015
@@ -31,8 +31,8 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
@@ -40,27 +40,8 @@ import org.apache.lucene.document.Sorted
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.BinaryDocValues;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.FieldInvertState;
-import org.apache.lucene.index.IndexOptions;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SlowCompositeReaderWrapper;
-import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
-import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum.SeekStatus;
-import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.SortingLeafReader.SortingDocsAndPositionsEnum;
 import org.apache.lucene.index.SortingLeafReader.SortingDocsEnum;
+import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.TermStatistics;
@@ -254,7 +235,7 @@ public abstract class SorterTestBase ext
   public void testDocsAndPositionsEnum() throws Exception {
     TermsEnum termsEnum = sortedReader.terms(DOC_POSITIONS_FIELD).iterator(null);
     assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOC_POSITIONS_TERM)));
-    DocsAndPositionsEnum sortedPositions = termsEnum.docsAndPositions(null, null);
+    PostingsEnum sortedPositions = termsEnum.postings(null, null, PostingsEnum.FLAG_ALL);
     int doc;
     
     // test nextDoc()
@@ -270,10 +251,10 @@ public abstract class SorterTestBase ext
     }
     
     // test advance()
-    final DocsAndPositionsEnum reuse = sortedPositions;
-    sortedPositions = termsEnum.docsAndPositions(null, reuse);
-    if (sortedPositions instanceof SortingDocsAndPositionsEnum) {
-      assertTrue(((SortingDocsAndPositionsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
+    final PostingsEnum reuse = sortedPositions;
+    sortedPositions = termsEnum.postings(null, reuse, PostingsEnum.FLAG_ALL);
+    if (sortedPositions instanceof SortingDocsEnum) {
+      assertTrue(((SortingDocsEnum) sortedPositions).reused(reuse)); // make sure reuse worked
     }
     doc = 0;
     while ((doc = sortedPositions.advance(doc + TestUtil.nextInt(random(), 1, 5))) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -315,7 +296,7 @@ public abstract class SorterTestBase ext
     Bits mappedLiveDocs = randomLiveDocs(sortedReader.maxDoc());
     TermsEnum termsEnum = sortedReader.terms(DOCS_ENUM_FIELD).iterator(null);
     assertEquals(SeekStatus.FOUND, termsEnum.seekCeil(new BytesRef(DOCS_ENUM_TERM)));
-    DocsEnum docs = termsEnum.docs(mappedLiveDocs, null);
+    PostingsEnum docs = termsEnum.postings(mappedLiveDocs, null);
 
     int doc;
     int prev = -1;
@@ -330,8 +311,8 @@ public abstract class SorterTestBase ext
       assertFalse("document " + prev + " not marked as deleted", mappedLiveDocs == null || mappedLiveDocs.get(prev));
     }
 
-    DocsEnum reuse = docs;
-    docs = termsEnum.docs(mappedLiveDocs, reuse);
+    PostingsEnum reuse = docs;
+    docs = termsEnum.postings(mappedLiveDocs, reuse);
     if (docs instanceof SortingDocsEnum) {
       assertTrue(((SortingDocsEnum) docs).reused(reuse)); // make sure reuse worked
     }

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/CustomScoreQuery.java Fri Feb  6 13:25:22 2015
@@ -18,23 +18,24 @@ package org.apache.lucene.queries;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
-import java.util.Arrays;
 
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.ValueSource;
 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.Weight;
 import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
 /**
@@ -286,6 +287,8 @@ public class CustomScoreQuery extends Qu
     private final CustomScoreProvider provider;
     private final float[] vScores; // reused in score() to avoid allocating this array for each doc
 
+    // TODO : can we use FilterScorer here instead?
+
     // constructor
     private CustomScorer(CustomScoreProvider provider, CustomWeight w, float qWeight,
         Scorer subQueryScorer, Scorer[] valSrcScorers) {
@@ -328,6 +331,26 @@ public class CustomScoreQuery extends Qu
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return subQueryScorer.nextPosition();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return subQueryScorer.startOffset();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return subQueryScorer.endOffset();
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return subQueryScorer.getPayload();
+    }
+
+    @Override
     public Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(subQueryScorer, "CUSTOM"));
     }

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermFilter.java Fri Feb  6 13:25:22 2015
@@ -18,7 +18,7 @@ package org.apache.lucene.queries;
  */
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -69,7 +69,7 @@ final public class TermFilter extends Fi
     return new DocIdSet() {
       @Override
       public DocIdSetIterator iterator() throws IOException {
-        return termsEnum.docs(acceptDocs, null, DocsEnum.FLAG_NONE);
+        return termsEnum.postings(acceptDocs, null, PostingsEnum.FLAG_NONE);
       }
 
       @Override

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/TermsFilter.java Fri Feb  6 13:25:22 2015
@@ -24,7 +24,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;
@@ -199,7 +199,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
@@ -207,7 +207,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/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/BoostedQuery.java Fri Feb  6 13:25:22 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
@@ -122,41 +128,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
@@ -165,13 +154,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 {
@@ -187,10 +171,6 @@ public class BoostedQuery extends Query
       return res;
     }
 
-    @Override
-    public long cost() {
-      return scorer.cost();
-    }
   }
 
 

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/FunctionQuery.java Fri Feb  6 13:25:22 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;
 
 
 /**
@@ -166,6 +172,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);
 
@@ -177,6 +203,7 @@ public class FunctionQuery extends Query
       result.addDetail(new Explanation(weight.queryNorm,"queryNorm"));
       return result;
     }
+
   }
 
 

Modified: lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/ValueSourceScorer.java Fri Feb  6 13:25:22 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/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/QueryValueSource.java Fri Feb  6 13:25:22 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/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TFValueSource.java Fri Feb  6 13:25:22 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/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java (original)
+++ lucene/dev/trunk/lucene/queries/src/java/org/apache/lucene/queries/function/valuesource/TermFreqValueSource.java Fri Feb  6 13:25:22 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/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsReader.java Fri Feb  6 13:25:22 2015
@@ -22,8 +22,7 @@ import java.io.IOException;
 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;
@@ -64,9 +63,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 {
@@ -78,21 +90,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/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionSegmentTermsEnum.java Fri Feb  6 13:25:22 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/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SingleDocsEnum.java Fri Feb  6 13:25:22 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();
+  }
 }

Added: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SinglePostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SinglePostingsEnum.java?rev=1657800&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SinglePostingsEnum.java (added)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/SinglePostingsEnum.java Fri Feb  6 13:25:22 2015
@@ -0,0 +1,105 @@
+package org.apache.lucene.codecs.idversion;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+class SinglePostingsEnum extends PostingsEnum {
+  private int doc;
+  private int pos;
+  private int singleDocID;
+  private Bits liveDocs;
+  private long version;
+  private final BytesRef payload;
+
+  public SinglePostingsEnum() {
+    payload = new BytesRef(8);
+    payload.length = 8;
+  }
+
+  /** For reuse */
+  public void reset(int singleDocID, long version, Bits liveDocs) {
+    doc = -1;
+    this.liveDocs = liveDocs;
+    this.singleDocID = singleDocID;
+    this.version = version;
+  }
+
+  @Override
+  public int nextDoc() {
+    if (doc == -1 && (liveDocs == null || liveDocs.get(singleDocID))) {
+      doc = singleDocID;
+    } else {
+      doc = NO_MORE_DOCS;
+    }
+    pos = -1;
+    
+    return doc;
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) {
+    if (doc == -1 && target <= singleDocID && (liveDocs == null || liveDocs.get(singleDocID))) {
+      doc = singleDocID;
+      pos = -1;
+    } else {
+      doc = NO_MORE_DOCS;
+    }
+    return doc;
+  }
+
+  @Override
+  public long cost() {
+    return 1;
+  }
+
+  @Override
+  public int freq() {
+    return 1;
+  }
+
+  @Override
+  public int nextPosition() {
+    assert pos == -1;
+    pos = 0;
+    IDVersionPostingsFormat.longToBytes(version, payload);
+    return pos;
+  }
+
+  @Override
+  public BytesRef getPayload() {
+    return payload;
+  }
+
+  @Override
+  public int startOffset() {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() {
+    return -1;
+  }
+}

Modified: lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/sandbox/queries/DuplicateFilter.java Fri Feb  6 13:25:22 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/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonQuery.java Fri Feb  6 13:25:22 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;
     }
@@ -399,8 +399,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/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/java/org/apache/lucene/search/TermAutomatonScorer.java Fri Feb  6 13:25:22 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/trunk/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/codecs/idversion/TestIDVersionPostingsFormat.java Fri Feb  6 13:25:22 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/trunk/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java (original)
+++ lucene/dev/trunk/lucene/sandbox/src/test/org/apache/lucene/sandbox/queries/DuplicateFilterTest.java Fri Feb  6 13:25:22 2015
@@ -134,7 +134,7 @@ public class DuplicateFilterTest extends
     for (ScoreDoc hit : hits) {
       StoredDocument 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) {
       StoredDocument 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/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeFilter.java Fri Feb  6 13:25:22 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/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/trunk/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Fri Feb  6 13:25:22 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)