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 [3/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/core/src/java/org/apache/lucene/index/FreqProxFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxFields.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/index/LeafReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/LeafReader.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/LeafReader.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/LeafReader.java Fri Feb  6 13:25:22 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,25 @@ 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;
+  public final PostingsEnum termDocsEnum(Term term) throws IOException {
+    return termDocsEnum(term, PostingsEnum.FLAG_FREQS);
   }
 
   /** Returns {@link NumericDocValues} for this field, or

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappedMultiFields.java Fri Feb  6 13:25:22 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;*/
     }
   }
 }

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java?rev=1657800&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MappingMultiPostingsEnum.java Fri Feb  6 13:25:22 2015
@@ -0,0 +1,141 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.MultiPostingsEnum.EnumWithSlice;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Exposes flex API, merged from flex API of sub-segments,
+ * remapping docIDs (this is used for segment merging).
+ *
+ * @lucene.experimental
+ */
+
+final class MappingMultiPostingsEnum extends PostingsEnum {
+  private MultiPostingsEnum.EnumWithSlice[] subs;
+  int numSubs;
+  int upto;
+  MergeState.DocMap currentMap;
+  PostingsEnum current;
+  int currentBase;
+  int doc = -1;
+  private MergeState mergeState;
+  MultiPostingsEnum multiDocsAndPositionsEnum;
+
+  /** Sole constructor. */
+  public MappingMultiPostingsEnum(MergeState mergeState) {
+    this.mergeState = mergeState;
+  }
+
+  MappingMultiPostingsEnum reset(MultiPostingsEnum postingsEnum) {
+    this.numSubs = postingsEnum.getNumSubs();
+    this.subs = postingsEnum.getSubs();
+    upto = -1;
+    current = null;
+    this.multiDocsAndPositionsEnum = postingsEnum;
+    return this;
+  }
+
+  /** How many sub-readers we are merging.
+   *  @see #getSubs */
+  public int getNumSubs() {
+    return numSubs;
+  }
+
+  /** Returns sub-readers we are merging. */
+  public EnumWithSlice[] getSubs() {
+    return subs;
+  }
+
+  @Override
+  public int freq() throws IOException {
+    return current.freq();
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    while(true) {
+      if (current == null) {
+        if (upto == numSubs-1) {
+          return this.doc = NO_MORE_DOCS;
+        } else {
+          upto++;
+          final int reader = subs[upto].slice.readerIndex;
+          current = subs[upto].postingsEnum;
+          currentBase = mergeState.docBase[reader];
+          currentMap = mergeState.docMaps[reader];
+        }
+      }
+
+      int doc = current.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        // compact deletions
+        doc = currentMap.get(doc);
+        if (doc == -1) {
+          continue;
+        }
+        return this.doc = currentBase + doc;
+      } else {
+        current = null;
+      }
+    }
+  }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return current.nextPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return current.startOffset();
+  }
+  
+  @Override
+  public int endOffset() throws IOException {
+    return current.endOffset();
+  }
+  
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return current.getPayload();
+  }
+
+  @Override
+  public long cost() {
+    long cost = 0;
+    for (EnumWithSlice enumWithSlice : subs) {
+      cost += enumWithSlice.postingsEnum.cost();
+    }
+    return cost;
+  }
+}
+

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiFields.java Fri Feb  6 13:25:22 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;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java?rev=1657800&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiPostingsEnum.java Fri Feb  6 13:25:22 2015
@@ -0,0 +1,198 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Exposes {@link PostingsEnum}, merged from {@link PostingsEnum}
+ * API of sub-segments.
+ *
+ * @lucene.experimental
+ */
+
+public final class MultiPostingsEnum extends PostingsEnum {
+  private final MultiTermsEnum parent;
+  final PostingsEnum[] subPostingsEnums;
+  private final EnumWithSlice[] subs;
+  int numSubs;
+  int upto;
+  PostingsEnum current;
+  int currentBase;
+  int doc = -1;
+
+  /** Sole constructor.
+   * @param parent The {@link MultiTermsEnum} that created us.
+   * @param subReaderCount How many sub-readers are being merged. */
+  public MultiPostingsEnum(MultiTermsEnum parent, int subReaderCount) {
+    this.parent = parent;
+    subPostingsEnums = new PostingsEnum[subReaderCount];
+    this.subs = new EnumWithSlice[subReaderCount];
+    for (int i = 0; i < subs.length; i++) {
+      subs[i] = new EnumWithSlice();
+    }
+  }
+
+  /** Returns {@code true} if this instance can be reused by
+   *  the provided {@link MultiTermsEnum}. */
+  public boolean canReuse(MultiTermsEnum parent) {
+    return this.parent == parent;
+  }
+
+  /** Rre-use and reset this instance on the provided slices. */
+  public MultiPostingsEnum reset(final EnumWithSlice[] subs, final int numSubs) {
+    this.numSubs = numSubs;
+    for(int i=0;i<numSubs;i++) {
+      this.subs[i].postingsEnum = subs[i].postingsEnum;
+      this.subs[i].slice = subs[i].slice;
+    }
+    upto = -1;
+    doc = -1;
+    current = null;
+    return this;
+  }
+
+  /** How many sub-readers we are merging.
+   *  @see #getSubs */
+  public int getNumSubs() {
+    return numSubs;
+  }
+
+  /** Returns sub-readers we are merging. */
+  public EnumWithSlice[] getSubs() {
+    return subs;
+  }
+
+  @Override
+  public int freq() throws IOException {
+    assert current != null;
+    return current.freq();
+  }
+
+  @Override
+  public int docID() {
+    return doc;
+  }
+
+  @Override
+  public int advance(int target) throws IOException {
+    assert target > doc;
+    while(true) {
+      if (current != null) {
+        final int doc;
+        if (target < currentBase) {
+          // target was in the previous slice but there was no matching doc after it
+          doc = current.nextDoc();
+        } else {
+          doc = current.advance(target-currentBase);
+        }
+        if (doc == NO_MORE_DOCS) {
+          current = null;
+        } else {
+          return this.doc = doc + currentBase;
+        }
+      } else if (upto == numSubs-1) {
+        return this.doc = NO_MORE_DOCS;
+      } else {
+        upto++;
+        current = subs[upto].postingsEnum;
+        currentBase = subs[upto].slice.start;
+      }
+    }
+  }
+
+  @Override
+  public int nextDoc() throws IOException {
+    while(true) {
+      if (current == null) {
+        if (upto == numSubs-1) {
+          return this.doc = NO_MORE_DOCS;
+        } else {
+          upto++;
+          current = subs[upto].postingsEnum;
+          currentBase = subs[upto].slice.start;
+        }
+      }
+
+      final int doc = current.nextDoc();
+      if (doc != NO_MORE_DOCS) {
+        return this.doc = currentBase + doc;
+      } else {
+        current = null;
+      }
+    }
+  }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return current.nextPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return current.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return current.endOffset();
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return current.getPayload();
+  }
+
+  // TODO: implement bulk read more efficiently than super
+  /** Holds a {@link PostingsEnum} along with the
+   *  corresponding {@link ReaderSlice}. */
+  public final static class EnumWithSlice {
+    EnumWithSlice() {
+    }
+
+    /** {@link PostingsEnum} for this sub-reader. */
+    public PostingsEnum postingsEnum;
+
+    /** {@link ReaderSlice} describing how this sub-reader
+     *  fits into the composite reader. */
+    public ReaderSlice slice;
+    
+    @Override
+    public String toString() {
+      return slice.toString()+":"+ postingsEnum;
+    }
+  }
+  
+  @Override
+  public long cost() {
+    long cost = 0;
+    for (int i = 0; i < numSubs; i++) {
+      cost += subs[i].postingsEnum.cost();
+    }
+    return cost;
+  }
+  
+  @Override
+  public String toString() {
+    return "MultiDocsAndPositionsEnum(" + Arrays.toString(getSubs()) + ")";
+  }
+}
+

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/MultiTermsEnum.java Fri Feb  6 13:25:22 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;

Added: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java?rev=1657800&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java (added)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/PostingsEnum.java Fri Feb  6 13:25:22 2015
@@ -0,0 +1,114 @@
+package org.apache.lucene.index;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+
+/** Iterates through the postings.
+ *  NOTE: you must first call {@link #nextDoc} before using
+ *  any of the per-doc methods. */
+public abstract class PostingsEnum extends DocIdSetIterator {
+  
+  /**
+   * Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)} if you don't
+   * require per-document postings in the returned enum.
+   */
+  public static final int FLAG_NONE = 0x0;
+
+  /** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
+   *  if you require term frequencies in the returned enum. */
+  public static final int FLAG_FREQS = 0x1;
+
+  /** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
+   * if you require term positions in the returned enum. */
+  public static final int FLAG_POSITIONS = 0x3;
+  
+  /** Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
+   *  if you require offsets in the returned enum. */
+  public static final int FLAG_OFFSETS = 0x7;
+
+  /** Flag to pass to  {@link TermsEnum#postings(Bits, PostingsEnum, int)}
+   *  if you require payloads in the returned enum. */
+  public static final int FLAG_PAYLOADS = 0xB;
+
+  /**
+   * Flag to pass to {@link TermsEnum#postings(Bits, PostingsEnum, int)}
+   * to get positions, payloads and offsets in the returned enum
+   */
+  public static final int FLAG_ALL = FLAG_POSITIONS | FLAG_PAYLOADS;
+
+  /**
+   * Returns true if the passed in flags require positions to be indexed
+   * @param flags the postings flags
+   * @return true if the passed in flags require positions to be indexed
+   */
+  public static boolean requiresPositions(int flags) {
+    return ((flags & FLAG_POSITIONS) >= FLAG_POSITIONS);
+  }
+
+  private AttributeSource atts = null;
+
+  /** Sole constructor. (For invocation by subclass 
+   *  constructors, typically implicit.) */
+  protected PostingsEnum() {
+  }
+
+  /**
+   * 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 PostingsEnum} was obtain with {@link #FLAG_NONE},
+   * the result of this method is undefined.
+   */
+  public abstract int freq() throws IOException;
+  
+  /** Returns the related attributes. */
+  public AttributeSource attributes() {
+    if (atts == null) atts = new AttributeSource();
+    return atts;
+  }
+
+  /**
+   * Returns the next position.  If there are no more
+   * positions, or the iterator does not support positions,
+   * this will return DocsEnum.NO_MORE_POSITIONS */
+  public abstract int nextPosition() throws IOException;
+
+  /** Returns start offset for the current position, or -1
+   *  if offsets were not indexed. */
+  public abstract int startOffset() throws IOException;
+
+  /** Returns end offset for the current position, or -1 if
+   *  offsets were not indexed. */
+  public abstract int endOffset() throws IOException;
+
+  /** Returns the payload at this position, or null if no
+   *  payload was indexed. You should not modify anything 
+   *  (neither members of the returned BytesRef nor bytes 
+   *  in the byte[]). */
+  public abstract BytesRef getPayload() throws IOException;
+
+}

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedDocValuesTermsEnum.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/SortedSetDocValuesTermsEnum.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermContext.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/Terms.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/index/TermsEnum.java Fri Feb  6 13:25:22 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/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java Fri Feb  6 13:25:22 2015
@@ -17,15 +17,16 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.RamUsageEstimator;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+
 /**
  * Caches all docs, and optionally also scores, coming from
  * a search, and is then able to replay them to another
@@ -74,10 +75,31 @@ public abstract class CachingCollector e
     public final int freq() { throw new UnsupportedOperationException(); }
 
     @Override
+    public int nextPosition() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
     public final int nextDoc() { throw new UnsupportedOperationException(); }
 
     @Override
     public long cost() { return 1; }
+
   }
 
   private static class NoScoreCachingCollector extends CachingCollector {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/Collector.java Fri Feb  6 13:25:22 2015
@@ -72,7 +72,7 @@ public interface Collector {
    *          next atomic reader context
    */
   LeafCollector getLeafCollector(LeafReaderContext context) throws IOException;
-
+  
   /**
    * Indicates if document scores are needed by this collector.
    * 

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Feb  6 13:25:22 2015
@@ -23,9 +23,11 @@ import java.util.Collection;
 import java.util.Comparator;
 
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 
 /** Scorer for conjunctions, sets of queries, all of which are required. */
 class ConjunctionScorer extends Scorer {
+
   protected int lastDoc = -1;
   protected final DocsAndFreqs[] docsAndFreqs;
   private final DocsAndFreqs lead;
@@ -34,7 +36,7 @@ class ConjunctionScorer extends Scorer {
   ConjunctionScorer(Weight weight, Scorer[] scorers) {
     this(weight, scorers, 1f);
   }
-  
+
   ConjunctionScorer(Weight weight, Scorer[] scorers, float coord) {
     super(weight);
     this.coord = coord;
@@ -109,13 +111,33 @@ class ConjunctionScorer extends Scorer {
     }
     return sum * coord;
   }
-  
+
   @Override
   public int freq() {
     return docsAndFreqs.length;
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
+
+  @Override
   public long cost() {
     return lead.scorer.cost();
   }
@@ -133,7 +155,7 @@ class ConjunctionScorer extends Scorer {
     final long cost;
     final Scorer scorer;
     int doc = -1;
-   
+
     DocsAndFreqs(Scorer scorer) {
       this.scorer = scorer;
       this.cost = scorer.cost();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Fri Feb  6 13:25:22 2015
@@ -17,17 +17,18 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.ToStringUtils;
-
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ToStringUtils;
+
 /**
  * A query that wraps another query or a filter and simply returns a constant score equal to the
  * query boost for every document that matches the filter or query.
@@ -135,7 +136,6 @@ public class ConstantScoreQuery extends
 
     @Override
     public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
-      final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
         return super.bulkScorer(context, acceptDocs, needsScores);
@@ -151,23 +151,26 @@ public class ConstantScoreQuery extends
 
     @Override
     public Scorer scorer(LeafReaderContext context, Bits acceptDocs, boolean needsScores) throws IOException {
-      final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
         final DocIdSet dis = filter.getDocIdSet(context, acceptDocs);
         if (dis == null) {
           return null;
         }
-        disi = dis.iterator();
+        final DocIdSetIterator disi = dis.iterator();
+        if (disi == null)
+          return null;
+        return new ConstantDocIdSetIteratorScorer(disi, this, queryWeight);
       } else {
         assert query != null && innerWeight != null;
-        disi = innerWeight.scorer(context, acceptDocs, false);
+        Scorer scorer = innerWeight.scorer(context, acceptDocs, false);
+        if (scorer == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(scorer, queryWeight);
       }
 
-      if (disi == null) {
-        return null;
-      }
-      return new ConstantScorer(disi, this, queryWeight);
+
     }
 
     @Override
@@ -216,7 +219,7 @@ public class ConstantScoreQuery extends
         @Override
         public void setScorer(Scorer scorer) throws IOException {
           // we must wrap again here, but using the scorer passed in as parameter:
-          in.setScorer(new ConstantScorer(scorer, weight, theScore));
+          in.setScorer(new ConstantScoreScorer(scorer, theScore));
         }
       };
     }
@@ -227,11 +230,40 @@ public class ConstantScoreQuery extends
     }
   }
 
-  protected class ConstantScorer extends Scorer {
+  protected class ConstantScoreScorer extends FilterScorer {
+
+    private final float score;
+
+    public ConstantScoreScorer(Scorer wrapped, float score) {
+      super(wrapped);
+      this.score = score;
+    }
+
+    @Override
+    public int freq() throws IOException {
+      return 1;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return score;
+    }
+
+    @Override
+    public Collection<ChildScorer> getChildren() {
+      if (query != null) {
+        return Collections.singletonList(new ChildScorer(in, "constant"));
+      } else {
+        return Collections.emptyList();
+      }
+    }
+  }
+
+  protected class ConstantDocIdSetIteratorScorer extends Scorer {
     final DocIdSetIterator docIdSetIterator;
     final float theScore;
 
-    public ConstantScorer(DocIdSetIterator docIdSetIterator, Weight w, float theScore) {
+    public ConstantDocIdSetIteratorScorer(DocIdSetIterator docIdSetIterator, Weight w, float theScore) {
       super(w);
       this.theScore = theScore;
       this.docIdSetIterator = docIdSetIterator;
@@ -259,10 +291,30 @@ public class ConstantScoreQuery extends
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return null;
+    }
+
+    @Override
     public int advance(int target) throws IOException {
       return docIdSetIterator.advance(target);
     }
-    
+
     @Override
     public long cost() {
       return docIdSetIterator.cost();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Fri Feb  6 13:25:22 2015
@@ -22,15 +22,18 @@ import java.util.ArrayList;
 import java.util.Collection;
 
 import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * Base class for Scorers that score disjunctions.
  */
 abstract class DisjunctionScorer extends Scorer {
+
   private final ScorerPriorityQueue subScorers;
 
   /** The document number of the current match. */
   protected int doc = -1;
+  protected int numScorers;
   /** Number of matching scorers for the current match. */
   private int freq = -1;
   /** Linked list of scorers which are on the current doc */
@@ -57,6 +60,26 @@ abstract class DisjunctionScorer extends
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
+
+  @Override
   public final long cost() {
     long sum = 0;
     for (ScorerWrapper scorer : subScorers) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java Fri Feb  6 13:25:22 2015
@@ -21,7 +21,6 @@ import java.io.IOException;
 
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;
-import org.apache.lucene.index.DocsEnum; // javadoc @link
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.util.Bits;
@@ -63,7 +62,7 @@ import org.apache.lucene.util.FixedBitSe
  * 
  * In contrast, TermsFilter builds up an {@link FixedBitSet},
  * keyed by docID, every time it's created, by enumerating
- * through all matching docs using {@link DocsEnum} to seek
+ * through all matching docs using {@link org.apache.lucene.index.PostingsEnum} to seek
  * and scan through each term's docID list.  While there is
  * no linear scan of all docIDs, besides the allocation of
  * the underlying array in the {@link FixedBitSet}, this

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Fri Feb  6 13:25:22 2015
@@ -22,6 +22,7 @@ import java.util.Arrays;
 
 import org.apache.lucene.index.*;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
 
 final class ExactPhraseScorer extends Scorer {
   private final int endMinus1;
@@ -35,21 +36,21 @@ final class ExactPhraseScorer extends Sc
   private final long cost;
 
   private final static class ChunkState {
-    final DocsAndPositionsEnum posEnum;
+    final PostingsEnum posEnum;
     final int offset;
     int posUpto;
     int posLimit;
     int pos;
     int lastPos;
 
-    public ChunkState(DocsAndPositionsEnum posEnum, int offset) {
+    public ChunkState(PostingsEnum posEnum, int offset) {
       this.posEnum = posEnum;
       this.offset = offset;
     }
   }
 
   private final ChunkState[] chunkStates;
-  private final DocsAndPositionsEnum lead;
+  private final PostingsEnum lead;
 
   private int docID = -1;
   private int freq;
@@ -81,7 +82,7 @@ final class ExactPhraseScorer extends Sc
       // TODO: don't dup this logic from conjunctionscorer :)
       advanceHead: for(;;) {
         for (int i = 1; i < chunkStates.length; i++) {
-          final DocsAndPositionsEnum de = chunkStates[i].posEnum;
+          final PostingsEnum de = chunkStates[i].posEnum;
           if (de.docID() < doc) {
             int d = de.advance(doc);
 
@@ -127,6 +128,26 @@ final class ExactPhraseScorer extends Sc
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
+
+  @Override
   public int docID() {
     return docID;
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java Fri Feb  6 13:25:22 2015
@@ -17,8 +17,11 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.Collection;
 
+import org.apache.lucene.util.BytesRef;
+
 /** Used by {@link BulkScorer}s that need to pass a {@link
  *  Scorer} to {@link LeafCollector#setScorer}. */
 final class FakeScorer extends Scorer {
@@ -46,6 +49,26 @@ final class FakeScorer extends Scorer {
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    throw new UnsupportedOperationException("FakeScorer doesn't support nextPosition()");
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    throw new UnsupportedOperationException("FakeScorer doesn't support startOffset()");
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    throw new UnsupportedOperationException("FakeScorer doesn't support endOffset()");
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    throw new UnsupportedOperationException("FakeScorer doesn't support getPayload()");
+  }
+
+  @Override
   public int nextDoc() {
     throw new UnsupportedOperationException("FakeScorer doesn't support nextDoc()");
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java Fri Feb  6 13:25:22 2015
@@ -18,9 +18,9 @@ package org.apache.lucene.search;
  */
 
 import java.io.IOException;
-import java.util.Collection;
 
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.BytesRef;
 
 /** 
  * A {@code FilterScorer} contains another {@code Scorer}, which it
@@ -32,13 +32,27 @@ import org.apache.lucene.util.AttributeS
  * further override some of these methods and may also provide additional
  * methods and fields.
  */
-abstract class FilterScorer extends Scorer {
+public abstract class FilterScorer extends Scorer {
   protected final Scorer in;
-  
+
+  /**
+   * Create a new FilterScorer
+   * @param in the {@link Scorer} to wrap
+   */
   public FilterScorer(Scorer in) {
     super(in.weight);
     this.in = in;
   }
+
+  /**
+   * Create a new FilterScorer with a specific weight
+   * @param in the {@link Scorer} to wrap
+   * @param weight a {@link Weight}
+   */
+  public FilterScorer(Scorer in, Weight weight) {
+    super(weight);
+    this.in = in;
+  }
   
   @Override
   public float score() throws IOException {
@@ -61,6 +75,11 @@ abstract class FilterScorer extends Scor
   }
 
   @Override
+  public int nextPosition() throws IOException {
+    return in.nextPosition();
+  }
+
+  @Override
   public int advance(int target) throws IOException {
     return in.advance(target);
   }
@@ -71,6 +90,21 @@ abstract class FilterScorer extends Scor
   }
 
   @Override
+  public int startOffset() throws IOException {
+    return in.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return in.endOffset();
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return in.getPayload();
+  }
+
+  @Override
   public AttributeSource attributes() {
     return in.attributes();
   }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Fri Feb  6 13:25:22 2015
@@ -17,17 +17,17 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.ToStringUtils;
-
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.ToStringUtils;
+
 
 /**
  * A query that applies a filter to the results of another query.
@@ -143,6 +143,7 @@ public class FilteredQuery extends Query
         }
 
         return strategy.filteredBulkScorer(context, weight, filterDocIdSet, needsScores);
+
       }
     };
   }
@@ -153,13 +154,13 @@ public class FilteredQuery extends Query
    * than document scoring or if the filter has a linear running time to compute
    * the next matching doc like exact geo distances.
    */
-  private static final class QueryFirstScorer extends Scorer {
+  private static final class QueryFirstScorer extends FilterScorer {
     private final Scorer scorer;
     private int scorerDoc = -1;
     private final Bits filterBits;
 
     protected QueryFirstScorer(Weight weight, Bits filterBits, Scorer other) {
-      super(weight);
+      super(other, weight);
       this.scorer = other;
       this.filterBits = filterBits;
     }
@@ -184,29 +185,16 @@ public class FilteredQuery extends Query
         return scorerDoc = doc;
       }
     }
-
     @Override
     public int docID() {
       return scorerDoc;
     }
-    
-    @Override
-    public float score() throws IOException {
-      return scorer.score();
-    }
-    
-    @Override
-    public int freq() throws IOException { return scorer.freq(); }
-    
+
     @Override
     public Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
     }
 
-    @Override
-    public long cost() {
-      return scorer.cost();
-    }
   }
 
   private static class QueryFirstBulkScorer extends BulkScorer {
@@ -254,7 +242,7 @@ public class FilteredQuery extends Query
    * jumping past the target document. When both land on the same document, it's
    * collected.
    */
-  private static final class LeapFrogScorer extends Scorer {
+  private static final class LeapFrogScorer extends FilterScorer {
     private final DocIdSetIterator secondary;
     private final DocIdSetIterator primary;
     private final Scorer scorer;
@@ -262,7 +250,7 @@ public class FilteredQuery extends Query
     private int secondaryDoc = -1;
 
     protected LeapFrogScorer(Weight weight, DocIdSetIterator primary, DocIdSetIterator secondary, Scorer scorer) {
-      super(weight);
+      super(scorer, weight);
       this.primary = primary;
       this.secondary = secondary;
       this.scorer = scorer;
@@ -302,17 +290,7 @@ public class FilteredQuery extends Query
     public final int docID() {
       return secondaryDoc;
     }
-    
-    @Override
-    public final float score() throws IOException {
-      return scorer.score();
-    }
-    
-    @Override
-    public final int freq() throws IOException {
-      return scorer.freq();
-    }
-    
+
     @Override
     public final Collection<ChildScorer> getChildren() {
       return Collections.singleton(new ChildScorer(scorer, "FILTERED"));
@@ -489,6 +467,7 @@ public class FilteredQuery extends Query
       // ignore scoreDocsInOrder:
       return new Weight.DefaultBulkScorer(scorer);
     }
+
   }
   
   /**
@@ -595,8 +574,7 @@ public class FilteredQuery extends Query
         return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet, needsScores);
       }
       final Scorer scorer = weight.scorer(context, null, needsScores);
-      return scorer == null ? null : new QueryFirstScorer(weight,
-          filterAcceptDocs, scorer);
+      return scorer == null ? null : new QueryFirstScorer(weight, filterAcceptDocs, scorer);
     }
 
     @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Fri Feb  6 13:25:22 2015
@@ -22,8 +22,7 @@ import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.FilteredTermsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermState;
@@ -266,14 +265,8 @@ public class FuzzyTermsEnum extends Term
   }
   
   @Override
-  public DocsEnum docs(Bits liveDocs, DocsEnum reuse, int flags) throws IOException {
-    return actualEnum.docs(liveDocs, reuse, flags);
-  }
-  
-  @Override
-  public DocsAndPositionsEnum docsAndPositions(Bits liveDocs,
-                                               DocsAndPositionsEnum reuse, int flags) throws IOException {
-    return actualEnum.docsAndPositions(liveDocs, reuse, flags);
+  public PostingsEnum postings(Bits liveDocs, PostingsEnum reuse, int flags) throws IOException {
+    return actualEnum.postings(liveDocs, reuse, flags);
   }
   
   @Override

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Fri Feb  6 13:25:22 2015
@@ -17,14 +17,15 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
+import java.io.IOException;
+import java.util.Set;
+
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.Bits;
-
-import java.util.Set;
-import java.io.IOException;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ToStringUtils;
 
 /**
  * A query that matches all documents.
@@ -73,6 +74,26 @@ public class MatchAllDocsQuery extends Q
     }
 
     @Override
+    public int nextPosition() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int startOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public int endOffset() throws IOException {
+      return -1;
+    }
+
+    @Override
+    public BytesRef getPayload() throws IOException {
+      return null;
+    }
+
+    @Override
     public int advance(int target) throws IOException {
       doc = target-1;
       return nextDoc();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java Fri Feb  6 13:25:22 2015
@@ -17,10 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import static org.apache.lucene.search.ScorerPriorityQueue.leftNode;
-import static org.apache.lucene.search.ScorerPriorityQueue.parentNode;
-import static org.apache.lucene.search.ScorerPriorityQueue.rightNode;
-
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -28,8 +24,13 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.lucene.search.ScorerPriorityQueue.ScorerWrapper;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.PriorityQueue;
 
+import static org.apache.lucene.search.ScorerPriorityQueue.leftNode;
+import static org.apache.lucene.search.ScorerPriorityQueue.parentNode;
+import static org.apache.lucene.search.ScorerPriorityQueue.rightNode;
+
 /**
  * A {@link Scorer} for {@link BooleanQuery} when
  * {@link BooleanQuery#setMinimumNumberShouldMatch(int) minShouldMatch} is
@@ -229,6 +230,26 @@ final class MinShouldMatchSumScorer exte
     }
   }
 
+  @Override
+  public int nextPosition() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return -1;
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return null;
+  }
+
   /** Advance tail to the lead until there is a match. */
   private int doNext() throws IOException {
     while (freq < minShouldMatch) {

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java Fri Feb  6 13:25:22 2015
@@ -111,7 +111,6 @@ public class MultiCollector implements C
     return new MultiLeafCollector(leafCollectors);
   }
 
-
   private static class MultiLeafCollector implements LeafCollector {
 
     private final LeafCollector[] collectors;

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri Feb  6 13:25:22 2015
@@ -21,9 +21,8 @@ import java.io.IOException;
 import java.util.*;
 
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.Term;
@@ -197,11 +196,11 @@ public class MultiPhraseQuery extends Qu
       for (int pos=0; pos<postingsFreqs.length; pos++) {
         Term[] terms = termArrays.get(pos);
 
-        final DocsAndPositionsEnum postingsEnum;
+        final PostingsEnum postingsEnum;
         int docFreq;
 
         if (terms.length > 1) {
-          postingsEnum = new UnionDocsAndPositionsEnum(liveDocs, context, terms, termContexts, termsEnum);
+          postingsEnum = new UnionPostingsEnum(liveDocs, context, terms, termContexts, termsEnum);
 
           // coarse -- this overcounts since a given doc can
           // have more than one term:
@@ -229,11 +228,11 @@ public class MultiPhraseQuery extends Qu
             return null;
           }
           termsEnum.seekExact(term.bytes(), termState);
-          postingsEnum = termsEnum.docsAndPositions(liveDocs, null, DocsEnum.FLAG_NONE);
+          postingsEnum = termsEnum.postings(liveDocs, null, PostingsEnum.FLAG_POSITIONS);
 
           if (postingsEnum == null) {
             // term does exist, but has no positions
-            assert termsEnum.docs(liveDocs, null, DocsEnum.FLAG_NONE) != null: "termstate found but no term exists in reader";
+            assert termsEnum.postings(liveDocs, null, PostingsEnum.FLAG_NONE) != null: "termstate found but no term exists in reader";
             throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
           }
 
@@ -408,15 +407,15 @@ public class MultiPhraseQuery extends Qu
  */
 
 // TODO: if ever we allow subclassing of the *PhraseScorer
-class UnionDocsAndPositionsEnum extends DocsAndPositionsEnum {
+class UnionPostingsEnum extends PostingsEnum {
 
-  private static final class DocsQueue extends PriorityQueue<DocsAndPositionsEnum> {
-    DocsQueue(List<DocsAndPositionsEnum> docsEnums) throws IOException {
-      super(docsEnums.size());
+  private static final class DocsQueue extends PriorityQueue<PostingsEnum> {
+    DocsQueue(List<PostingsEnum> postingsEnums) throws IOException {
+      super(postingsEnums.size());
 
-      Iterator<DocsAndPositionsEnum> i = docsEnums.iterator();
+      Iterator<PostingsEnum> i = postingsEnums.iterator();
       while (i.hasNext()) {
-        DocsAndPositionsEnum postings = i.next();
+        PostingsEnum postings = i.next();
         if (postings.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
           add(postings);
         }
@@ -424,7 +423,7 @@ class UnionDocsAndPositionsEnum extends
     }
 
     @Override
-    public final boolean lessThan(DocsAndPositionsEnum a, DocsAndPositionsEnum b) {
+    public final boolean lessThan(PostingsEnum a, PostingsEnum b) {
       return a.docID() < b.docID();
     }
   }
@@ -473,8 +472,8 @@ class UnionDocsAndPositionsEnum extends
   private IntQueue _posList;
   private long cost;
 
-  public UnionDocsAndPositionsEnum(Bits liveDocs, LeafReaderContext context, Term[] terms, Map<Term,TermContext> termContexts, TermsEnum termsEnum) throws IOException {
-    List<DocsAndPositionsEnum> docsEnums = new LinkedList<>();
+  public UnionPostingsEnum(Bits liveDocs, LeafReaderContext context, Term[] terms, Map<Term, TermContext> termContexts, TermsEnum termsEnum) throws IOException {
+    List<PostingsEnum> postingsEnums = new LinkedList<>();
     for (int i = 0; i < terms.length; i++) {
       final Term term = terms[i];
       TermState termState = termContexts.get(term).get(context.ord);
@@ -483,16 +482,16 @@ class UnionDocsAndPositionsEnum extends
         continue;
       }
       termsEnum.seekExact(term.bytes(), termState);
-      DocsAndPositionsEnum postings = termsEnum.docsAndPositions(liveDocs, null, DocsEnum.FLAG_NONE);
+      PostingsEnum postings = termsEnum.postings(liveDocs, null, PostingsEnum.FLAG_POSITIONS);
       if (postings == null) {
         // term does exist, but has no positions
         throw new IllegalStateException("field \"" + term.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + term.text() + ")");
       }
       cost += postings.cost();
-      docsEnums.add(postings);
+      postingsEnums.add(postings);
     }
 
-    _queue = new DocsQueue(docsEnums);
+    _queue = new DocsQueue(postingsEnums);
     _posList = new IntQueue();
   }
 
@@ -509,7 +508,7 @@ class UnionDocsAndPositionsEnum extends
     _doc = _queue.top().docID();
 
     // merge sort all positions together
-    DocsAndPositionsEnum postings;
+    PostingsEnum postings;
     do {
       postings = _queue.top();
 
@@ -554,7 +553,7 @@ class UnionDocsAndPositionsEnum extends
   @Override
   public final int advance(int target) throws IOException {
     while (_queue.top() != null && target > _queue.top().docID()) {
-      DocsAndPositionsEnum postings = _queue.pop();
+      PostingsEnum postings = _queue.pop();
       if (postings.advance(target) != NO_MORE_DOCS) {
         _queue.add(postings);
       }

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Fri Feb  6 13:25:22 2015
@@ -19,9 +19,7 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.Fields;
-import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
@@ -94,9 +92,9 @@ public class MultiTermQueryWrapperFilter
     assert termsEnum != null;
 
     BitDocIdSet.Builder builder = new BitDocIdSet.Builder(context.reader().maxDoc());
-    DocsEnum docs = null;
+    PostingsEnum docs = null;
     while (termsEnum.next() != null) {
-      docs = termsEnum.docs(acceptDocs, docs, DocsEnum.FLAG_NONE);
+      docs = termsEnum.postings(acceptDocs, docs, PostingsEnum.FLAG_NONE);
       builder.or(docs);
     }
     return builder.build();

Modified: lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java?rev=1657800&r1=1657799&r2=1657800&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java (original)
+++ lucene/dev/trunk/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java Fri Feb  6 13:25:22 2015
@@ -29,13 +29,13 @@ final class PhrasePositions {
   int count;            // remaining pos in this doc
   int offset;           // position in phrase
   final int ord;                                  // unique across all PhrasePositions instances
-  final DocsAndPositionsEnum postings;            // stream of docs & positions
+  final PostingsEnum postings;            // stream of docs & positions
   PhrasePositions next;                           // used to make lists
   int rptGroup = -1; // >=0 indicates that this is a repeating PP
   int rptInd; // index in the rptGroup
   final Term[] terms; // for repetitions initialization 
 
-  PhrasePositions(DocsAndPositionsEnum postings, int o, int ord, Term[] terms) {
+  PhrasePositions(PostingsEnum postings, int o, int ord, Term[] terms) {
     this.postings = postings;
     offset = o;
     this.ord = ord;