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 [2/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/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsReader.java Fri Feb  6 13:25:22 2015
@@ -17,16 +17,6 @@ package org.apache.lucene.codecs.simplet
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
-import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
-
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Collection;
@@ -37,11 +27,11 @@ import java.util.Map;
 import java.util.TreeMap;
 
 import org.apache.lucene.codecs.FieldsProducer;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -68,11 +58,21 @@ import org.apache.lucene.util.fst.PairOu
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
 
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.DOC;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.END_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FIELD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.FREQ;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.PAYLOAD;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.POS;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.START_OFFSET;
+import static org.apache.lucene.codecs.simpletext.SimpleTextFieldsWriter.TERM;
+
 class SimpleTextFieldsReader extends FieldsProducer {
 
   private static final long BASE_RAM_BYTES_USED =
-        RamUsageEstimator.shallowSizeOfInstance(SimpleTextFieldsReader.class)
-      + RamUsageEstimator.shallowSizeOfInstance(TreeMap.class);
+      RamUsageEstimator.shallowSizeOfInstance(SimpleTextFieldsReader.class)
+          + RamUsageEstimator.shallowSizeOfInstance(TreeMap.class);
 
   private final TreeMap<String,Long> fields;
   private final IndexInput in;
@@ -93,12 +93,12 @@ class SimpleTextFieldsReader extends Fie
       }
     }
   }
-  
+
   private TreeMap<String,Long> readFields(IndexInput in) throws IOException {
     ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
     BytesRefBuilder scratch = new BytesRefBuilder();
     TreeMap<String,Long> fields = new TreeMap<>();
-    
+
     while (true) {
       SimpleTextUtil.readLine(input, scratch);
       if (scratch.get().equals(END)) {
@@ -206,9 +206,26 @@ class SimpleTextFieldsReader extends Fie
     public long totalTermFreq() {
       return indexOptions == IndexOptions.DOCS ? -1 : totalTermFreq;
     }
- 
+
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+
+      if (PostingsEnum.requiresPositions(flags)) {
+        if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
+          // Positions were not indexed
+          return null;
+        }
+
+        SimpleTextPostingsEnum docsAndPositionsEnum;
+        if (reuse != null && reuse instanceof SimpleTextPostingsEnum && ((SimpleTextPostingsEnum) reuse).canReuse(SimpleTextFieldsReader.this.in)) {
+          docsAndPositionsEnum = (SimpleTextPostingsEnum) reuse;
+        } else {
+          docsAndPositionsEnum = new SimpleTextPostingsEnum();
+        }
+        return docsAndPositionsEnum.reset(docsStart, liveDocs, indexOptions, docFreq);
+
+      }
+
       SimpleTextDocsEnum docsEnum;
       if (reuse != null && reuse instanceof SimpleTextDocsEnum && ((SimpleTextDocsEnum) reuse).canReuse(SimpleTextFieldsReader.this.in)) {
         docsEnum = (SimpleTextDocsEnum) reuse;
@@ -218,22 +235,6 @@ class SimpleTextFieldsReader extends Fie
       return docsEnum.reset(docsStart, liveDocs, indexOptions == IndexOptions.DOCS, docFreq);
     }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-
-      if (indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
-        // Positions were not indexed
-        return null;
-      }
-
-      SimpleTextDocsAndPositionsEnum docsAndPositionsEnum;
-      if (reuse != null && reuse instanceof SimpleTextDocsAndPositionsEnum && ((SimpleTextDocsAndPositionsEnum) reuse).canReuse(SimpleTextFieldsReader.this.in)) {
-        docsAndPositionsEnum = (SimpleTextDocsAndPositionsEnum) reuse;
-      } else {
-        docsAndPositionsEnum = new SimpleTextDocsAndPositionsEnum();
-      } 
-      return docsAndPositionsEnum.reset(docsStart, liveDocs, indexOptions, docFreq);
-    }
   }
 
   private class SimpleTextDocsEnum extends DocsEnum {
@@ -246,7 +247,7 @@ class SimpleTextFieldsReader extends Fie
     private final BytesRefBuilder scratch = new BytesRefBuilder();
     private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
     private int cost;
-    
+
     public SimpleTextDocsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
       this.in = this.inStart.clone();
@@ -277,6 +278,12 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      assert false : "Shouldn't be calling nextPosition() if you haven't asked for positions";
+      return -1;
+    }
+
+    @Override
     public int nextDoc() throws IOException {
       if (docID == NO_MORE_DOCS) {
         return docID;
@@ -328,14 +335,14 @@ class SimpleTextFieldsReader extends Fie
       // Naive -- better to index skip data
       return slowAdvance(target);
     }
-    
+
     @Override
     public long cost() {
       return cost;
     }
   }
 
-  private class SimpleTextDocsAndPositionsEnum extends DocsAndPositionsEnum {
+  private class SimpleTextPostingsEnum extends PostingsEnum {
     private final IndexInput inStart;
     private final IndexInput in;
     private int docID = -1;
@@ -345,6 +352,7 @@ class SimpleTextFieldsReader extends Fie
     private final BytesRefBuilder scratch2 = new BytesRefBuilder();
     private final CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
     private final CharsRefBuilder scratchUTF16_2 = new CharsRefBuilder();
+    private int pos;
     private BytesRef payload;
     private long nextDocStart;
     private boolean readOffsets;
@@ -353,7 +361,7 @@ class SimpleTextFieldsReader extends Fie
     private int endOffset;
     private int cost;
 
-    public SimpleTextDocsAndPositionsEnum() {
+    public SimpleTextPostingsEnum() {
       this.inStart = SimpleTextFieldsReader.this.in;
       this.in = inStart.clone();
     }
@@ -362,7 +370,7 @@ class SimpleTextFieldsReader extends Fie
       return in == inStart;
     }
 
-    public SimpleTextDocsAndPositionsEnum reset(long fp, Bits liveDocs, IndexOptions indexOptions, int docFreq) {
+    public SimpleTextPostingsEnum reset(long fp, Bits liveDocs, IndexOptions indexOptions, int docFreq) {
       this.liveDocs = liveDocs;
       nextDocStart = fp;
       docID = -1;
@@ -437,7 +445,6 @@ class SimpleTextFieldsReader extends Fie
 
     @Override
     public int nextPosition() throws IOException {
-      final int pos;
       if (readPositions) {
         SimpleTextUtil.readLine(in, scratch);
         assert StringHelper.startsWith(scratch.get(), POS): "got line=" + scratch.get().utf8ToString();
@@ -488,7 +495,7 @@ class SimpleTextFieldsReader extends Fie
     public BytesRef getPayload() {
       return payload;
     }
-    
+
     @Override
     public long cost() {
       return cost;
@@ -506,9 +513,9 @@ class SimpleTextFieldsReader extends Fie
   }
 
   private static final long TERMS_BASE_RAM_BYTES_USED =
-        RamUsageEstimator.shallowSizeOfInstance(SimpleTextTerms.class)
-      + RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
-      + RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
+      RamUsageEstimator.shallowSizeOfInstance(SimpleTextTerms.class)
+          + RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
+          + RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
   private class SimpleTextTerms extends Terms implements Accountable {
     private final long termsStart;
     private final FieldInfo fieldInfo;
@@ -533,7 +540,7 @@ class SimpleTextFieldsReader extends Fie
       final Builder<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> b;
       final PairOutputs<Long,Long> outputsInner = new PairOutputs<>(posIntOutputs, posIntOutputs);
       final PairOutputs<Long,PairOutputs.Pair<Long,Long>> outputs = new PairOutputs<>(posIntOutputs,
-                                                                                                                      outputsInner);
+          outputsInner);
       b = new Builder<>(FST.INPUT_TYPE.BYTE1, outputs);
       IndexInput in = SimpleTextFieldsReader.this.in.clone();
       in.seek(termsStart);
@@ -548,8 +555,8 @@ class SimpleTextFieldsReader extends Fie
         if (scratch.get().equals(END) || StringHelper.startsWith(scratch.get(), FIELD)) {
           if (lastDocsStart != -1) {
             b.add(Util.toIntsRef(lastTerm.get(), scratchIntsRef),
-                  outputs.newPair(lastDocsStart,
-                                  outputsInner.newPair((long) docFreq, totalTermFreq)));
+                outputs.newPair(lastDocsStart,
+                    outputsInner.newPair((long) docFreq, totalTermFreq)));
             sumTotalTermFreq += totalTermFreq;
           }
           break;
@@ -565,7 +572,7 @@ class SimpleTextFieldsReader extends Fie
         } else if (StringHelper.startsWith(scratch.get(), TERM)) {
           if (lastDocsStart != -1) {
             b.add(Util.toIntsRef(lastTerm.get(), scratchIntsRef), outputs.newPair(lastDocsStart,
-                                                                            outputsInner.newPair((long) docFreq, totalTermFreq)));
+                outputsInner.newPair((long) docFreq, totalTermFreq)));
           }
           lastDocsStart = in.getFilePointer();
           final int len = scratch.length() - TERM.length;
@@ -652,7 +659,7 @@ class SimpleTextFieldsReader extends Fie
     public boolean hasPositions() {
       return fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
     }
-    
+
     @Override
     public boolean hasPayloads() {
       return fieldInfo.hasPayloads();

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextFieldsWriter.java Fri Feb  6 13:25:22 2015
@@ -20,8 +20,7 @@ package org.apache.lucene.codecs.simplet
 import java.io.IOException;
 
 import org.apache.lucene.codecs.FieldsConsumer;
-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.Fields;
@@ -33,7 +32,7 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
 class SimpleTextFieldsWriter extends FieldsConsumer {
-  
+
   private IndexOutput out;
   private final BytesRefBuilder scratch = new BytesRefBuilder();
   private final SegmentWriteState writeState;
@@ -79,22 +78,21 @@ class SimpleTextFieldsWriter extends Fie
 
       int flags = 0;
       if (hasPositions) {
-        
+        flags = PostingsEnum.FLAG_POSITIONS;
         if (hasPayloads) {
-          flags = flags | DocsAndPositionsEnum.FLAG_PAYLOADS;
+          flags = flags | PostingsEnum.FLAG_PAYLOADS;
         }
         if (hasOffsets) {
-          flags = flags | DocsAndPositionsEnum.FLAG_OFFSETS;
+          flags = flags | PostingsEnum.FLAG_OFFSETS;
         }
       } else {
         if (hasFreqs) {
-          flags = flags | DocsEnum.FLAG_FREQS;
+          flags = flags | PostingsEnum.FLAG_FREQS;
         }
       }
 
       TermsEnum termsEnum = terms.iterator(null);
-      DocsAndPositionsEnum posEnum = null;
-      DocsEnum docsEnum = null;
+      PostingsEnum postingsEnum = null;
 
       // for each term in field
       while(true) {
@@ -103,20 +101,16 @@ class SimpleTextFieldsWriter extends Fie
           break;
         }
 
-        if (hasPositions) {
-          posEnum = termsEnum.docsAndPositions(null, posEnum, flags);
-          docsEnum = posEnum;
-        } else {
-          docsEnum = termsEnum.docs(null, docsEnum, flags);
-        }
-        assert docsEnum != null: "termsEnum=" + termsEnum + " hasPos=" + hasPositions + " flags=" + flags;
+        postingsEnum = termsEnum.postings(null, postingsEnum, flags);
+
+        assert postingsEnum != null: "termsEnum=" + termsEnum + " hasPos=" + hasPositions + " flags=" + flags;
 
         boolean wroteTerm = false;
 
         // for each doc in field+term
         while(true) {
-          int doc = docsEnum.nextDoc();
-          if (doc == DocsEnum.NO_MORE_DOCS) {
+          int doc = postingsEnum.nextDoc();
+          if (doc == PostingsEnum.NO_MORE_DOCS) {
             break;
           }
 
@@ -143,7 +137,7 @@ class SimpleTextFieldsWriter extends Fie
           write(Integer.toString(doc));
           newline();
           if (hasFreqs) {
-            int freq = docsEnum.freq();
+            int freq = postingsEnum.freq();
             write(FREQ);
             write(Integer.toString(freq));
             newline();
@@ -154,15 +148,15 @@ class SimpleTextFieldsWriter extends Fie
 
               // for each pos in field+term+doc
               for(int i=0;i<freq;i++) {
-                int position = posEnum.nextPosition();
+                int position = postingsEnum.nextPosition();
 
                 write(POS);
                 write(Integer.toString(position));
                 newline();
 
                 if (hasOffsets) {
-                  int startOffset = posEnum.startOffset();
-                  int endOffset = posEnum.endOffset();
+                  int startOffset = postingsEnum.startOffset();
+                  int endOffset = postingsEnum.endOffset();
                   assert endOffset >= startOffset;
                   assert startOffset >= lastStartOffset: "startOffset=" + startOffset + " lastStartOffset=" + lastStartOffset;
                   lastStartOffset = startOffset;
@@ -174,7 +168,7 @@ class SimpleTextFieldsWriter extends Fie
                   newline();
                 }
 
-                BytesRef payload = posEnum.getPayload();
+                BytesRef payload = postingsEnum.getPayload();
 
                 if (payload != null && payload.length > 0) {
                   assert payload.length != 0;

Modified: lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsReader.java Fri Feb  6 13:25:22 2015
@@ -25,8 +25,8 @@ import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.lucene.codecs.TermVectorsReader;
-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.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -59,15 +59,15 @@ import static org.apache.lucene.codecs.s
 public class SimpleTextTermVectorsReader extends TermVectorsReader {
 
   private static final long BASE_RAM_BYTES_USED =
-        RamUsageEstimator.shallowSizeOfInstance(SimpleTextTermVectorsReader.class)
-      + RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
-      + RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
+      RamUsageEstimator.shallowSizeOfInstance(SimpleTextTermVectorsReader.class)
+          + RamUsageEstimator.shallowSizeOfInstance(BytesRef.class)
+          + RamUsageEstimator.shallowSizeOfInstance(CharsRef.class);
 
   private long offsets[]; /* docid -> offset in .vec file */
   private IndexInput in;
   private BytesRefBuilder scratch = new BytesRefBuilder();
   private CharsRefBuilder scratchUTF16 = new CharsRefBuilder();
-  
+
   public SimpleTextTermVectorsReader(Directory directory, SegmentInfo si, IOContext context) throws IOException {
     boolean success = false;
     try {
@@ -82,15 +82,15 @@ public class SimpleTextTermVectorsReader
     }
     readIndex(si.getDocCount());
   }
-  
+
   // used by clone
   SimpleTextTermVectorsReader(long offsets[], IndexInput in) {
     this.offsets = offsets;
     this.in = in;
   }
-  
-  // we don't actually write a .tvx-like index, instead we read the 
-  // vectors file in entirety up-front and save the offsets 
+
+  // we don't actually write a .tvx-like index, instead we read the
+  // vectors file in entirety up-front and save the offsets
   // so we can seek to the data later.
   private void readIndex(int maxDoc) throws IOException {
     ChecksumIndexInput input = new BufferedChecksumIndexInput(in);
@@ -106,7 +106,7 @@ public class SimpleTextTermVectorsReader
     SimpleTextUtil.checkFooter(input);
     assert upto == offsets.length;
   }
-  
+
   @Override
   public Fields get(int doc) throws IOException {
     SortedMap<String,SimpleTVTerms> fields = new TreeMap<>();
@@ -122,30 +122,30 @@ public class SimpleTextTermVectorsReader
       assert StringHelper.startsWith(scratch.get(), FIELD);
       // skip fieldNumber:
       parseIntAt(FIELD.length);
-      
+
       readLine();
       assert StringHelper.startsWith(scratch.get(), FIELDNAME);
       String fieldName = readString(FIELDNAME.length, scratch);
-      
+
       readLine();
       assert StringHelper.startsWith(scratch.get(), FIELDPOSITIONS);
       boolean positions = Boolean.parseBoolean(readString(FIELDPOSITIONS.length, scratch));
-      
+
       readLine();
       assert StringHelper.startsWith(scratch.get(), FIELDOFFSETS);
       boolean offsets = Boolean.parseBoolean(readString(FIELDOFFSETS.length, scratch));
-      
+
       readLine();
       assert StringHelper.startsWith(scratch.get(), FIELDPAYLOADS);
       boolean payloads = Boolean.parseBoolean(readString(FIELDPAYLOADS.length, scratch));
-      
+
       readLine();
       assert StringHelper.startsWith(scratch.get(), FIELDTERMCOUNT);
       int termCount = parseIntAt(FIELDTERMCOUNT.length);
-      
+
       SimpleTVTerms terms = new SimpleTVTerms(offsets, positions, payloads);
       fields.put(fieldName, terms);
-      
+
       BytesRefBuilder term = new BytesRefBuilder();
       for (int j = 0; j < termCount; j++) {
         readLine();
@@ -154,14 +154,14 @@ public class SimpleTextTermVectorsReader
         term.grow(termLength);
         term.setLength(termLength);
         System.arraycopy(scratch.bytes(), TERMTEXT.length, term.bytes(), 0, termLength);
-        
+
         SimpleTVPostings postings = new SimpleTVPostings();
         terms.terms.put(term.toBytesRef(), postings);
-        
+
         readLine();
         assert StringHelper.startsWith(scratch.get(), TERMFREQ);
         postings.freq = parseIntAt(TERMFREQ.length);
-        
+
         if (positions || offsets) {
           if (positions) {
             postings.positions = new int[postings.freq];
@@ -169,12 +169,12 @@ public class SimpleTextTermVectorsReader
               postings.payloads = new BytesRef[postings.freq];
             }
           }
-        
+
           if (offsets) {
             postings.startOffsets = new int[postings.freq];
             postings.endOffsets = new int[postings.freq];
           }
-          
+
           for (int k = 0; k < postings.freq; k++) {
             if (positions) {
               readLine();
@@ -192,12 +192,12 @@ public class SimpleTextTermVectorsReader
                 }
               }
             }
-            
+
             if (offsets) {
               readLine();
               assert StringHelper.startsWith(scratch.get(), STARTOFFSET);
               postings.startOffsets[k] = parseIntAt(STARTOFFSET.length);
-              
+
               readLine();
               assert StringHelper.startsWith(scratch.get(), ENDOFFSET);
               postings.endOffsets[k] = parseIntAt(ENDOFFSET.length);
@@ -216,11 +216,11 @@ public class SimpleTextTermVectorsReader
     }
     return new SimpleTextTermVectorsReader(offsets, in.clone());
   }
-  
+
   @Override
   public void close() throws IOException {
     try {
-      IOUtils.close(in); 
+      IOUtils.close(in);
     } finally {
       in = null;
       offsets = null;
@@ -230,20 +230,20 @@ public class SimpleTextTermVectorsReader
   private void readLine() throws IOException {
     SimpleTextUtil.readLine(in, scratch);
   }
-  
+
   private int parseIntAt(int offset) {
     scratchUTF16.copyUTF8Bytes(scratch.bytes(), offset, scratch.length()-offset);
     return ArrayUtil.parseInt(scratchUTF16.chars(), 0, scratchUTF16.length());
   }
-  
+
   private String readString(int offset, BytesRefBuilder scratch) {
     scratchUTF16.copyUTF8Bytes(scratch.bytes(), offset, scratch.length()-offset);
     return scratchUTF16.toString();
   }
-  
+
   private class SimpleTVFields extends Fields {
     private final SortedMap<String,SimpleTVTerms> fields;
-    
+
     SimpleTVFields(SortedMap<String,SimpleTVTerms> fields) {
       this.fields = fields;
     }
@@ -263,20 +263,20 @@ public class SimpleTextTermVectorsReader
       return fields.size();
     }
   }
-  
+
   private static class SimpleTVTerms extends Terms {
     final SortedMap<BytesRef,SimpleTVPostings> terms;
     final boolean hasOffsets;
     final boolean hasPositions;
     final boolean hasPayloads;
-    
+
     SimpleTVTerms(boolean hasOffsets, boolean hasPositions, boolean hasPayloads) {
       this.hasOffsets = hasOffsets;
       this.hasPositions = hasPositions;
       this.hasPayloads = hasPayloads;
       terms = new TreeMap<>();
     }
-    
+
     @Override
     public TermsEnum iterator(TermsEnum reuse) throws IOException {
       // TODO: reuse
@@ -317,13 +317,13 @@ public class SimpleTextTermVectorsReader
     public boolean hasPositions() {
       return hasPositions;
     }
-    
+
     @Override
     public boolean hasPayloads() {
       return hasPayloads;
     }
   }
-  
+
   private static class SimpleTVPostings {
     private int freq;
     private int positions[];
@@ -331,17 +331,17 @@ public class SimpleTextTermVectorsReader
     private int endOffsets[];
     private BytesRef payloads[];
   }
-  
+
   private static class SimpleTVTermsEnum extends TermsEnum {
     SortedMap<BytesRef,SimpleTVPostings> terms;
     Iterator<Map.Entry<BytesRef,SimpleTextTermVectorsReader.SimpleTVPostings>> iterator;
     Map.Entry<BytesRef,SimpleTextTermVectorsReader.SimpleTVPostings> current;
-    
+
     SimpleTVTermsEnum(SortedMap<BytesRef,SimpleTVPostings> terms) {
       this.terms = terms;
       this.iterator = terms.entrySet().iterator();
     }
-    
+
     @Override
     public SeekStatus seekCeil(BytesRef text) throws IOException {
       iterator = terms.tailMap(text).entrySet().iterator();
@@ -388,26 +388,27 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-      // TODO: reuse
-      SimpleTVDocsEnum e = new SimpleTVDocsEnum();
-      e.reset(liveDocs, (flags & DocsEnum.FLAG_FREQS) == 0 ? 1 : current.getValue().freq);
-      return e;
-    }
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      SimpleTVPostings postings = current.getValue();
-      if (postings.positions == null && postings.startOffsets == null) {
-        return null;
+      if (PostingsEnum.requiresPositions(flags)) {
+        SimpleTVPostings postings = current.getValue();
+        if (postings.positions == null && postings.startOffsets == null) {
+          return null;
+        }
+        // TODO: reuse
+        SimpleTVPostingsEnum e = new SimpleTVPostingsEnum();
+        e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
+        return e;
       }
+
       // TODO: reuse
-      SimpleTVDocsAndPositionsEnum e = new SimpleTVDocsAndPositionsEnum();
-      e.reset(liveDocs, postings.positions, postings.startOffsets, postings.endOffsets, postings.payloads);
+      SimpleTVDocsEnum e = new SimpleTVDocsEnum();
+      e.reset(liveDocs, (flags & PostingsEnum.FLAG_FREQS) == 0 ? 1 : current.getValue().freq);
       return e;
     }
+
   }
-  
+
   // note: these two enum classes are exactly like the Default impl...
   private static class SimpleTVDocsEnum extends DocsEnum {
     private boolean didNext;
@@ -422,6 +423,12 @@ public class SimpleTextTermVectorsReader
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      assert false;
+      return -1;
+    }
+
+    @Override
     public int docID() {
       return doc;
     }
@@ -447,14 +454,14 @@ public class SimpleTextTermVectorsReader
       this.doc = -1;
       didNext = false;
     }
-    
+
     @Override
     public long cost() {
       return 1;
     }
   }
-  
-  private static class SimpleTVDocsAndPositionsEnum extends DocsAndPositionsEnum {
+
+  private static class SimpleTVPostingsEnum extends PostingsEnum {
     private boolean didNext;
     private int doc = -1;
     private int nextPos;
@@ -512,11 +519,11 @@ public class SimpleTextTermVectorsReader
 
     @Override
     public int nextPosition() {
-      assert (positions != null && nextPos < positions.length) ||
-        startOffsets != null && nextPos < startOffsets.length;
       if (positions != null) {
+        assert nextPos < positions.length;
         return positions[nextPos++];
       } else {
+        assert nextPos < startOffsets.length;
         nextPos++;
         return -1;
       }
@@ -539,7 +546,7 @@ public class SimpleTextTermVectorsReader
         return endOffsets[nextPos-1];
       }
     }
-    
+
     @Override
     public long cost() {
       return 1;
@@ -550,7 +557,7 @@ public class SimpleTextTermVectorsReader
   public long ramBytesUsed() {
     return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(offsets);
   }
-  
+
   @Override
   public String toString() {
     return getClass().getSimpleName();

Modified: lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/codecs/src/test/org/apache/lucene/codecs/memory/TestDirectPostingsFormat.java Fri Feb  6 13:25:22 2015
@@ -29,6 +29,11 @@ public class TestDirectPostingsFormat ex
   private final Codec codec = TestUtil.alwaysPostingsFormat(new DirectPostingsFormat());
 
   @Override
+  protected boolean isPostingsEnumReuseImplemented() {
+    return false;
+  }
+
+  @Override
   protected Codec getCodec() {
     return codec;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/Token.java Fri Feb  6 13:25:22 2015
@@ -20,7 +20,7 @@ package org.apache.lucene.analysis;
 import org.apache.lucene.analysis.tokenattributes.FlagsAttribute;
 import org.apache.lucene.analysis.tokenattributes.PackedTokenAttributeImpl;
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
-import org.apache.lucene.index.DocsAndPositionsEnum; // for javadoc
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.AttributeFactory;
 import org.apache.lucene.util.AttributeImpl;
@@ -43,7 +43,7 @@ import org.apache.lucene.util.BytesRef;
   with type "eos".  The default token type is "word".  
   <p>
   A Token can optionally have metadata (a.k.a. payload) in the form of a variable
-  length byte array. Use {@link DocsAndPositionsEnum#getPayload()} to retrieve the 
+  length byte array. Use {@link org.apache.lucene.index.PostingsEnum#getPayload()} to retrieve the
   payloads from the index.
   
   <br><br>

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PayloadAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PayloadAttribute.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PayloadAttribute.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PayloadAttribute.java Fri Feb  6 13:25:22 2015
@@ -17,7 +17,6 @@ package org.apache.lucene.analysis.token
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
 import org.apache.lucene.util.Attribute;
 import org.apache.lucene.util.BytesRef;
 
@@ -33,7 +32,7 @@ import org.apache.lucene.util.BytesRef;
  * best to use the minimum number of bytes necessary. Some codec implementations
  * may optimize payload storage when all payloads have the same length.
  * 
- * @see DocsAndPositionsEnum
+ * @see org.apache.lucene.index.PostingsEnum
  */
 public interface PayloadAttribute extends Attribute {
   /**

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/analysis/tokenattributes/PositionIncrementAttribute.java Fri Feb  6 13:25:22 2015
@@ -43,7 +43,7 @@ import org.apache.lucene.util.Attribute;
  *
  * </ul>
  * 
- * @see org.apache.lucene.index.DocsAndPositionsEnum
+ * @see org.apache.lucene.index.PostingsEnum
  */
 public interface PositionIncrementAttribute extends Attribute {
   /** Set the position increment. The default value is one.

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/BlockTermState.java Fri Feb  6 13:25:22 2015
@@ -16,13 +16,12 @@ package org.apache.lucene.codecs;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.DocsEnum; // javadocs
 import org.apache.lucene.index.OrdTermState;
 import org.apache.lucene.index.TermState;
 
 /**
  * Holds all state required for {@link PostingsReaderBase}
- * to produce a {@link DocsEnum} without re-seeking the
+ * to produce a {@link org.apache.lucene.index.PostingsEnum} without re-seeking the
  * terms dict.
  */
 public class BlockTermState extends OrdTermState {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsReaderBase.java Fri Feb  6 13:25:22 2015
@@ -20,8 +20,7 @@ package org.apache.lucene.codecs;
 import java.io.Closeable;
 import java.io.IOException;
 
-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;
@@ -31,8 +30,8 @@ import org.apache.lucene.util.Bits;
 
 /** The core terms dictionaries (BlockTermsReader,
  *  BlockTreeTermsReader) interact with a single instance
- *  of this class to manage creation of {@link DocsEnum} and
- *  {@link DocsAndPositionsEnum} instances.  It provides an
+ *  of this class to manage creation of {@link org.apache.lucene.index.PostingsEnum} and
+ *  {@link org.apache.lucene.index.PostingsEnum} instances.  It provides an
  *  IndexInput (termsIn) where this class may read any
  *  previously stored data that it had written in its
  *  corresponding {@link PostingsWriterBase} at indexing
@@ -66,12 +65,7 @@ public abstract class PostingsReaderBase
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
-  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse, int flags) throws IOException;
-
-  /** Must fully consume state, since after this call that
-   *  TermState may be reused. */
-  public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse,
-                                                        int flags) throws IOException;
+  public abstract PostingsEnum postings(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, PostingsEnum reuse, int flags) throws IOException;
   
   /** 
    * Checks consistency of this reader.
@@ -81,7 +75,7 @@ public abstract class PostingsReaderBase
    * @lucene.internal
    */
   public abstract void checkIntegrity() throws IOException;
-  
+
   @Override
   public abstract void close() throws IOException;
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PostingsWriterBase.java Fri Feb  6 13:25:22 2015
@@ -17,12 +17,7 @@ package org.apache.lucene.codecs;
  * limitations under the License.
  */
 
-import java.io.Closeable;
-import java.io.IOException;
-
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
-import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
-import org.apache.lucene.index.DocsEnum; // javadocs
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.TermsEnum;
@@ -31,6 +26,9 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 
+import java.io.Closeable;
+import java.io.IOException;
+
 /**
  * Class that plugs into term dictionaries, such as {@link
  * BlockTreeTermsWriter}, and handles writing postings.
@@ -54,8 +52,8 @@ public abstract class PostingsWriterBase
   public abstract void init(IndexOutput termsOut, SegmentWriteState state) throws IOException;
 
   /** Write all postings for one term; use the provided
-   *  {@link TermsEnum} to pull a {@link DocsEnum} or {@link
-   *  DocsAndPositionsEnum}.  This method should not
+   *  {@link TermsEnum} to pull a {@link org.apache.lucene.index.PostingsEnum}.
+   *  This method should not
    *  re-position the {@code TermsEnum}!  It is already
    *  positioned on the term that should be written.  This
    *  method must set the bit in the provided {@link

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java Fri Feb  6 13:25:22 2015
@@ -19,8 +19,7 @@ package org.apache.lucene.codecs;
 
 import java.io.IOException;
 
-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.IndexOptions;
 import org.apache.lucene.index.TermsEnum;
@@ -42,8 +41,7 @@ import org.apache.lucene.util.FixedBitSe
 public abstract class PushPostingsWriterBase extends PostingsWriterBase {
 
   // Reused in writeTerm
-  private DocsEnum docsEnum;
-  private DocsAndPositionsEnum posEnum;
+  private PostingsEnum postingsEnum;
   private int enumFlags;
 
   /** {@link FieldInfo} of current field being written. */
@@ -100,18 +98,18 @@ public abstract class PushPostingsWriter
     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;
+        enumFlags = PostingsEnum.FLAG_POSITIONS;
       }
     } 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;
       }
     }
 
@@ -121,26 +119,21 @@ public abstract class PushPostingsWriter
   @Override
   public final BlockTermState writeTerm(BytesRef term, TermsEnum termsEnum, FixedBitSet docsSeen) throws IOException {
     startTerm();
-    if (writePositions == false) {
-      docsEnum = termsEnum.docs(null, docsEnum, enumFlags);
-    } else {
-      posEnum = termsEnum.docsAndPositions(null, posEnum, enumFlags);
-      docsEnum = posEnum;
-    }
-    assert docsEnum != null;
+    postingsEnum = termsEnum.postings(null, postingsEnum, enumFlags);
+    assert postingsEnum != null;
 
     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;
       }
       docFreq++;
       docsSeen.set(docID);
       int freq;
       if (writeFreqs) {
-        freq = docsEnum.freq();
+        freq = postingsEnum.freq();
         totalTermFreq += freq;
       } else {
         freq = -1;
@@ -149,13 +142,13 @@ public abstract class PushPostingsWriter
 
       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;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java Fri Feb  6 13:25:22 2015
@@ -21,7 +21,6 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; // javadocs
-import org.apache.lucene.index.DocsAndPositionsEnum; // javadocs
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.util.Accountable;
 
@@ -40,7 +39,7 @@ public abstract class TermVectorsReader
   /** Returns term vectors for this document, or null if
    *  term vectors were not indexed. If offsets are
    *  available they are in an {@link OffsetAttribute}
-   *  available from the {@link DocsAndPositionsEnum}. */
+   *  available from the {@link org.apache.lucene.index.PostingsEnum}. */
   public abstract Fields get(int doc) throws IOException;
   
   /** 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Fri Feb  6 13:25:22 2015
@@ -21,7 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.Iterator;
 
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
@@ -225,7 +225,7 @@ public abstract class TermVectorsWriter
     String lastFieldName = null;
     
     TermsEnum termsEnum = null;
-    DocsAndPositionsEnum docsAndPositionsEnum = null;
+    PostingsEnum docsAndPositionsEnum = null;
     
     int fieldCount = 0;
     for(String fieldName : vectors) {
@@ -268,7 +268,7 @@ public abstract class TermVectorsWriter
         startTerm(termsEnum.term(), freq);
 
         if (hasPositions || hasOffsets) {
-          docsAndPositionsEnum = termsEnum.docsAndPositions(null, docsAndPositionsEnum);
+          docsAndPositionsEnum = termsEnum.postings(null, docsAndPositionsEnum, PostingsEnum.FLAG_OFFSETS | PostingsEnum.FLAG_PAYLOADS);
           assert docsAndPositionsEnum != null;
           
           final int docID = docsAndPositionsEnum.nextDoc();
@@ -282,7 +282,7 @@ public abstract class TermVectorsWriter
             
             final BytesRef payload = docsAndPositionsEnum.getPayload();
 
-            assert !hasPositions || pos >= 0;
+            assert !hasPositions || pos >= 0 ;
             addPosition(pos, startOffset, endOffset, payload);
           }
         }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java Fri Feb  6 13:25:22 2015
@@ -19,9 +19,7 @@ package org.apache.lucene.codecs.blocktr
 
 import java.io.IOException;
 
-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.IndexInput;
@@ -203,20 +201,9 @@ final class IntersectTermsEnum extends T
   }
 
   @Override
-  public DocsEnum docs(Bits skipDocs, DocsEnum reuse, int flags) throws IOException {
+  public PostingsEnum postings(Bits skipDocs, PostingsEnum reuse, int flags) throws IOException {
     currentFrame.decodeMetaData();
-    return fr.parent.postingsReader.docs(fr.fieldInfo, currentFrame.termState, 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;
-    }
-
-    currentFrame.decodeMetaData();
-    return fr.parent.postingsReader.docsAndPositions(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
+    return fr.parent.postingsReader.postings(fr.fieldInfo, currentFrame.termState, skipDocs, reuse, flags);
   }
 
   private int getState() {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.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;
@@ -981,7 +979,7 @@ final class SegmentTermsEnum extends Ter
   }
 
   @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);
@@ -990,19 +988,7 @@ final class SegmentTermsEnum extends Ter
     //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/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Fri Feb  6 13:25:22 2015
@@ -17,19 +17,6 @@ package org.apache.lucene.codecs.compres
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PACKED_BLOCK_SIZE;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.CODEC_SFX_DAT;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.CODEC_SFX_IDX;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.FLAGS_BITS;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.OFFSETS;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PAYLOADS;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.POSITIONS;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_EXTENSION;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CHUNK_STATS;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CURRENT;
-import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_START;
-
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
@@ -40,8 +27,7 @@ import java.util.NoSuchElementException;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.TermVectorsReader;
 import org.apache.lucene.index.CorruptIndexException;
-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.Fields;
@@ -65,6 +51,18 @@ import org.apache.lucene.util.LongsRef;
 import org.apache.lucene.util.packed.BlockPackedReaderIterator;
 import org.apache.lucene.util.packed.PackedInts;
 
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.CODEC_SFX_DAT;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.CODEC_SFX_IDX;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.FLAGS_BITS;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.OFFSETS;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PACKED_BLOCK_SIZE;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.PAYLOADS;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.POSITIONS;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_EXTENSION;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VECTORS_INDEX_EXTENSION;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CHUNK_STATS;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_CURRENT;
+import static org.apache.lucene.codecs.compressing.CompressingTermVectorsWriter.VERSION_START;
 
 /**
  * {@link TermVectorsReader} for {@link CompressingTermVectorsFormat}.
@@ -937,30 +935,27 @@ public final class CompressingTermVector
     }
 
     @Override
-    public final DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-      final TVDocsEnum docsEnum;
-      if (reuse != null && reuse instanceof TVDocsEnum) {
-        docsEnum = (TVDocsEnum) reuse;
+    public final PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+
+      if ((flags & PostingsEnum.FLAG_POSITIONS) >= PostingsEnum.FLAG_POSITIONS) {
+        if (positions == null && startOffsets == null)
+          return null;
+      }
+
+      final TVPostingsEnum docsEnum;
+      if (reuse != null && reuse instanceof TVPostingsEnum) {
+        docsEnum = (TVPostingsEnum) reuse;
       } else {
-        docsEnum = new TVDocsEnum();
+        docsEnum = new TVPostingsEnum();
       }
 
       docsEnum.reset(liveDocs, termFreqs[ord], positionIndex[ord], positions, startOffsets, lengths, payloads, payloadIndex);
       return docsEnum;
     }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      if (positions == null && startOffsets == null) {
-        return null;
-      }
-      // TODO: slightly sheisty
-      return (DocsAndPositionsEnum) docs(liveDocs, reuse, flags);
-    }
-
   }
 
-  private static class TVDocsEnum extends DocsAndPositionsEnum {
+  private static class TVPostingsEnum extends PostingsEnum {
 
     private Bits liveDocs;
     private int doc = -1;
@@ -974,7 +969,7 @@ public final class CompressingTermVector
     private int basePayloadOffset;
     private int i;
 
-    TVDocsEnum() {
+    TVPostingsEnum() {
       payload = new BytesRef();
     }
 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java Fri Feb  6 13:25:22 2015
@@ -17,27 +17,7 @@ package org.apache.lucene.codecs.lucene5
  * limitations under the License.
  */
 
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_LIVE;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_MISSING;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_MASK;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_SHIFT;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.CONST_COMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.DELTA_COMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.GCD_COMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_COUNT;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_MASK;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_SHIFT;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.MONOTONIC_COMPRESSED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_MASK;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_SHIFT;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_SINGLE_VALUED;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_WITH_ADDRESSES;
-import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.TABLE_COMPRESSED;
-
-import java.io.Closeable; // javadocs
+import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -52,8 +32,7 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocValues;
-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.IndexFileNames;
@@ -78,6 +57,26 @@ import org.apache.lucene.util.RamUsageEs
 import org.apache.lucene.util.packed.DirectReader;
 import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
 
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_LIVE;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.ALL_MISSING;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_FIXED_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_PREFIX_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.BLOCK_INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.CONST_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.DELTA_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.GCD_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_COUNT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.MONOTONIC_COMPRESSED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_MASK;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.REVERSE_INTERVAL_SHIFT;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_SINGLE_VALUED;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.SORTED_WITH_ADDRESSES;
+import static org.apache.lucene.codecs.lucene50.Lucene50DocValuesConsumer.TABLE_COMPRESSED;
+
 /** reader for {@link Lucene50DocValuesFormat} */
 class Lucene50DocValuesProducer extends DocValuesProducer implements Closeable {
   private final Map<String,NumericEntry> numerics = new HashMap<>();
@@ -1141,14 +1140,10 @@ class Lucene50DocValuesProducer extends
       }
       
       @Override
-      public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-      
-      @Override
-      public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
+      public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
         throw new UnsupportedOperationException();
       }
+
     }
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java Fri Feb  6 13:25:22 2015
@@ -30,7 +30,7 @@ import org.apache.lucene.codecs.Postings
 import org.apache.lucene.codecs.PostingsWriterBase;
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsReader;
 import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
@@ -141,7 +141,7 @@ import org.apache.lucene.util.packed.Pac
  *        determined by the largest integer. Smaller block size result in smaller variance among width 
  *        of integers hence smaller indexes. Larger block size result in more efficient bulk i/o hence
  *        better acceleration. This value should always be a multiple of 64, currently fixed as 128 as 
- *        a tradeoff. It is also the skip interval used to accelerate {@link DocsEnum#advance(int)}.
+ *        a tradeoff. It is also the skip interval used to accelerate {@link org.apache.lucene.index.PostingsEnum#advance(int)}.
  *    <li>DocFPDelta determines the position of this term's TermFreqs within the .doc file. 
  *        In particular, it is the difference of file offset between this term's
  *        data and previous term's data (or zero, for the first term in the block).On disk it is 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java Fri Feb  6 13:25:22 2015
@@ -24,11 +24,11 @@ import org.apache.lucene.codecs.BlockTer
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.PostingsReaderBase;
 import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.IntBlockTermState;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.IndexInput;
@@ -193,39 +193,38 @@ public final class Lucene50PostingsReade
   }
     
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-    BlockDocsEnum docsEnum;
-    if (reuse instanceof BlockDocsEnum) {
-      docsEnum = (BlockDocsEnum) reuse;
-      if (!docsEnum.canReuse(docIn, fieldInfo)) {
+  public PostingsEnum postings(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+
+    if ((flags & PostingsEnum.FLAG_POSITIONS) < PostingsEnum.FLAG_POSITIONS) {
+      BlockDocsEnum docsEnum;
+      if (reuse instanceof BlockDocsEnum) {
+        docsEnum = (BlockDocsEnum) reuse;
+        if (!docsEnum.canReuse(docIn, fieldInfo)) {
+          docsEnum = new BlockDocsEnum(fieldInfo);
+        }
+      } else {
         docsEnum = new BlockDocsEnum(fieldInfo);
       }
-    } else {
-      docsEnum = new BlockDocsEnum(fieldInfo);
+      return docsEnum.reset(liveDocs, (IntBlockTermState) termState, flags);
     }
-    return docsEnum.reset(liveDocs, (IntBlockTermState) termState, flags);
-  }
-
-  // TODO: specialize to liveDocs vs not
-  
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, int flags)
-    throws IOException {
 
+    boolean indexHasPositions = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
     boolean indexHasOffsets = fieldInfo.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) >= 0;
     boolean indexHasPayloads = fieldInfo.hasPayloads();
 
-    if ((!indexHasOffsets || (flags & DocsAndPositionsEnum.FLAG_OFFSETS) == 0) &&
-        (!indexHasPayloads || (flags & DocsAndPositionsEnum.FLAG_PAYLOADS) == 0)) {
-      BlockDocsAndPositionsEnum docsAndPositionsEnum;
-      if (reuse instanceof BlockDocsAndPositionsEnum) {
-        docsAndPositionsEnum = (BlockDocsAndPositionsEnum) reuse;
+    if (!indexHasPositions)
+      return null;
+
+    if ((!indexHasOffsets || (flags & PostingsEnum.FLAG_OFFSETS) == 0) &&
+        (!indexHasPayloads || (flags & PostingsEnum.FLAG_PAYLOADS) == 0)) {
+      BlockPostingsEnum docsAndPositionsEnum;
+      if (reuse instanceof BlockPostingsEnum) {
+        docsAndPositionsEnum = (BlockPostingsEnum) reuse;
         if (!docsAndPositionsEnum.canReuse(docIn, fieldInfo)) {
-          docsAndPositionsEnum = new BlockDocsAndPositionsEnum(fieldInfo);
+          docsAndPositionsEnum = new BlockPostingsEnum(fieldInfo);
         }
       } else {
-        docsAndPositionsEnum = new BlockDocsAndPositionsEnum(fieldInfo);
+        docsAndPositionsEnum = new BlockPostingsEnum(fieldInfo);
       }
       return docsAndPositionsEnum.reset(liveDocs, (IntBlockTermState) termState);
     } else {
@@ -302,7 +301,7 @@ public final class Lucene50PostingsReade
         indexHasPayloads == fieldInfo.hasPayloads();
     }
     
-    public DocsEnum reset(Bits liveDocs, IntBlockTermState termState, int flags) throws IOException {
+    public PostingsEnum reset(Bits liveDocs, IntBlockTermState termState, int flags) throws IOException {
       this.liveDocs = liveDocs;
 
       docFreq = termState.docFreq;
@@ -319,7 +318,7 @@ public final class Lucene50PostingsReade
       }
 
       doc = -1;
-      this.needsFreq = (flags & DocsEnum.FLAG_FREQS) != 0;
+      this.needsFreq = (flags & PostingsEnum.FLAG_FREQS) != 0;
       if (indexHasFreq == false || needsFreq == false) {
         Arrays.fill(freqBuffer, 1);
       }
@@ -337,6 +336,12 @@ public final class Lucene50PostingsReade
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      assert false;   // shouldn't be calling nextPosition() on this
+      return -1;
+    }
+
+    @Override
     public int docID() {
       return doc;
     }
@@ -472,7 +477,7 @@ public final class Lucene50PostingsReade
   }
 
 
-  final class BlockDocsAndPositionsEnum extends DocsAndPositionsEnum {
+  final class BlockPostingsEnum extends PostingsEnum {
     
     private final byte[] encoded;
 
@@ -535,7 +540,7 @@ public final class Lucene50PostingsReade
     private Bits liveDocs;
     private int singletonDocID; // docid when there is a single pulsed posting, otherwise -1
     
-    public BlockDocsAndPositionsEnum(FieldInfo fieldInfo) throws IOException {
+    public BlockPostingsEnum(FieldInfo fieldInfo) throws IOException {
       this.startDocIn = Lucene50PostingsReader.this.docIn;
       this.docIn = null;
       this.posIn = Lucene50PostingsReader.this.posIn.clone();
@@ -550,7 +555,7 @@ public final class Lucene50PostingsReade
         indexHasPayloads == fieldInfo.hasPayloads();
     }
     
-    public DocsAndPositionsEnum reset(Bits liveDocs, IntBlockTermState termState) throws IOException {
+    public PostingsEnum reset(Bits liveDocs, IntBlockTermState termState) throws IOException {
       this.liveDocs = liveDocs;
 
       docFreq = termState.docFreq;
@@ -769,6 +774,9 @@ public final class Lucene50PostingsReade
 
     @Override
     public int nextPosition() throws IOException {
+
+      assert posPendingCount > 0;
+
       if (posPendingFP != -1) {
         posIn.seek(posPendingFP);
         posPendingFP = -1;
@@ -813,7 +821,7 @@ public final class Lucene50PostingsReade
   }
 
   // Also handles payloads + offsets
-  final class EverythingEnum extends DocsAndPositionsEnum {
+  final class EverythingEnum extends PostingsEnum {
     
     private final byte[] encoded;
 
@@ -960,8 +968,8 @@ public final class Lucene50PostingsReade
         lastPosBlockFP = posTermStartFP + termState.lastPosBlockOffset;
       }
 
-      this.needsOffsets = (flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0;
-      this.needsPayloads = (flags & DocsAndPositionsEnum.FLAG_PAYLOADS) != 0;
+      this.needsOffsets = (flags & PostingsEnum.FLAG_OFFSETS) != 0;
+      this.needsPayloads = (flags & PostingsEnum.FLAG_PAYLOADS) != 0;
 
       doc = -1;
       accum = 0;
@@ -1228,6 +1236,8 @@ public final class Lucene50PostingsReade
 
     @Override
     public int nextPosition() throws IOException {
+      assert posPendingCount > 0;
+      
       if (posPendingFP != -1) {
         posIn.seek(posPendingFP);
         posPendingFP = -1;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java Fri Feb  6 13:25:22 2015
@@ -374,7 +374,7 @@ class BufferedUpdatesStream implements A
     final int startDelCount;
 
     TermsEnum termsEnum;
-    DocsEnum docsEnum;
+    PostingsEnum postingsEnum;
     BytesRef term;
     boolean any;
 
@@ -562,12 +562,12 @@ class BufferedUpdatesStream implements A
         if (state.delGen < delGen) {
 
           // we don't need term frequencies for this
-          state.docsEnum = state.termsEnum.docs(state.rld.getLiveDocs(), state.docsEnum, DocsEnum.FLAG_NONE);
+          state.postingsEnum = state.termsEnum.postings(state.rld.getLiveDocs(), state.postingsEnum, PostingsEnum.FLAG_NONE);
 
-          assert state.docsEnum != null;
+          assert state.postingsEnum != null;
 
           while (true) {
-            final int docID = state.docsEnum.nextDoc();
+            final int docID = state.postingsEnum.nextDoc();
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
             }
@@ -623,7 +623,7 @@ class BufferedUpdatesStream implements A
     
     String currentField = null;
     TermsEnum termsEnum = null;
-    DocsEnum docsEnum = null;
+    PostingsEnum postingsEnum = null;
     
     for (DocValuesUpdate update : updates) {
       Term term = update.term;
@@ -658,14 +658,14 @@ class BufferedUpdatesStream implements A
 
       if (termsEnum.seekExact(term.bytes())) {
         // we don't need term frequencies for this
-        docsEnum = termsEnum.docs(segState.rld.getLiveDocs(), docsEnum, DocsEnum.FLAG_NONE);
+        postingsEnum = termsEnum.postings(segState.rld.getLiveDocs(), postingsEnum, PostingsEnum.FLAG_NONE);
 
         DocValuesFieldUpdates dvUpdates = dvUpdatesContainer.getUpdates(update.field, update.type);
         if (dvUpdates == null) {
           dvUpdates = dvUpdatesContainer.newUpdates(update.field, update.type, segState.reader.maxDoc());
         }
         int doc;
-        while ((doc = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        while ((doc = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
           if (doc >= limit) {
             break; // no more docs that can be updated for this term
           }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Fri Feb  6 13:25:22 2015
@@ -882,9 +882,9 @@ public class CheckIndex implements Close
     final Status.TermIndexStatus status = new Status.TermIndexStatus();
     int computedFieldCount = 0;
     
-    DocsEnum docs = null;
-    DocsEnum docsAndFreqs = null;
-    DocsAndPositionsEnum postings = null;
+    PostingsEnum docs = null;
+    PostingsEnum docsAndFreqs = null;
+    PostingsEnum postings = null;
     
     String lastField = null;
     for (String field : fields) {
@@ -1026,8 +1026,8 @@ public class CheckIndex implements Close
         }
         sumDocFreq += docFreq;
         
-        docs = termsEnum.docs(liveDocs, docs);
-        postings = termsEnum.docsAndPositions(liveDocs, postings);
+        docs = termsEnum.postings(liveDocs, docs);
+        postings = termsEnum.postings(liveDocs, postings, PostingsEnum.FLAG_ALL);
 
         if (hasFreqs == false) {
           if (termsEnum.totalTermFreq() != -1) {
@@ -1051,7 +1051,7 @@ public class CheckIndex implements Close
           }
         }
         
-        final DocsEnum docs2;
+        final PostingsEnum docs2;
         if (postings != null) {
           docs2 = postings;
         } else {
@@ -1152,7 +1152,7 @@ public class CheckIndex implements Close
         // Re-count if there are deleted docs:
         if (liveDocs != null) {
           if (hasFreqs) {
-            final DocsEnum docsNoDel = termsEnum.docs(null, docsAndFreqs);
+            final PostingsEnum docsNoDel = termsEnum.postings(null, docsAndFreqs);
             docCount = 0;
             totalTermFreq = 0;
             while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
@@ -1161,7 +1161,7 @@ public class CheckIndex implements Close
               totalTermFreq += docsNoDel.freq();
             }
           } else {
-            final DocsEnum docsNoDel = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
+            final PostingsEnum docsNoDel = termsEnum.postings(null, docs, PostingsEnum.FLAG_NONE);
             docCount = 0;
             totalTermFreq = -1;
             while(docsNoDel.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
@@ -1188,7 +1188,7 @@ public class CheckIndex implements Close
         if (hasPositions) {
           for(int idx=0;idx<7;idx++) {
             final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
-            postings = termsEnum.docsAndPositions(liveDocs, postings);
+            postings = termsEnum.postings(liveDocs, postings, PostingsEnum.FLAG_ALL);
             final int docID = postings.advance(skipDocID);
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
@@ -1247,7 +1247,7 @@ public class CheckIndex implements Close
         } else {
           for(int idx=0;idx<7;idx++) {
             final int skipDocID = (int) (((idx+1)*(long) maxDoc)/8);
-            docs = termsEnum.docs(liveDocs, docs, DocsEnum.FLAG_NONE);
+            docs = termsEnum.postings(liveDocs, docs, PostingsEnum.FLAG_NONE);
             final int docID = docs.advance(skipDocID);
             if (docID == DocIdSetIterator.NO_MORE_DOCS) {
               break;
@@ -1315,7 +1315,7 @@ public class CheckIndex implements Close
           }
           
           int expectedDocFreq = termsEnum.docFreq();
-          DocsEnum d = termsEnum.docs(null, null, DocsEnum.FLAG_NONE);
+          PostingsEnum d = termsEnum.postings(null, null, PostingsEnum.FLAG_NONE);
           int docFreq = 0;
           while (d.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
             docFreq++;
@@ -1356,7 +1356,7 @@ public class CheckIndex implements Close
                 throw new RuntimeException("seek to existing term " + seekTerms[i] + " failed");
               }
               
-              docs = termsEnum.docs(liveDocs, docs, DocsEnum.FLAG_NONE);
+              docs = termsEnum.postings(liveDocs, docs, PostingsEnum.FLAG_NONE);
               if (docs == null) {
                 throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
               }
@@ -1374,7 +1374,7 @@ public class CheckIndex implements Close
               }
               
               totDocFreq += termsEnum.docFreq();
-              docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
+              docs = termsEnum.postings(null, docs, PostingsEnum.FLAG_NONE);
               if (docs == null) {
                 throw new RuntimeException("null DocsEnum from to existing term " + seekTerms[i]);
               }
@@ -1806,12 +1806,12 @@ public class CheckIndex implements Close
         infoStream.print("    test: term vectors........");
       }
 
-      DocsEnum docs = null;
-      DocsAndPositionsEnum postings = null;
+      PostingsEnum docs = null;
+      PostingsEnum postings = null;
 
       // Only used if crossCheckTermVectors is true:
-      DocsEnum postingsDocs = null;
-      DocsAndPositionsEnum postingsPostings = null;
+      PostingsEnum postingsDocs = null;
+      PostingsEnum postingsPostings = null;
 
       final Bits liveDocs = reader.getLiveDocs();
 
@@ -1878,16 +1878,16 @@ public class CheckIndex implements Close
               while ((term = termsEnum.next()) != null) {
 
                 if (hasProx) {
-                  postings = termsEnum.docsAndPositions(null, postings);
+                  postings = termsEnum.postings(null, postings, PostingsEnum.FLAG_ALL);
                   assert postings != null;
                   docs = null;
                 } else {
-                  docs = termsEnum.docs(null, docs);
+                  docs = termsEnum.postings(null, docs);
                   assert docs != null;
                   postings = null;
                 }
 
-                final DocsEnum docs2;
+                final PostingsEnum docs2;
                 if (hasProx) {
                   assert postings != null;
                   docs2 = postings;
@@ -1896,14 +1896,14 @@ public class CheckIndex implements Close
                   docs2 = docs;
                 }
 
-                final DocsEnum postingsDocs2;
+                final PostingsEnum postingsDocs2;
                 if (!postingsTermsEnum.seekExact(term)) {
                   throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
                 }
-                postingsPostings = postingsTermsEnum.docsAndPositions(null, postingsPostings);
+                postingsPostings = postingsTermsEnum.postings(null, postingsPostings, PostingsEnum.FLAG_ALL);
                 if (postingsPostings == null) {
                   // Term vectors were indexed w/ pos but postings were not
-                  postingsDocs = postingsTermsEnum.docs(null, postingsDocs);
+                  postingsDocs = postingsTermsEnum.postings(null, postingsDocs);
                   if (postingsDocs == null) {
                     throw new RuntimeException("vector term=" + term + " field=" + field + " does not exist in postings; doc=" + j);
                   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java Fri Feb  6 13:25:22 2015
@@ -19,49 +19,52 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.Bits; // javadocs
-
-/** Iterates through the documents and term freqs.
- *  NOTE: you must first call {@link #nextDoc} before using
- *  any of the per-doc methods. */
-public abstract class DocsEnum extends DocIdSetIterator {
-  
-  /**
-   * Flag to pass to {@link TermsEnum#docs(Bits,DocsEnum,int)} if you don't
-   * require term frequencies in the returned enum. When passed to
-   * {@link TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)} means
-   * that no offsets and payloads will be returned.
-   */
-  public static final int FLAG_NONE = 0x0;
+import org.apache.lucene.util.BytesRef;
 
-  /** Flag to pass to {@link TermsEnum#docs(Bits,DocsEnum,int)}
-   *  if you require term frequencies in the returned enum. */
-  public static final int FLAG_FREQS = 0x1;
+/**
+ * Convenience class returning empty values for positions, offsets and payloads
+ */
+public abstract class DocsEnum extends PostingsEnum {
 
-  private AttributeSource atts = null;
-
-  /** Sole constructor. (For invocation by subclass 
+  /** Sole constructor. (For invocation by subclass
    *  constructors, typically implicit.) */
   protected DocsEnum() {
+    super();
+  }
+
+  /**
+   * @return -1, indicating no positions are available
+   * @throws IOException if a low-level IO exception occurred
+   */
+  @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  /**
+   * @return -1, indicating no offsets are available
+   * @throws IOException if a low-level IO exception occurred
+   */
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  /**
+   * @return -1, indicating no offsets are available
+   * @throws IOException if a low-level IO exception occurred
+   */
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
   }
 
   /**
-   * Returns term frequency in the current document, or 1 if the field was
-   * indexed with {@link IndexOptions#DOCS}. Do not call this before
-   * {@link #nextDoc} is first called, nor after {@link #nextDoc} returns
-   * {@link DocIdSetIterator#NO_MORE_DOCS}.
-   * 
-   * <p>
-   * <b>NOTE:</b> if the {@link DocsEnum} was obtain with {@link #FLAG_NONE},
-   * the result of this method is undefined.
+   * @return null, indicating no payloads are available
+   * @throws IOException if a low-level IO exception occurred
    */
-  public abstract int freq() throws IOException;
-  
-  /** Returns the related attributes. */
-  public AttributeSource attributes() {
-    if (atts == null) atts = new AttributeSource();
-    return atts;
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
   }
 }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java Fri Feb  6 13:25:22 2015
@@ -215,26 +215,22 @@ public class FilterLeafReader extends Le
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-      return in.docs(liveDocs, reuse, flags);
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+      return in.postings(liveDocs, reuse, flags);
     }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      return in.docsAndPositions(liveDocs, reuse, flags);
-    }
   }
 
-  /** Base class for filtering {@link DocsEnum} implementations. */
-  public static class FilterDocsEnum extends DocsEnum {
+  /** Base class for filtering {@link PostingsEnum} implementations. */
+  public static class FilterDocsEnum extends PostingsEnum {
     /** The underlying DocsEnum instance. */
-    protected final DocsEnum in;
+    protected final PostingsEnum in;
 
     /**
      * Create a new FilterDocsEnum
      * @param in the underlying DocsEnum instance.
      */
-    public FilterDocsEnum(DocsEnum in) {
+    public FilterDocsEnum(PostingsEnum in) {
       if (in == null) {
         throw new NullPointerException("incoming DocsEnum cannot be null");
       }
@@ -267,53 +263,6 @@ public class FilterLeafReader extends Le
     }
 
     @Override
-    public long cost() {
-      return in.cost();
-    }
-  }
-
-  /** Base class for filtering {@link DocsAndPositionsEnum} implementations. */
-  public static class FilterDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    /** The underlying DocsAndPositionsEnum instance. */
-    protected final DocsAndPositionsEnum in;
-
-    /**
-     * Create a new FilterDocsAndPositionsEnum
-     * @param in the underlying DocsAndPositionsEnum instance.
-     */
-    public FilterDocsAndPositionsEnum(DocsAndPositionsEnum in) {
-      if (in == null) {
-        throw new NullPointerException("incoming DocsAndPositionsEnum cannot be null");
-      }
-      this.in = in;
-    }
-
-    @Override
-    public AttributeSource attributes() {
-      return in.attributes();
-    }
-
-    @Override
-    public int docID() {
-      return in.docID();
-    }
-
-    @Override
-    public int freq() throws IOException {
-      return in.freq();
-    }
-
-    @Override
-    public int nextDoc() throws IOException {
-      return in.nextDoc();
-    }
-
-    @Override
-    public int advance(int target) throws IOException {
-      return in.advance(target);
-    }
-
-    @Override
     public int nextPosition() throws IOException {
       return in.nextPosition();
     }
@@ -332,7 +281,7 @@ public class FilterLeafReader extends Le
     public BytesRef getPayload() throws IOException {
       return in.getPayload();
     }
-    
+
     @Override
     public long cost() {
       return in.cost();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java Fri Feb  6 13:25:22 2015
@@ -179,13 +179,8 @@ public abstract class FilteredTermsEnum
   }
 
   @Override
-  public DocsEnum docs(Bits bits, DocsEnum reuse, int flags) throws IOException {
-    return tenum.docs(bits, reuse, flags);
-  }
-    
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse, int flags) throws IOException {
-    return tenum.docsAndPositions(bits, reuse, flags);
+  public PostingsEnum postings(Bits bits, PostingsEnum reuse, int flags) throws IOException {
+    return tenum.postings(bits, reuse, flags);
   }
   
   /** This enum does not support seeking!