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 [4/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/search/CachingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/CachingCollector.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Collector.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConjunctionScorer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ConstantScoreQuery.java Fri Feb  6 16:39:45 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.
@@ -131,7 +132,6 @@ public class ConstantScoreQuery extends
 
     @Override
     public BulkScorer bulkScorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
-      final DocIdSetIterator disi;
       if (filter != null) {
         assert query == null;
         return super.bulkScorer(context, acceptDocs);
@@ -147,23 +147,26 @@ public class ConstantScoreQuery extends
 
     @Override
     public Scorer scorer(LeafReaderContext context, Bits acceptDocs) 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);
+        Scorer scorer = innerWeight.scorer(context, acceptDocs);
+        if (scorer == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(scorer, queryWeight);
       }
 
-      if (disi == null) {
-        return null;
-      }
-      return new ConstantScorer(disi, this, queryWeight);
+
     }
 
     @Override
@@ -212,7 +215,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));
         }
       };
     }
@@ -223,11 +226,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;
@@ -255,10 +287,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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DisjunctionScorer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/DocValuesTermsFilter.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FakeScorer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilterScorer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FilteredQuery.java Fri Feb  6 16:39:45 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.
@@ -137,6 +137,7 @@ public class FilteredQuery extends Query
         }
 
         return strategy.filteredBulkScorer(context, weight, filterDocIdSet);
+
       }
     };
   }
@@ -147,13 +148,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;
     }
@@ -178,29 +179,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 {
@@ -248,7 +236,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;
@@ -256,7 +244,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;
@@ -296,17 +284,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"));
@@ -483,6 +461,7 @@ public class FilteredQuery extends Query
       // ignore scoreDocsInOrder:
       return new Weight.DefaultBulkScorer(scorer);
     }
+
   }
   
   /**
@@ -589,8 +568,7 @@ public class FilteredQuery extends Query
         return LEAP_FROG_QUERY_FIRST_STRATEGY.filteredScorer(context, weight, docIdSet);
       }
       final Scorer scorer = weight.scorer(context, null);
-      return scorer == null ? null : new QueryFirstScorer(weight,
-          filterAcceptDocs, scorer);
+      return scorer == null ? null : new QueryFirstScorer(weight, filterAcceptDocs, scorer);
     }
 
     @Override

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MatchAllDocsQuery.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MinShouldMatchSumScorer.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiCollector.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/MultiTermQueryWrapperFilter.java Fri Feb  6 16:39:45 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/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhrasePositions.java Fri Feb  6 16:39:45 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;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Fri Feb  6 16:39:45 2015
@@ -22,19 +22,18 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Set;
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
@@ -138,13 +137,13 @@ public class PhraseQuery extends Query {
   }
 
   static class PostingsAndFreq implements Comparable<PostingsAndFreq> {
-    final DocsAndPositionsEnum postings;
+    final PostingsEnum postings;
     final int docFreq;
     final int position;
     final Term[] terms;
     final int nTerms; // for faster comparisons
 
-    public PostingsAndFreq(DocsAndPositionsEnum postings, int docFreq, int position, Term... terms) {
+    public PostingsAndFreq(PostingsEnum postings, int docFreq, int position, Term... terms) {
       this.postings = postings;
       this.docFreq = docFreq;
       this.position = position;
@@ -267,7 +266,7 @@ public class PhraseQuery extends Query {
           return null;
         }
         te.seekExact(t.bytes(), state);
-        DocsAndPositionsEnum postingsEnum = te.docsAndPositions(liveDocs, null, DocsEnum.FLAG_NONE);
+        PostingsEnum postingsEnum = te.postings(liveDocs, null, PostingsEnum.FLAG_POSITIONS);
 
         // PhraseQuery on a field that did not index
         // positions.
@@ -276,7 +275,7 @@ public class PhraseQuery extends Query {
           // term does exist, but has no positions
           throw new IllegalStateException("field \"" + t.field() + "\" was indexed without position data; cannot run PhraseQuery (term=" + t.text() + ")");
         }
-        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, te.docFreq(), positions.get(i).intValue(), t);
+        postingsFreqs[i] = new PostingsAndFreq(postingsEnum, te.docFreq(), positions.get(i), t);
       }
 
       // sort by increasing docFreq order

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryRescorer.java Fri Feb  6 16:39:45 2015
@@ -17,13 +17,14 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.LeafReaderContext;
+
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 
-import org.apache.lucene.index.LeafReaderContext;
-
 /** A {@link Rescorer} that uses a provided Query to assign
  *  scores to the first-pass hits.
  *

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/QueryWrapperFilter.java Fri Feb  6 16:39:45 2015
@@ -17,11 +17,12 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import java.io.IOException;
-
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.Bits;
 
+import java.io.IOException;
+
 /** 
  * Constrains search results to only match those which also match a provided
  * query.  

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqExclScorer.java Fri Feb  6 16:39:45 2015
@@ -27,7 +27,7 @@ import java.util.Collections;
  * This <code>Scorer</code> implements {@link Scorer#advance(int)},
  * and it uses the skipTo() on the given scorers.
  */
-class ReqExclScorer extends Scorer {
+class ReqExclScorer extends FilterScorer {
   private Scorer reqScorer;
   private DocIdSetIterator exclDisi;
   private int doc = -1;
@@ -37,7 +37,7 @@ class ReqExclScorer extends Scorer {
    * @param exclDisi indicates exclusion.
    */
   public ReqExclScorer(Scorer reqScorer, DocIdSetIterator exclDisi) {
-    super(reqScorer.weight);
+    super(reqScorer);
     this.reqScorer = reqScorer;
     this.exclDisi = exclDisi;
   }
@@ -103,11 +103,6 @@ class ReqExclScorer extends Scorer {
   public float score() throws IOException {
     return reqScorer.score(); // reqScorer may be null when next() or skipTo() already return false
   }
-  
-  @Override
-  public int freq() throws IOException {
-    return reqScorer.freq();
-  }
 
   @Override
   public Collection<ChildScorer> getChildren() {
@@ -129,8 +124,4 @@ class ReqExclScorer extends Scorer {
     return doc = toNonExcluded();
   }
 
-  @Override
-  public long cost() {
-    return reqScorer.cost();
-  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/ReqOptSumScorer.java Fri Feb  6 16:39:45 2015
@@ -20,6 +20,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.apache.lucene.util.BytesRef;
+
 /** A Scorer for queries with a required part and an optional part.
  * Delays skipTo() on the optional part until a score() is needed.
  * <br>
@@ -93,6 +95,26 @@ class ReqOptSumScorer extends Scorer {
   }
 
   @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 Collection<ChildScorer> getChildren() {
     ArrayList<ChildScorer> children = new ArrayList<>(2);
     children.add(new ChildScorer(reqScorer, "MUST"));

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Scorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Scorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Scorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Scorer.java Fri Feb  6 16:39:45 2015
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 
 /**
  * Expert: Common scoring functionality for different types of queries.
@@ -41,7 +41,7 @@ import org.apache.lucene.index.DocsEnum;
  * TopScoreDocCollector}) will not properly collect hits
  * with these scores.
  */
-public abstract class Scorer extends DocsEnum {
+public abstract class Scorer extends PostingsEnum {
   /** the Scorer's parent Weight. in some cases this may be null */
   // TODO can we clean this up?
   protected final Weight weight;

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Fri Feb  6 16:39:45 2015
@@ -27,6 +27,7 @@ import java.util.LinkedHashMap;
 
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 
 final class SloppyPhraseScorer extends Scorer {
@@ -527,7 +528,27 @@ final class SloppyPhraseScorer extends S
   public int freq() {
     return numMatches;
   }
-  
+
+  @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;
+  }
+
   float sloppyFreq() {
     return sloppyFreq;
   }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermQuery.java Fri Feb  6 16:39:45 2015
@@ -20,28 +20,29 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Set;
 
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
-/** A Query that matches documents containing a term.
-  This may be combined with other terms with a {@link BooleanQuery}.
-  */
+/**
+ * A Query that matches documents containing a term. This may be combined with
+ * other terms with a {@link BooleanQuery}.
+ */
 public class TermQuery extends Query {
   private final Term term;
   private final int docFreq;
   private final TermContext perReaderTermState;
-
+  
   final class TermWeight extends Weight {
     private final Similarity similarity;
     private final Similarity.SimWeight stats;
@@ -49,31 +50,32 @@ public class TermQuery extends Query {
     private final boolean needsScores;
 
     public TermWeight(IndexSearcher searcher, boolean needsScores, TermContext termStates)
-      throws IOException {
+        throws IOException {
       super(TermQuery.this);
       this.needsScores = needsScores;
       assert termStates != null : "TermContext must not be null";
       this.termStates = termStates;
       this.similarity = searcher.getSimilarity();
-      this.stats = similarity.computeWeight(
-          getBoost(), 
-          searcher.collectionStatistics(term.field()), 
+      this.stats = similarity.computeWeight(getBoost(),
+          searcher.collectionStatistics(term.field()),
           searcher.termStatistics(term, termStates));
     }
-
+    
     @Override
-    public String toString() { return "weight(" + TermQuery.this + ")"; }
-
+    public String toString() {
+      return "weight(" + TermQuery.this + ")";
+    }
+    
     @Override
     public float getValueForNormalization() {
       return stats.getValueForNormalization();
     }
-
+    
     @Override
     public void normalize(float queryNorm, float topLevelBoost) {
       stats.normalize(queryNorm, topLevelBoost);
     }
-
+    
     @Override
     public Scorer scorer(LeafReaderContext context, Bits acceptDocs) throws IOException {
       assert termStates.topReaderContext == ReaderUtil.getTopLevelContext(context) : "The top-reader used to create Weight (" + termStates.topReaderContext + ") is not the same as the current reader's top-reader (" + ReaderUtil.getTopLevelContext(context);
@@ -81,7 +83,7 @@ public class TermQuery extends Query {
       if (termsEnum == null) {
         return null;
       }
-      DocsEnum docs = termsEnum.docs(acceptDocs, null, needsScores ? DocsEnum.FLAG_FREQS : DocsEnum.FLAG_NONE);
+      PostingsEnum docs = termsEnum.postings(acceptDocs, null, needsScores ? PostingsEnum.FLAG_FREQS : PostingsEnum.FLAG_NONE);
       assert docs != null;
       return new TermScorer(this, docs, similarity.simScorer(stats, context));
     }
@@ -96,15 +98,18 @@ public class TermQuery extends Query {
         assert termNotInReader(context.reader(), term) : "no termstate found but term exists in reader term=" + term;
         return null;
       }
-      //System.out.println("LD=" + reader.getLiveDocs() + " set?=" + (reader.getLiveDocs() != null ? reader.getLiveDocs().get(0) : "null"));
-      final TermsEnum termsEnum = context.reader().terms(term.field()).iterator(null);
+      // System.out.println("LD=" + reader.getLiveDocs() + " set?=" +
+      // (reader.getLiveDocs() != null ? reader.getLiveDocs().get(0) : "null"));
+      final TermsEnum termsEnum = context.reader().terms(term.field())
+          .iterator(null);
       termsEnum.seekExact(term.bytes(), state);
       return termsEnum;
     }
     
     private boolean termNotInReader(LeafReader reader, Term term) throws IOException {
       // only called from assert
-      //System.out.println("TQ.termNotInReader reader=" + reader + " term=" + field + ":" + bytes.utf8ToString());
+      // System.out.println("TQ.termNotInReader reader=" + reader + " term=" +
+      // field + ":" + bytes.utf8ToString());
       return reader.docFreq(term) == 0;
     }
     
@@ -117,69 +122,76 @@ public class TermQuery extends Query {
           float freq = scorer.freq();
           SimScorer docScorer = similarity.simScorer(stats, context);
           ComplexExplanation result = new ComplexExplanation();
-          result.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");
-          Explanation scoreExplanation = docScorer.explain(doc, new Explanation(freq, "termFreq=" + freq));
+          result.setDescription("weight(" + getQuery() + " in " + doc + ") ["
+              + similarity.getClass().getSimpleName() + "], result of:");
+          Explanation scoreExplanation = docScorer.explain(doc,
+              new Explanation(freq, "termFreq=" + freq));
           result.addDetail(scoreExplanation);
           result.setValue(scoreExplanation.getValue());
           result.setMatch(true);
           return result;
         }
       }
-      return new ComplexExplanation(false, 0.0f, "no matching term");      
+      return new ComplexExplanation(false, 0.0f, "no matching term");
     }
   }
-
+  
   /** Constructs a query for the term <code>t</code>. */
   public TermQuery(Term t) {
     this(t, -1);
   }
-
-  /** Expert: constructs a TermQuery that will use the
-   *  provided docFreq instead of looking up the docFreq
-   *  against the searcher. */
+  
+  /**
+   * Expert: constructs a TermQuery that will use the provided docFreq instead
+   * of looking up the docFreq against the searcher.
+   */
   public TermQuery(Term t, int docFreq) {
     term = t;
     this.docFreq = docFreq;
     perReaderTermState = null;
   }
   
-  /** Expert: constructs a TermQuery that will use the
-   *  provided docFreq instead of looking up the docFreq
-   *  against the searcher. */
+  /**
+   * Expert: constructs a TermQuery that will use the provided docFreq instead
+   * of looking up the docFreq against the searcher.
+   */
   public TermQuery(Term t, TermContext states) {
     assert states != null;
     term = t;
     docFreq = states.docFreq();
     perReaderTermState = states;
   }
-
+  
   /** Returns the term of this query. */
-  public Term getTerm() { return term; }
-
+  public Term getTerm() {
+    return term;
+  }
+  
   @Override
   public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
     final IndexReaderContext context = searcher.getTopReaderContext();
     final TermContext termState;
-    if (perReaderTermState == null || perReaderTermState.topReaderContext != context) {
-      // make TermQuery single-pass if we don't have a PRTS or if the context differs!
+    if (perReaderTermState == null
+        || perReaderTermState.topReaderContext != context) {
+      // make TermQuery single-pass if we don't have a PRTS or if the context
+      // differs!
       termState = TermContext.build(context, term);
     } else {
-     // PRTS was pre-build for this IS
-     termState = this.perReaderTermState;
+      // PRTS was pre-build for this IS
+      termState = this.perReaderTermState;
     }
-
+    
     // we must not ignore the given docFreq - if set use the given value (lie)
-    if (docFreq != -1)
-      termState.setDocFreq(docFreq);
+    if (docFreq != -1) termState.setDocFreq(docFreq);
     
     return new TermWeight(searcher, needsScores, termState);
   }
-
+  
   @Override
   public void extractTerms(Set<Term> terms) {
     terms.add(getTerm());
   }
-
+  
   /** Prints a user-readable version of this query. */
   @Override
   public String toString(String field) {
@@ -192,21 +204,20 @@ public class TermQuery extends Query {
     buffer.append(ToStringUtils.boost(getBoost()));
     return buffer.toString();
   }
-
+  
   /** Returns true iff <code>o</code> is equal to this. */
   @Override
   public boolean equals(Object o) {
-    if (!(o instanceof TermQuery))
-      return false;
-    TermQuery other = (TermQuery)o;
+    if (!(o instanceof TermQuery)) return false;
+    TermQuery other = (TermQuery) o;
     return (this.getBoost() == other.getBoost())
-      && this.term.equals(other.term);
+        && this.term.equals(other.term);
   }
-
-  /** Returns a hash code value for this object.*/
+  
+  /** Returns a hash code value for this object. */
   @Override
   public int hashCode() {
     return Float.floatToIntBits(getBoost()) ^ term.hashCode();
   }
-
+  
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Fri Feb  6 16:39:45 2015
@@ -19,78 +19,99 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
 
 /** Expert: A <code>Scorer</code> for documents matching a <code>Term</code>.
  */
 final class TermScorer extends Scorer {
-  private final DocsEnum docsEnum;
+  private final PostingsEnum postingsEnum;
   private final Similarity.SimScorer docScorer;
-  
+
   /**
    * Construct a <code>TermScorer</code>.
-   * 
+   *
    * @param weight
    *          The weight of the <code>Term</code> in the query.
    * @param td
    *          An iterator over the documents matching the <code>Term</code>.
    * @param docScorer
-   *          The </code>Similarity.SimScorer</code> implementation 
+   *          The </code>Similarity.SimScorer</code> implementation
    *          to be used for score computations.
    */
-  TermScorer(Weight weight, DocsEnum td, Similarity.SimScorer docScorer) {
+  TermScorer(Weight weight, PostingsEnum td, Similarity.SimScorer docScorer) {
     super(weight);
     this.docScorer = docScorer;
-    this.docsEnum = td;
+    this.postingsEnum = td;
   }
 
   @Override
   public int docID() {
-    return docsEnum.docID();
+    return postingsEnum.docID();
   }
 
   @Override
   public int freq() throws IOException {
-    return docsEnum.freq();
+    return postingsEnum.freq();
+  }
+
+  @Override
+  public int nextPosition() throws IOException {
+    return postingsEnum.nextPosition();
+  }
+
+  @Override
+  public int startOffset() throws IOException {
+    return postingsEnum.startOffset();
+  }
+
+  @Override
+  public int endOffset() throws IOException {
+    return postingsEnum.endOffset();
+  }
+
+  @Override
+  public BytesRef getPayload() throws IOException {
+    return postingsEnum.getPayload();
   }
 
   /**
    * Advances to the next document matching the query. <br>
-   * 
+   *
    * @return the document matching the query or NO_MORE_DOCS if there are no more documents.
    */
   @Override
   public int nextDoc() throws IOException {
-    return docsEnum.nextDoc();
+    return postingsEnum.nextDoc();
   }
-  
+
   @Override
   public float score() throws IOException {
     assert docID() != NO_MORE_DOCS;
-    return docScorer.score(docsEnum.docID(), docsEnum.freq());  
+    return docScorer.score(postingsEnum.docID(), postingsEnum.freq());
   }
 
   /**
    * Advances to the first match beyond the current whose document number is
    * greater than or equal to a given target. <br>
-   * The implementation uses {@link DocsEnum#advance(int)}.
-   * 
+   * The implementation uses {@link org.apache.lucene.index.PostingsEnum#advance(int)}.
+   *
    * @param target
    *          The target document number.
    * @return the matching document or NO_MORE_DOCS if none exist.
    */
   @Override
   public int advance(int target) throws IOException {
-    return docsEnum.advance(target);
+    return postingsEnum.advance(target);
   }
-  
+
   @Override
   public long cost() {
-    return docsEnum.cost();
+    return postingsEnum.cost();
   }
 
   /** Returns a string representation of this <code>TermScorer</code>. */
   @Override
-  public String toString() { return "scorer(" + weight + ")"; }
+  public String toString() { return "scorer(" + weight + ")[" + super.toString() + "]"; }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TimeLimitingCollector.java Fri Feb  6 16:39:45 2015
@@ -17,12 +17,12 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.ThreadInterruptedException;
 
-import java.io.IOException;
-
 /**
  * The {@link TimeLimitingCollector} is used to timeout search requests that
  * take longer than the maximum allowed search time limit. After this time is
@@ -156,7 +156,7 @@ public class TimeLimitingCollector imple
       
     };
   }
-  
+
   @Override
   public boolean needsScores() {
     return collector.needsScores();

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/TotalHitCountCollector.java Fri Feb  6 16:39:45 2015
@@ -17,7 +17,6 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-
 /**
  * Just counts the total number of hits.
  */

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/Weight.java Fri Feb  6 16:39:45 2015
@@ -19,8 +19,8 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
+import org.apache.lucene.index.IndexReaderContext;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.IndexReaderContext; // javadocs
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Bits;
 
@@ -194,4 +194,5 @@ public abstract class Weight {
       }
     }
   }
+
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Fri Feb  6 16:39:45 2015
@@ -17,11 +17,15 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
@@ -37,10 +41,6 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
-import java.io.IOException;
-import java.util.Collection;
-import java.util.Iterator;
-
 /**
  * This class is very similar to
  * {@link org.apache.lucene.search.spans.SpanNearQuery} except that it factors

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Fri Feb  6 16:39:45 2015
@@ -17,27 +17,27 @@ package org.apache.lucene.search.payload
  * limitations under the License.
  */
 
+import java.io.IOException;
+
 import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.search.ComplexExplanation;
+import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.spans.SpanQuery;
-import org.apache.lucene.search.spans.TermSpans;
+import org.apache.lucene.search.spans.SpanScorer;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.SpanWeight;
-import org.apache.lucene.search.spans.SpanScorer;
+import org.apache.lucene.search.spans.TermSpans;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
-import java.io.IOException;
-
 /**
  * This class is very similar to
  * {@link org.apache.lucene.search.spans.SpanTermQuery} except that it factors
@@ -120,7 +120,7 @@ public class PayloadTermQuery extends Sp
 
       protected void processPayload(Similarity similarity) throws IOException {
         if (termSpans.isPayloadAvailable()) {
-          final DocsAndPositionsEnum postings = termSpans.getPostings();
+          final PostingsEnum postings = termSpans.getPostings();
           payload = postings.getPayload();
           if (payload != null) {
             payloadScore = function.currentScore(doc, term.field(),

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java Fri Feb  6 16:39:45 2015
@@ -17,10 +17,8 @@ package org.apache.lucene.search.similar
  * limitations under the License.
  */
 
-import java.io.IOException;
-
-import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.FieldInvertState;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.Explanation;
@@ -28,9 +26,11 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.TermStatistics;
-import org.apache.lucene.search.spans.SpanQuery; // javadoc
+import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SmallFloat; // javadoc
+import org.apache.lucene.util.SmallFloat;
+
+import java.io.IOException;
 
 /** 
  * Similarity defines the components of Lucene scoring.

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java Fri Feb  6 16:39:45 2015
@@ -19,9 +19,10 @@ package org.apache.lucene.search.spans;
 
 import java.io.IOException;
 
-import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.similarities.Similarity;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * Public for extension only.
@@ -96,16 +97,37 @@ public class SpanScorer extends Scorer {
   public int freq() throws IOException {
     return numMatches;
   }
-  
+
+  @Override
+  public int nextPosition() throws IOException {
+    throw new UnsupportedOperationException("SpanQueries do not support nextPosition() iteration");
+  }
+
+  @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();
+  }
+
   /** Returns the intermediate "sloppy freq" adjusted for edit distance 
    *  @lucene.internal */
   // only public so .payloads can see it.
   public float sloppyFreq() throws IOException {
     return freq;
   }
-  
+
   @Override
   public long cost() {
     return spans.cost();
   }
+
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanTermQuery.java Fri Feb  6 16:39:45 2015
@@ -17,10 +17,13 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
@@ -28,10 +31,6 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
 
-import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
-
 /** Matches spans containing a term. */
 public class SpanTermQuery extends SpanQuery {
   protected Term term;
@@ -115,7 +114,7 @@ public class SpanTermQuery extends SpanQ
     final TermsEnum termsEnum = context.reader().terms(term.field()).iterator(null);
     termsEnum.seekExact(term.bytes(), state);
     
-    final DocsAndPositionsEnum postings = termsEnum.docsAndPositions(acceptDocs, null, DocsAndPositionsEnum.FLAG_PAYLOADS);
+    final PostingsEnum postings = termsEnum.postings(acceptDocs, null, PostingsEnum.FLAG_PAYLOADS);
 
     if (postings != null) {
       return new TermSpans(postings, term);

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Fri Feb  6 16:39:45 2015
@@ -17,20 +17,26 @@ package org.apache.lucene.search.spans;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.LeafReaderContext;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.TreeSet;
+
 import org.apache.lucene.index.IndexReaderContext;
+import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
-import org.apache.lucene.search.*;
+import org.apache.lucene.search.ComplexExplanation;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermStatistics;
+import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.Bits;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.TreeSet;
-
 /**
  * Expert-only.  Public for use by other weight implementations
  */

Modified: lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/java/org/apache/lucene/search/spans/TermSpans.java Fri Feb  6 16:39:45 2015
@@ -17,7 +17,7 @@ package org.apache.lucene.search.spans;
 
 
 import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.util.BytesRef;
 
@@ -30,7 +30,7 @@ import java.util.Collection;
  * Public for extension only
  */
 public class TermSpans extends Spans {
-  protected final DocsAndPositionsEnum postings;
+  protected final PostingsEnum postings;
   protected final Term term;
   protected int doc;
   protected int freq;
@@ -38,7 +38,7 @@ public class TermSpans extends Spans {
   protected int position;
   protected boolean readPayload;
 
-  public TermSpans(DocsAndPositionsEnum postings, Term term) {
+  public TermSpans(PostingsEnum postings, Term term) {
     this.postings = postings;
     this.term = term;
     doc = -1;
@@ -132,7 +132,7 @@ public class TermSpans extends Spans {
             (doc == -1 ? "START" : (doc == Integer.MAX_VALUE) ? "END" : doc + "-" + position);
   }
 
-  public DocsAndPositionsEnum getPostings() {
+  public PostingsEnum getPostings() {
     return postings;
   }
 

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java Fri Feb  6 16:39:45 2015
@@ -22,11 +22,27 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.Random;
 
-import org.apache.lucene.store.*;
-import org.apache.lucene.document.*;
-import org.apache.lucene.analysis.*;
-import org.apache.lucene.index.*;
-import org.apache.lucene.search.*;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.IntField;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.MergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 
 public class TestSearchForDuplicates extends LuceneTestCase {

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestCachingTokenFilter.java Fri Feb  6 16:39:45 2015
@@ -25,9 +25,9 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
@@ -84,7 +84,7 @@ public class TestCachingTokenFilter exte
     writer.addDocument(doc);
     
     IndexReader reader = writer.getReader();
-    DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader,
+    PostingsEnum termPositions = MultiFields.getTermPositionsEnum(reader,
                                                                           MultiFields.getLiveDocs(reader),
                                                                           "preanalyzed",
                                                                           new BytesRef("term1"));

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/analysis/TestMockAnalyzer.java Fri Feb  6 16:39:45 2015
@@ -25,7 +25,7 @@ import java.util.Random;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.LeafReader;
@@ -321,7 +321,7 @@ public class TestMockAnalyzer extends Ba
     final Terms terms = fields.terms("f");
     final TermsEnum te = terms.iterator(null);
     assertEquals(new BytesRef("a"), te.next());
-    final DocsAndPositionsEnum dpe = te.docsAndPositions(null, null);
+    final PostingsEnum dpe = te.postings(null, null, PostingsEnum.FLAG_ALL);
     assertEquals(0, dpe.nextDoc());
     assertEquals(2, dpe.freq());
     assertEquals(0, dpe.nextPosition());

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingDocValuesFormat.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingDocValuesFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingDocValuesFormat.java Fri Feb  6 16:39:45 2015
@@ -27,5 +27,10 @@ public class TestAssertingDocValuesForma
   @Override
   protected Codec getCodec() {
     return codec;
-  } 
+  }
+
+  @Override
+  protected boolean isPostingsEnumReuseImplemented() {
+    return false;
+  }
 }

Modified: lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingPostingsFormat.java?rev=1657890&r1=1657889&r2=1657890&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingPostingsFormat.java (original)
+++ lucene/dev/branches/branch_5x/lucene/core/src/test/org/apache/lucene/codecs/asserting/TestAssertingPostingsFormat.java Fri Feb  6 16:39:45 2015
@@ -27,5 +27,10 @@ public class TestAssertingPostingsFormat
   @Override
   protected Codec getCodec() {
     return codec;
-  } 
+  }
+
+  @Override
+  protected boolean isPostingsEnumReuseImplemented() {
+    return false;
+  }
 }