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

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

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/PushPostingsWriterBase.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsReader.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/IntersectTermsEnum.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/blocktree/SegmentTermsEnum.java Fri Feb  6 16:39:45 2015
@@ -21,9 +21,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 
 import org.apache.lucene.codecs.BlockTermState;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.store.ByteArrayDataInput;
@@ -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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsReader.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50DocValuesProducer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsFormat.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50PostingsReader.java Fri Feb  6 16:39:45 2015
@@ -26,11 +26,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;
@@ -196,39 +196,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 {
@@ -305,7 +304,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;
@@ -322,7 +321,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);
       }
@@ -340,6 +339,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;
     }
@@ -475,7 +480,7 @@ public final class Lucene50PostingsReade
   }
 
 
-  final class BlockDocsAndPositionsEnum extends DocsAndPositionsEnum {
+  final class BlockPostingsEnum extends PostingsEnum {
     
     private final byte[] encoded;
 
@@ -538,7 +543,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();
@@ -553,7 +558,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;
@@ -772,6 +777,9 @@ public final class Lucene50PostingsReade
 
     @Override
     public int nextPosition() throws IOException {
+
+      assert posPendingCount > 0;
+
       if (posPendingFP != -1) {
         posIn.seek(posPendingFP);
         posPendingFP = -1;
@@ -816,7 +824,7 @@ public final class Lucene50PostingsReade
   }
 
   // Also handles payloads + offsets
-  final class EverythingEnum extends DocsAndPositionsEnum {
+  final class EverythingEnum extends PostingsEnum {
     
     private final byte[] encoded;
 
@@ -963,8 +971,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;
@@ -1231,6 +1239,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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java Fri Feb  6 16:39:45 2015
@@ -380,7 +380,7 @@ class BufferedUpdatesStream implements A
     final int startDelCount;
 
     TermsEnum termsEnum;
-    DocsEnum docsEnum;
+    PostingsEnum postingsEnum;
     BytesRef term;
     boolean any;
 
@@ -568,12 +568,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;
             }
@@ -629,7 +629,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;
@@ -664,14 +664,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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java Fri Feb  6 16:39:45 2015
@@ -883,9 +883,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) {
@@ -1027,8 +1027,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) {
@@ -1052,7 +1052,7 @@ public class CheckIndex implements Close
           }
         }
         
-        final DocsEnum docs2;
+        final PostingsEnum docs2;
         if (postings != null) {
           docs2 = postings;
         } else {
@@ -1153,7 +1153,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) {
@@ -1162,7 +1162,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) {
@@ -1189,7 +1189,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;
@@ -1248,7 +1248,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;
@@ -1316,7 +1316,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++;
@@ -1357,7 +1357,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]);
               }
@@ -1375,7 +1375,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]);
               }
@@ -1807,12 +1807,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();
 
@@ -1879,16 +1879,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;
@@ -1897,14 +1897,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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/DocsEnum.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilterLeafReader.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FilteredTermsEnum.java Fri Feb  6 16:39:45 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!

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java Fri Feb  6 16:39:45 2015
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray;
-import org.apache.lucene.util.AttributeSource; // javadocs
+import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -230,14 +230,41 @@ class FreqProxFields extends Fields {
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
       if (liveDocs != null) {
         throw new IllegalArgumentException("liveDocs must be null");
       }
 
+      if ((flags & PostingsEnum.FLAG_POSITIONS) >= PostingsEnum.FLAG_POSITIONS) {
+        FreqProxPostingsEnum posEnum;
+
+        if (!terms.hasProx) {
+          // Caller wants positions but we didn't index them;
+          // don't lie:
+          throw new IllegalArgumentException("did not index positions");
+        }
+
+        if (!terms.hasOffsets && (flags & PostingsEnum.FLAG_OFFSETS) == PostingsEnum.FLAG_OFFSETS) {
+          // Caller wants offsets but we didn't index them;
+          // don't lie:
+          throw new IllegalArgumentException("did not index offsets");
+        }
+
+        if (reuse instanceof FreqProxPostingsEnum) {
+          posEnum = (FreqProxPostingsEnum) reuse;
+          if (posEnum.postingsArray != postingsArray) {
+            posEnum = new FreqProxPostingsEnum(terms, postingsArray);
+          }
+        } else {
+          posEnum = new FreqProxPostingsEnum(terms, postingsArray);
+        }
+        posEnum.reset(sortedTermIDs[ord]);
+        return posEnum;
+      }
+
       FreqProxDocsEnum docsEnum;
 
-      if (!terms.hasFreq && (flags & DocsEnum.FLAG_FREQS) != 0) {
+      if (!terms.hasFreq && (flags & PostingsEnum.FLAG_FREQS) != 0) {
         // Caller wants freqs but we didn't index them;
         // don't lie:
         throw new IllegalArgumentException("did not index freq");
@@ -255,37 +282,6 @@ class FreqProxFields extends Fields {
       return docsEnum;
     }
 
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
-      if (liveDocs != null) {
-        throw new IllegalArgumentException("liveDocs must be null");
-      }
-      FreqProxDocsAndPositionsEnum posEnum;
-
-      if (!terms.hasProx) {
-        // Caller wants positions but we didn't index them;
-        // don't lie:
-        throw new IllegalArgumentException("did not index positions");
-      }
-
-      if (!terms.hasOffsets && (flags & DocsAndPositionsEnum.FLAG_OFFSETS) != 0) {
-        // Caller wants offsets but we didn't index them;
-        // don't lie:
-        throw new IllegalArgumentException("did not index offsets");
-      }
-
-      if (reuse instanceof FreqProxDocsAndPositionsEnum) {
-        posEnum = (FreqProxDocsAndPositionsEnum) reuse;
-        if (posEnum.postingsArray != postingsArray) {
-          posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray);
-        }
-      } else {
-        posEnum = new FreqProxDocsAndPositionsEnum(terms, postingsArray);
-      }
-      posEnum.reset(sortedTermIDs[ord]);
-      return posEnum;
-    }
-
     /**
      * Expert: Returns the TermsEnums internal state to position the TermsEnum
      * without re-seeking the term dictionary.
@@ -348,6 +344,12 @@ class FreqProxFields extends Fields {
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      assert false : "Shouldn't be calling nextPositions on DocsEnum";
+      return -1;
+    }
+
+    @Override
     public int nextDoc() throws IOException {
       if (reader.eof()) {
         if (ended) {
@@ -389,7 +391,7 @@ class FreqProxFields extends Fields {
     }
   }
 
-  private static class FreqProxDocsAndPositionsEnum extends DocsAndPositionsEnum {
+  private static class FreqProxPostingsEnum extends PostingsEnum {
 
     final FreqProxTermsWriterPerField terms;
     final FreqProxPostingsArray postingsArray;
@@ -407,7 +409,7 @@ class FreqProxFields extends Fields {
     boolean hasPayload;
     BytesRefBuilder payload = new BytesRefBuilder();
 
-    public FreqProxDocsAndPositionsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) {
+    public FreqProxPostingsEnum(FreqProxTermsWriterPerField terms, FreqProxPostingsArray postingsArray) {
       this.terms = terms;
       this.postingsArray = postingsArray;
       this.readOffsets = terms.hasOffsets;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Fri Feb  6 16:39:45 2015
@@ -43,7 +43,7 @@ final class FreqProxTermsWriter extends
       Collections.sort(deleteTerms);
       String lastField = null;
       TermsEnum termsEnum = null;
-      DocsEnum docsEnum = null;
+      PostingsEnum postingsEnum = null;
       for(Term deleteTerm : deleteTerms) {
         if (deleteTerm.field().equals(lastField) == false) {
           lastField = deleteTerm.field();
@@ -56,11 +56,11 @@ final class FreqProxTermsWriter extends
         }
 
         if (termsEnum != null && termsEnum.seekExact(deleteTerm.bytes())) {
-          docsEnum = termsEnum.docs(null, docsEnum, 0);
+          postingsEnum = termsEnum.postings(null, postingsEnum, 0);
           int delDocLimit = segDeletes.get(deleteTerm);
-          assert delDocLimit < DocsEnum.NO_MORE_DOCS;
+          assert delDocLimit < PostingsEnum.NO_MORE_DOCS;
           while (true) {
-            int doc = docsEnum.nextDoc();
+            int doc = postingsEnum.nextDoc();
             if (doc < delDocLimit) {
               if (state.liveDocs == null) {
                 state.liveDocs = state.segmentInfo.getCodec().liveDocsFormat().newLiveDocs(state.segmentInfo.getDocCount());

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/LeafReader.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/LeafReader.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/LeafReader.java Fri Feb  6 16:39:45 2015
@@ -17,11 +17,11 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.index.IndexReader.ReaderClosedListener;
 import org.apache.lucene.util.Bits;
 
+import java.io.IOException;
+
 /** {@code LeafReader} is an abstract class, providing an interface for accessing an
  index.  Search of an index is done entirely through this abstract interface,
  so that any subclass which implements it is searchable. IndexReaders implemented
@@ -205,38 +205,27 @@ public abstract class LeafReader extends
     return fields().terms(field);
   }
 
-  /** Returns {@link DocsEnum} for the specified term.
+  /** Returns {@link PostingsEnum} for the specified term.
    *  This will return null if either the field or
    *  term does not exist.
-   *  @see TermsEnum#docs(Bits, DocsEnum) */
-  public final DocsEnum termDocsEnum(Term term) throws IOException {
+   *  @see TermsEnum#postings(Bits, PostingsEnum) */
+  public final PostingsEnum termDocsEnum(Term term, int flags) throws IOException {
     assert term.field() != null;
     assert term.bytes() != null;
     final Terms terms = terms(term.field());
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term.bytes())) {
-        return termsEnum.docs(getLiveDocs(), null);
+        return termsEnum.postings(getLiveDocs(), null, flags);
       }
     }
     return null;
   }
 
-  /** Returns {@link DocsAndPositionsEnum} for the specified
-   *  term.  This will return null if the
-   *  field or term does not exist or positions weren't indexed.
-   *  @see TermsEnum#docsAndPositions(Bits, DocsAndPositionsEnum) */
-  public final DocsAndPositionsEnum termPositionsEnum(Term term) throws IOException {
-    assert term.field() != null;
-    assert term.bytes() != null;
-    final Terms terms = terms(term.field());
-    if (terms != null) {
-      final TermsEnum termsEnum = terms.iterator(null);
-      if (termsEnum.seekExact(term.bytes())) {
-        return termsEnum.docsAndPositions(getLiveDocs(), null);
-      }
-    }
-    return null;
+  /** Returns {@link PostingsEnum} for the specified term
+   *  with {@link PostingsEnum#FLAG_FREQS}. */
+  public final PostingsEnum termDocsEnum(Term term) throws IOException {
+    return termDocsEnum(term, PostingsEnum.FLAG_FREQS);
   }
 
   /** Returns {@link NumericDocValues} for this field, or

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java Fri Feb  6 16:39:45 2015
@@ -103,10 +103,23 @@ public class MappedMultiFields extends F
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
       if (liveDocs != null) {
         throw new IllegalArgumentException("liveDocs must be null");
       }
+
+      MappingMultiPostingsEnum mappingDocsAndPositionsEnum;
+      if (reuse instanceof MappingMultiPostingsEnum) {
+        mappingDocsAndPositionsEnum = (MappingMultiPostingsEnum) reuse;
+      } else {
+        mappingDocsAndPositionsEnum = new MappingMultiPostingsEnum(mergeState);
+      }
+
+      MultiPostingsEnum docsAndPositionsEnum = (MultiPostingsEnum) in.postings(liveDocs, mappingDocsAndPositionsEnum.multiDocsAndPositionsEnum, flags);
+      mappingDocsAndPositionsEnum.reset(docsAndPositionsEnum);
+      return mappingDocsAndPositionsEnum;
+
+/*
       MappingMultiDocsEnum mappingDocsEnum;
       if (reuse instanceof MappingMultiDocsEnum) {
         mappingDocsEnum = (MappingMultiDocsEnum) reuse;
@@ -116,24 +129,7 @@ public class MappedMultiFields extends F
       
       MultiDocsEnum docsEnum = (MultiDocsEnum) in.docs(liveDocs, mappingDocsEnum.multiDocsEnum, flags);
       mappingDocsEnum.reset(docsEnum);
-      return mappingDocsEnum;
-    }
-
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-      if (liveDocs != null) {
-        throw new IllegalArgumentException("liveDocs must be null");
-      }
-      MappingMultiDocsAndPositionsEnum mappingDocsAndPositionsEnum;
-      if (reuse instanceof MappingMultiDocsAndPositionsEnum) {
-        mappingDocsAndPositionsEnum = (MappingMultiDocsAndPositionsEnum) reuse;
-      } else {
-        mappingDocsAndPositionsEnum = new MappingMultiDocsAndPositionsEnum(mergeState);
-      }
-      
-      MultiDocsAndPositionsEnum docsAndPositionsEnum = (MultiDocsAndPositionsEnum) in.docsAndPositions(liveDocs, mappingDocsAndPositionsEnum.multiDocsAndPositionsEnum, flags);
-      mappingDocsAndPositionsEnum.reset(docsAndPositionsEnum);
-      return mappingDocsAndPositionsEnum;
+      return mappingDocsEnum;*/
     }
   }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Fri Feb  6 16:39:45 2015
@@ -120,55 +120,55 @@ public final class MultiFields extends F
     return getFields(r).terms(field);
   }
   
-  /** Returns {@link DocsEnum} for the specified field and
+  /** Returns {@link PostingsEnum} for the specified field and
    *  term.  This will return null if the field or term does
    *  not exist. */
-  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
-    return getTermDocsEnum(r, liveDocs, field, term, DocsEnum.FLAG_FREQS);
+  public static PostingsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
+    return getTermDocsEnum(r, liveDocs, field, term, PostingsEnum.FLAG_FREQS);
   }
   
-  /** Returns {@link DocsEnum} for the specified field and
+  /** Returns {@link PostingsEnum} for the specified field and
    *  term, with control over whether freqs are required.
    *  Some codecs may be able to optimize their
    *  implementation when freqs are not required.  This will
    *  return null if the field or term does not exist.  See {@link
-   *  TermsEnum#docs(Bits,DocsEnum,int)}.*/
-  public static DocsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
+   *  TermsEnum#postings(Bits, PostingsEnum,int)}.*/
+  public static PostingsEnum getTermDocsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term)) {
-        return termsEnum.docs(liveDocs, null, flags);
+        return termsEnum.postings(liveDocs, null, flags);
       }
     }
     return null;
   }
 
-  /** Returns {@link DocsAndPositionsEnum} for the specified
+  /** Returns {@link PostingsEnum} for the specified
    *  field and term.  This will return null if the field or
    *  term does not exist or positions were not indexed. 
    *  @see #getTermPositionsEnum(IndexReader, Bits, String, BytesRef, int) */
-  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
-    return getTermPositionsEnum(r, liveDocs, field, term, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
+  public static PostingsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term) throws IOException {
+    return getTermPositionsEnum(r, liveDocs, field, term, PostingsEnum.FLAG_OFFSETS | PostingsEnum.FLAG_PAYLOADS);
   }
 
-  /** Returns {@link DocsAndPositionsEnum} for the specified
+  /** Returns {@link PostingsEnum} for the specified
    *  field and term, with control over whether offsets and payloads are
    *  required.  Some codecs may be able to optimize
    *  their implementation when offsets and/or payloads are not
    *  required. This will return null if the field or term does not
    *  exist or positions were not indexed. See {@link
-   *  TermsEnum#docsAndPositions(Bits,DocsAndPositionsEnum,int)}. */
-  public static DocsAndPositionsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
+   *  TermsEnum#postings(Bits, PostingsEnum,int)}. */
+  public static PostingsEnum getTermPositionsEnum(IndexReader r, Bits liveDocs, String field, BytesRef term, int flags) throws IOException {
     assert field != null;
     assert term != null;
     final Terms terms = getTerms(r, field);
     if (terms != null) {
       final TermsEnum termsEnum = terms.iterator(null);
       if (termsEnum.seekExact(term)) {
-        return termsEnum.docsAndPositions(liveDocs, null, flags);
+        return termsEnum.postings(liveDocs, null, flags);
       }
     }
     return null;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Fri Feb  6 16:39:45 2015
@@ -17,14 +17,14 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.PriorityQueue;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.Bits;
-
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.PriorityQueue;
+
 /**
  * Exposes {@link TermsEnum} API, merged from {@link TermsEnum} API of sub-segments.
  * This does a merge sort, by term text, of the sub-readers.
@@ -37,8 +37,7 @@ public final class MultiTermsEnum extend
   private final TermsEnumWithSlice[] subs;        // all of our subs (one per sub-reader)
   private final TermsEnumWithSlice[] currentSubs; // current subs that have at least one term for this field
   private final TermsEnumWithSlice[] top;
-  private final MultiDocsEnum.EnumWithSlice[] subDocs;
-  private final MultiDocsAndPositionsEnum.EnumWithSlice[] subDocsAndPositions;
+  private final MultiPostingsEnum.EnumWithSlice[] subDocs;
 
   private BytesRef lastSeek;
   private boolean lastSeekExact;
@@ -77,14 +76,11 @@ public final class MultiTermsEnum extend
     queue = new TermMergeQueue(slices.length);
     top = new TermsEnumWithSlice[slices.length];
     subs = new TermsEnumWithSlice[slices.length];
-    subDocs = new MultiDocsEnum.EnumWithSlice[slices.length];
-    subDocsAndPositions = new MultiDocsAndPositionsEnum.EnumWithSlice[slices.length];
+    subDocs = new MultiPostingsEnum.EnumWithSlice[slices.length];
     for(int i=0;i<slices.length;i++) {
       subs[i] = new TermsEnumWithSlice(i, slices[i]);
-      subDocs[i] = new MultiDocsEnum.EnumWithSlice();
+      subDocs[i] = new MultiPostingsEnum.EnumWithSlice();
       subDocs[i].slice = slices[i];
-      subDocsAndPositions[i] = new MultiDocsAndPositionsEnum.EnumWithSlice();
-      subDocsAndPositions[i].slice = slices[i];
     }
     currentSubs = new TermsEnumWithSlice[slices.length];
   }
@@ -331,17 +327,18 @@ public final class MultiTermsEnum extend
   }
 
   @Override
-  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-    MultiDocsEnum docsEnum;
+  public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+    MultiPostingsEnum docsEnum;
+
     // Can only reuse if incoming enum is also a MultiDocsEnum
-    if (reuse != null && reuse instanceof MultiDocsEnum) {
-      docsEnum = (MultiDocsEnum) reuse;
+    if (reuse != null && reuse instanceof MultiPostingsEnum) {
+      docsEnum = (MultiPostingsEnum) reuse;
       // ... and was previously created w/ this MultiTermsEnum:
       if (!docsEnum.canReuse(this)) {
-        docsEnum = new MultiDocsEnum(this, subs.length);
+        docsEnum = new MultiPostingsEnum(this, subs.length);
       }
     } else {
-      docsEnum = new MultiDocsEnum(this, subs.length);
+      docsEnum = new MultiPostingsEnum(this, subs.length);
     }
     
     final MultiBits multiLiveDocs;
@@ -380,16 +377,16 @@ public final class MultiTermsEnum extend
         b = null;
       }
 
-      assert entry.index < docsEnum.subDocsEnum.length: entry.index + " vs " + docsEnum.subDocsEnum.length + "; " + subs.length;
-      final DocsEnum subDocsEnum = entry.terms.docs(b, docsEnum.subDocsEnum[entry.index], flags);
-      if (subDocsEnum != null) {
-        docsEnum.subDocsEnum[entry.index] = subDocsEnum;
-        subDocs[upto].docsEnum = subDocsEnum;
+      assert entry.index < docsEnum.subPostingsEnums.length: entry.index + " vs " + docsEnum.subPostingsEnums.length + "; " + subs.length;
+      final PostingsEnum subPostingsEnum = entry.terms.postings(b, docsEnum.subPostingsEnums[entry.index], flags);
+      if (subPostingsEnum != null) {
+        docsEnum.subPostingsEnums[entry.index] = subPostingsEnum;
+        subDocs[upto].postingsEnum = subPostingsEnum;
         subDocs[upto].slice = entry.subSlice;
         upto++;
       } else {
         // should this be an error?
-        assert false : "One of our subs cannot provide a docsenum";
+        return null;    // We can't support what is being asked for
       }
     }
 
@@ -400,82 +397,6 @@ public final class MultiTermsEnum extend
     }
   }
 
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException {
-    MultiDocsAndPositionsEnum docsAndPositionsEnum;
-    // Can only reuse if incoming enum is also a MultiDocsAndPositionsEnum
-    if (reuse != null && reuse instanceof MultiDocsAndPositionsEnum) {
-      docsAndPositionsEnum = (MultiDocsAndPositionsEnum) reuse;
-      // ... and was previously created w/ this MultiTermsEnum:
-      if (!docsAndPositionsEnum.canReuse(this)) {
-        docsAndPositionsEnum = new MultiDocsAndPositionsEnum(this, subs.length);
-      }
-    } else {
-      docsAndPositionsEnum = new MultiDocsAndPositionsEnum(this, subs.length);
-    }
-    
-    final MultiBits multiLiveDocs;
-    if (liveDocs instanceof MultiBits) {
-      multiLiveDocs = (MultiBits) liveDocs;
-    } else {
-      multiLiveDocs = null;
-    }
-
-    int upto = 0;
-
-    for(int i=0;i<numTop;i++) {
-
-      final TermsEnumWithSlice entry = top[i];
-
-      final Bits b;
-
-      if (multiLiveDocs != null) {
-        // Optimize for common case: requested skip docs is a
-        // congruent sub-slice of MultiBits: in this case, we
-        // just pull the liveDocs from the sub reader, rather
-        // than making the inefficient
-        // Slice(Multi(sub-readers)):
-        final MultiBits.SubResult sub = multiLiveDocs.getMatchingSub(top[i].subSlice);
-        if (sub.matches) {
-          b = sub.result;
-        } else {
-          // custom case: requested skip docs is foreign:
-          // must slice it on every access (very
-          // inefficient)
-          b = new BitsSlice(liveDocs, top[i].subSlice);
-        }
-      } else if (liveDocs != null) {
-        b = new BitsSlice(liveDocs, top[i].subSlice);
-      } else {
-        // no deletions
-        b = null;
-      }
-
-      assert entry.index < docsAndPositionsEnum.subDocsAndPositionsEnum.length: entry.index + " vs " + docsAndPositionsEnum.subDocsAndPositionsEnum.length + "; " + subs.length;
-      final DocsAndPositionsEnum subPostings = entry.terms.docsAndPositions(b, docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index], flags);
-
-      if (subPostings != null) {
-        docsAndPositionsEnum.subDocsAndPositionsEnum[entry.index] = subPostings;
-        subDocsAndPositions[upto].docsAndPositionsEnum = subPostings;
-        subDocsAndPositions[upto].slice = entry.subSlice;
-        upto++;
-      } else {
-        if (entry.terms.docs(b, null, DocsEnum.FLAG_NONE) != null) {
-          // At least one of our subs does not store
-          // offsets or positions -- we can't correctly
-          // produce a MultiDocsAndPositions enum
-          return null;
-        }
-      }
-    }
-
-    if (upto == 0) {
-      return null;
-    } else {
-      return docsAndPositionsEnum.reset(subDocsAndPositions, upto);
-    }
-  }
-
   final static class TermsEnumWithSlice {
     private final ReaderSlice subSlice;
     TermsEnum terms;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java Fri Feb  6 16:39:45 2015
@@ -106,12 +106,7 @@ class SortedDocValuesTermsEnum extends T
   }
 
   @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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java Fri Feb  6 16:39:45 2015
@@ -17,12 +17,12 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
+import java.io.IOException;
+
 /** Implements a {@link TermsEnum} wrapping a provided
  * {@link SortedSetDocValues}. */
 
@@ -106,12 +106,7 @@ class SortedSetDocValuesTermsEnum extend
   }
 
   @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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermContext.java Fri Feb  6 16:39:45 2015
@@ -17,11 +17,11 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.util.BytesRef;
+
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.lucene.util.BytesRef;
-
 /**
  * Maintains a {@link IndexReader} {@link TermState} view over
  * {@link IndexReader} instances containing a single term. The

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/Terms.java Fri Feb  6 16:39:45 2015
@@ -111,7 +111,7 @@ public abstract class Terms {
   public abstract int getDocCount() throws IOException;
 
   /** Returns true if documents in this field store
-   *  per-document term frequency ({@link DocsEnum#freq}). */
+   *  per-document term frequency ({@link PostingsEnum#freq}). */
   public abstract boolean hasFreqs();
 
   /** Returns true if documents in this field store offsets. */

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Fri Feb  6 16:39:45 2015
@@ -27,9 +27,9 @@ import org.apache.lucene.util.BytesRefIt
 /** Iterator to seek ({@link #seekCeil(BytesRef)}, {@link
  * #seekExact(BytesRef)}) or step through ({@link
  * #next} terms to obtain frequency information ({@link
- * #docFreq}), {@link DocsEnum} or {@link
- * DocsAndPositionsEnum} for the current term ({@link
- * #docs}.
+ * #docFreq}), {@link PostingsEnum} or {@link
+ * PostingsEnum} for the current term ({@link
+ * #postings}.
  * 
  * <p>Term enumerations are always ordered by
  * BytesRef.compareTo, which is Unicode sort
@@ -138,57 +138,30 @@ public abstract class TermsEnum implemen
    *  deleted documents into account. */
   public abstract long totalTermFreq() throws IOException;
 
-  /** Get {@link DocsEnum} for the current term.  Do not
+  /** Get {@link PostingsEnum} for the current term.  Do not
    *  call this when the enum is unpositioned.  This method
    *  will not return null.
    *  
    * @param liveDocs unset bits are documents that should not
    * be returned
-   * @param reuse pass a prior DocsEnum for possible reuse */
-  public final DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
-    return docs(liveDocs, reuse, DocsEnum.FLAG_FREQS);
+   * @param reuse pass a prior PostingsEnum for possible reuse */
+  public final PostingsEnum postings(Bits liveDocs, PostingsEnum reuse) throws IOException {
+    return postings(liveDocs, reuse, PostingsEnum.FLAG_FREQS);
   }
 
-  /** Get {@link DocsEnum} for the current term, with
-   *  control over whether freqs are required.  Do not
-   *  call this when the enum is unpositioned.  This method
-   *  will not return null.
+  /** Get {@link PostingsEnum} for the current term, with
+   *  control over whether freqs, positions, offsets or payloads
+   *  are required.  Do not call this when the enum is
+   *  unpositioned.  This method may return null if the postings
+   *  information required is not available from the index
    *  
    * @param liveDocs unset bits are documents that should not
    * be returned
-   * @param reuse pass a prior DocsEnum for possible reuse
+   * @param reuse pass a prior PostingsEnum for possible reuse
    * @param flags specifies which optional per-document values
-   *        you require; see {@link DocsEnum#FLAG_FREQS} 
-   * @see #docs(Bits, DocsEnum, int) */
-  public abstract DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException;
-
-  /** Get {@link DocsAndPositionsEnum} for the current term.
-   *  Do not call this when the enum is unpositioned.  This
-   *  method will return null if positions were not
-   *  indexed.
-   *  
-   *  @param liveDocs unset bits are documents that should not
-   *  be returned
-   *  @param reuse pass a prior DocsAndPositionsEnum for possible reuse
-   *  @see #docsAndPositions(Bits, DocsAndPositionsEnum, int) */
-  public final DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
-    return docsAndPositions(liveDocs, reuse, DocsAndPositionsEnum.FLAG_OFFSETS | DocsAndPositionsEnum.FLAG_PAYLOADS);
-  }
-
-  /** Get {@link DocsAndPositionsEnum} for the current term,
-   *  with control over whether offsets and payloads are
-   *  required.  Some codecs may be able to optimize their
-   *  implementation when offsets and/or payloads are not required.
-   *  Do not call this when the enum is unpositioned.  This
-   *  will return null if positions were not indexed.
-
-   *  @param liveDocs unset bits are documents that should not
-   *  be returned
-   *  @param reuse pass a prior DocsAndPositionsEnum for possible reuse
-   *  @param flags specifies which optional per-position values you
-   *         require; see {@link DocsAndPositionsEnum#FLAG_OFFSETS} and 
-   *         {@link DocsAndPositionsEnum#FLAG_PAYLOADS}. */
-  public abstract DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) throws IOException;
+   *        you require; see {@link PostingsEnum#FLAG_FREQS}
+   * @see #postings(Bits, PostingsEnum, int) */
+  public abstract PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException;
 
   /**
    * Expert: Returns the TermsEnums internal state to position the TermsEnum
@@ -245,12 +218,7 @@ public abstract class TermsEnum implemen
     }
 
     @Override
-    public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) {
-      throw new IllegalStateException("this method should never be called");
-    }
-      
-    @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse, int flags) {
+    public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) {
       throw new IllegalStateException("this method should never be called");
     }
       
@@ -273,5 +241,6 @@ public abstract class TermsEnum implemen
     public void seekExact(BytesRef term, TermState state) {
       throw new IllegalStateException("this method should never be called");
     }
+
   };
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/index/package.html Fri Feb  6 16:39:45 2015
@@ -99,9 +99,7 @@ if (found) {
   // get the document frequency
   System.out.println(termsEnum.docFreq());
   // enumerate through documents
-  DocsEnum docs = termsEnum.docs(null, null);
-  // enumerate through documents and positions
-  DocsAndPositionsEnum docsAndPositions = termsEnum.docsAndPositions(null, null);
+  PostingsEnum docs = termsEnum.postings(null, null);
 }
 </pre>
 </p>
@@ -110,7 +108,7 @@ if (found) {
     Documents
 </h4>
 <p>
-{@link org.apache.lucene.index.DocsEnum} is an extension of 
+{@link org.apache.lucene.index.PostingsEnum} is an extension of
 {@link org.apache.lucene.search.DocIdSetIterator}that iterates over the list of
 documents for a term, along with the term frequency within that document.
 <pre class="prettyprint">
@@ -126,20 +124,21 @@ while ((docid = docsEnum.nextDoc()) != D
     Positions
 </h4>
 <p>
-{@link org.apache.lucene.index.DocsAndPositionsEnum} is an extension of 
-{@link org.apache.lucene.index.DocsEnum} that additionally allows iteration
+PostingsEnum also allows iteration
 of the positions a term occurred within the document, and any additional
-per-position information (offsets and payload)
+per-position information (offsets and payload).  The information available
+is controlled by flags passed to TermsEnum#postings
 <pre class="prettyprint">
 int docid;
-while ((docid = docsAndPositionsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+PostingsEnum postings = termsEnum.postings(null, null, PostingsEnum.FLAG_PAYLOADS | PostingsEnum.FLAG_OFFSETS);
+while ((docid = postings.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
   System.out.println(docid);
-  int freq = docsAndPositionsEnum.freq();
+  int freq = postings.freq();
   for (int i = 0; i < freq; i++) {
-     System.out.println(docsAndPositionsEnum.nextPosition());
-     System.out.println(docsAndPositionsEnum.startOffset());
-     System.out.println(docsAndPositionsEnum.endOffset());
-     System.out.println(docsAndPositionsEnum.getPayload());
+     System.out.println(postings.nextPosition());
+     System.out.println(postings.startOffset());
+     System.out.println(postings.endOffset());
+     System.out.println(postings.getPayload());
   }
 }
 </pre>