Index: lucene/src/java/org/apache/lucene/index/BulkPostingsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/BulkPostingsEnum.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/index/BulkPostingsEnum.java	(working copy)
@@ -19,6 +19,8 @@
 
 import java.io.IOException;
 
+import org.apache.lucene.util.BytesRef;
+
 /** Low level bulk iterator through postings (documents,
  *  term freq, positions).  This API shifts much
  *  responsibility to the caller, in order to maximize
@@ -79,6 +81,25 @@
 
     // nocommit messy
     public abstract void setOffset(int offset);
+    
+    public BulkPayload getPayloadBuffer() {
+      return null;
+    }
+    public boolean payloadsAvailable() {
+      return false;
+    }
+  }
+  
+  public static final class BulkPayload {
+    public final BytesRef bytes;
+    public final int[] start;
+    public final int[] length;
+    
+    public BulkPayload(BytesRef bytes, int[] start, int[] length) {
+      this.bytes = bytes;
+      this.start = start;
+      this.length = length;
+    }
   }
 
   public abstract BlockReader getDocDeltasReader() throws IOException;
Index: lucene/src/java/org/apache/lucene/index/MultiBulkPostingsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/MultiBulkPostingsEnum.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/index/MultiBulkPostingsEnum.java	(working copy)
@@ -147,7 +147,7 @@
         int limit = current.fill();
         //int offset = current.offset();
         return doCopy(0, limit);
-      }
+       }
     }
 
     protected abstract BlockReader getBlockReader(int upto) throws IOException;
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(working copy)
@@ -23,9 +23,7 @@
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
-import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.BulkPostingsEnum;
-import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
@@ -1044,6 +1042,7 @@
       final int[] buffer = new int[BULK_BUFFER_SIZE];
       int limit;
       int pending;
+      BulkPayload payloads;
   
       @Override
       public int[] getBuffer() {
@@ -1059,15 +1058,25 @@
       public int fill() throws IOException {
         limit = pending > BULK_BUFFER_SIZE ? BULK_BUFFER_SIZE : pending;
         if (storePayloads) {
+          int payloadOffset = 0;
           for(int i=0;i<limit;i++) {
             final int code = proxIn.readVInt();
             buffer[i] = code >>> 1;
-            if ((code & 1) != 0) {
+            if ((code & 1) != 0) { // TODO maybe optimize this and preallocate space for payloads if payloadLength doesn't change regularly
               payloadLength = proxIn.readVInt();
             }
             if (payloadLength != 0) {
-              // skip payload
-              proxIn.seek(proxIn.getFilePointer()+payloadLength);
+
+              if (payloads != null) {
+                payloads.start[i] = payloadOffset;
+                payloads.length[i] = payloadLength;
+                payloads.bytes.grow(payloadOffset + payloadLength);
+                proxIn.readBytes(payloads.bytes.bytes, payloadOffset, payloadLength);
+                payloadOffset += payloadLength;
+              } else {
+                // skip payload
+                proxIn.seek(proxIn.getFilePointer()+payloadLength);
+              }
             }
           }
         } else {
@@ -1088,6 +1097,19 @@
       public void setOffset(int offset) {
         throw new UnsupportedOperationException();
       }
+
+      @Override
+      public BulkPayload getPayloadBuffer() {
+        if (storePayloads && payloads == null) { // only create if needed
+          this.payloads = new BulkPayload(new BytesRef(), new int[BULK_BUFFER_SIZE], new int[BULK_BUFFER_SIZE]);
+        }
+        return this.payloads;
+      }
+
+      @Override
+      public boolean payloadsAvailable() {
+        return storePayloads;
+      }
     }
     
     @Override
Index: lucene/src/java/org/apache/lucene/search/BooleanQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/BooleanQuery.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/BooleanQuery.java	(working copy)
@@ -299,7 +299,7 @@
       Iterator<BooleanClause> cIter = clauses.iterator();
       for (Weight w  : weights) {
         BooleanClause c =  cIter.next();
-        Scorer subScorer = w.scorer(context, ScorerContext.def());
+        Scorer subScorer = w.scorer(context, scorerContext.topScorer(false));
         if (subScorer == null) {
           if (c.isRequired()) {
             return null;
Index: lucene/src/java/org/apache/lucene/search/BooleanScorer.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/BooleanScorer.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/BooleanScorer.java	(working copy)
@@ -380,4 +380,9 @@
     }
   }
 
+  @Override
+  public PositionIntervalIterator positions() throws IOException {
+    return super.positions();
+  }
+
 }
Index: lucene/src/java/org/apache/lucene/search/BooleanScorer2.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/BooleanScorer2.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/BooleanScorer2.java	(working copy)
@@ -316,6 +316,13 @@
   public int advance(int target) throws IOException {
     return doc = countingSumScorer.advance(target);
   }
+  
+  
+
+  @Override
+  public PositionIntervalIterator positions() throws IOException {
+    return countingSumScorer.positions();
+  }
 
   @Override
   protected void visitSubScorers(Query parent, Occur relationship, ScorerVisitor<Query, Query, Scorer> visitor) {
Index: lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/ConjunctionScorer.java	(working copy)
@@ -17,7 +17,10 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.search.PositionIntervalIterator.PositionInterval;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.PriorityQueue;
+
 import java.io.IOException;
 import java.util.Collection;
 import java.util.Comparator;
@@ -32,7 +35,7 @@
   public ConjunctionScorer(Weight weight, float coord, Collection<Scorer> scorers) throws IOException {
     this(weight, coord, scorers.toArray(new Scorer[scorers.size()]));
   }
-
+  
   public ConjunctionScorer(Weight weight, float coord, Scorer... scorers) throws IOException {
     super(weight);
     this.scorers = scorers;
@@ -136,4 +139,136 @@
     }
     return sum * coord;
   }
+
+  @Override
+  public PositionIntervalIterator positions() throws IOException {
+    // only created if needed for this scorer - no penalty for non-positional queries
+    return new ConjunctionPositionIterator(scorers);
+  }
+
+  /**
+   * ConjuctionPositionIterator based on minimal interval semantics for AND
+   * operator
+   * 
+   * <a href=
+   * "http://vigna.dsi.unimi.it/ftp/papers/EfficientAlgorithmsMinimalIntervalSemantics"
+   * >"Efficient Optimally Lazy Algorithms for Minimal-Interval Semantic</a>
+   * 
+   */
+  // TODO if we respect the inOrder boolean on subs we can make a ordered span filter on top of this!
+  public class ConjunctionPositionIterator extends PositionIntervalIterator {
+    private int docId = -1;
+    final PositionIntervalIterator[] iterators;
+    final IntervalQueue queue;
+
+    public ConjunctionPositionIterator(Scorer[] scorers) throws IOException {
+      iterators = new PositionIntervalIterator[scorers.length];
+      queue = new IntervalQueue(scorers.length);
+      for (int i = 0; i < scorers.length; i++) {
+        iterators[i] = scorers[i].positions();
+      }
+    }
+
+    private void advance() throws IOException {
+      IntervalRef top = queue.top();
+      PositionInterval interval = null;
+      if ((interval = iterators[top.index].next()) != null) {
+        top.interval = interval;
+        queue.updateTop();
+      } else {
+        queue.pop();
+      }
+    }
+
+    @Override
+    public PositionInterval next() throws IOException {
+      if (docId != lastDoc) {
+        docId = lastDoc;
+        queue.reset();
+        for (int i = 0; i < iterators.length; i++) {
+          queue.add(new IntervalRef(iterators[i].next(), i));
+        }
+      }
+      if (queue.size() != iterators.length) {
+        return null;
+      }
+      while (containsLastInterval()) {
+        advance();
+        if (queue.size() != iterators.length)
+          return null;
+      }
+      do {
+        PositionInterval interval = queue.top().interval;
+        queue.lastInterval.begin = interval.begin;
+        queue.lastInterval.end = queue.rightExtreme;
+        advance();
+        if (queue.size() != iterators.length)
+          break;
+      } while (containsLastInterval());
+      return queue.lastInterval; // TODO support payloads
+    }
+
+    private boolean containsLastInterval() {
+      PositionInterval interval = queue.top().interval;
+      return interval.begin <= queue.lastInterval.begin
+          && queue.lastInterval.end <= queue.rightExtreme;
+    }
+
+    @Override
+    public int docID() {
+      return lastDoc;
+    }
+
+    @Override
+    public Scorer getScorer() {
+      return ConjunctionScorer.this;
+    }
+
+    @Override
+    public PositionIntervalIterator[] subs(boolean inOrder) {
+      return iterators;
+    }
+
+  }
+
+  static class IntervalRef {
+    PositionInterval interval;
+    int index;
+
+    IntervalRef(PositionInterval interval, int index) {
+      super();
+      this.interval = interval;
+      this.index = index;
+    }
+  }
+
+  static final class IntervalQueue extends PriorityQueue<IntervalRef> {
+    int rightExtreme = Integer.MIN_VALUE;
+    final PositionInterval lastInterval = new PositionInterval(
+        Integer.MIN_VALUE, Integer.MAX_VALUE);
+
+    public void reset() {
+      clear();
+      rightExtreme = Integer.MIN_VALUE;
+      lastInterval.begin = Integer.MIN_VALUE;
+      lastInterval.end = Integer.MAX_VALUE;
+
+    }
+
+    public IntervalQueue(int size) {
+      this.initialize(size);
+    }
+
+    @Override
+    protected boolean lessThan(IntervalRef left, IntervalRef right) {
+      final PositionInterval a = left.interval;
+      final PositionInterval b = right.interval;
+      /*
+       *TODO we don't need to update rightExtreme here all the time  - if we override add & updateTop we can do it there 
+       */
+      rightExtreme = Math.max(rightExtreme, Math.max(a.end, b.end)); 
+      return a.begin < b.begin || (a.begin == b.begin && a.end >= b.end);
+    }
+  }
+
 }
Index: lucene/src/java/org/apache/lucene/search/PositionFilterQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/PositionFilterQuery.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/PositionFilterQuery.java	(revision 0)
@@ -0,0 +1,170 @@
+package org.apache.lucene.search;
+
+/**
+ * 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.Set;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.PositionIntervalIterator.PositionIntervalFilter;
+
+/**
+ *
+ *
+ **/
+public class PositionFilterQuery extends Query implements Cloneable {
+
+  private Query inner;
+  private PositionIntervalFilter filter;
+
+  public PositionFilterQuery(Query inner, PositionIntervalFilter filter) {
+    this.inner = inner;
+    this.filter = filter;
+  }
+
+  @Override
+  public void extractTerms(Set<Term> terms) {
+    inner.extractTerms(terms);
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    PositionFilterQuery clone = null;
+
+    Query rewritten = (Query) inner.rewrite(reader);
+    if (rewritten != inner) {
+      clone = (PositionFilterQuery) this.clone();
+      clone.inner = rewritten;
+    }
+
+    if (clone != null) {
+      return clone; // some clauses rewrote
+    } else {
+      return this; // no clauses rewrote
+    }
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
+    return new PositionFilterWeight(inner.createWeight(searcher));
+  }
+
+  class PositionFilterWeight extends Weight {
+
+    private final Weight other;
+
+    public PositionFilterWeight(Weight other) {
+      this.other = other;
+    }
+
+    @Override
+    public Explanation explain(AtomicReaderContext context, int doc)
+        throws IOException {
+      return other.explain(context, doc);
+    }
+
+    @Override
+    public Query getQuery() {
+      return PositionFilterQuery.this;
+    }
+
+    @Override
+    public float getValue() {
+      return other.getValue();
+    }
+
+    @Override
+    public void normalize(float norm) {
+      other.normalize(norm);
+    }
+
+    @Override
+    public Scorer scorer(AtomicReaderContext context,
+        ScorerContext scorerContext) throws IOException {
+      Scorer scorer = other.scorer(context,
+          scorerContext.needsPositions(true));
+      return scorer == null ? null : new PositionFilterScorer(this, scorer);
+    }
+
+    @Override
+    public float sumOfSquaredWeights() throws IOException {
+      return other.sumOfSquaredWeights();
+    }
+
+  }
+
+  class PositionFilterScorer extends Scorer {
+
+    private final Scorer other;
+    private PositionIntervalIterator filter;
+
+    public PositionFilterScorer(Weight weight, Scorer other) throws IOException {
+      super(weight);
+      this.other = other;
+      this.filter = PositionFilterQuery.this.filter != null ? PositionFilterQuery.this.filter.filter(other.positions())
+          : other.positions();
+    }
+
+    @Override
+    public float score() throws IOException {
+      return other.score();
+    }
+
+    @Override
+    public PositionIntervalIterator positions() throws IOException {
+      return filter;
+    }
+
+    @Override
+    public int docID() {
+      return other.docID();
+    }
+
+    @Override
+    public int nextDoc() throws IOException {
+      while (other.nextDoc() != Scorer.NO_MORE_DOCS) {
+        if (filter.next() != null) {
+          return other.docID();
+        }
+      }
+      return Scorer.NO_MORE_DOCS;
+    }
+
+    @Override
+    public int advance(int target) throws IOException {
+      int advance = other.advance(target);
+      if (advance == Scorer.NO_MORE_DOCS)
+        return NO_MORE_DOCS;
+      do {
+        if (filter.next() != null) {
+          return other.docID();
+        }
+      } while (other.nextDoc() != Scorer.NO_MORE_DOCS);
+      return NO_MORE_DOCS;
+    }
+
+  }
+
+  @Override
+  public String toString(String field) {
+    return inner.toString();
+  }
+
+}
\ No newline at end of file
Index: lucene/src/java/org/apache/lucene/search/PositionIntervalIterator.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/PositionIntervalIterator.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/PositionIntervalIterator.java	(revision 0)
@@ -0,0 +1,86 @@
+package org.apache.lucene.search;
+
+/**
+ * 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.util.BytesRef;
+
+/**
+ * 
+ * TODO add documentation
+ */
+public abstract class PositionIntervalIterator {
+
+  public static final PositionIntervalIterator[] EMPTY = new PositionIntervalIterator[0];
+
+  public abstract PositionInterval next() throws IOException;
+
+  public abstract PositionIntervalIterator[] subs(boolean inOrder);
+
+  public abstract int docID();
+
+  public abstract Scorer getScorer();
+
+  public static interface PositionIntervalFilter {
+    public abstract PositionIntervalIterator filter(
+        PositionIntervalIterator iter);
+  }
+
+  public static class PositionInterval implements Cloneable {
+    
+
+    public int begin;
+    public int end;
+
+    public PositionInterval(int begin, int end) {
+      this.begin = begin;
+      this.end = end;
+    }
+
+    public PositionInterval() {
+      this(0, 0);
+    }
+
+    public boolean nextPayload(BytesRef ref) throws IOException {
+      return false;
+    }
+
+    public boolean payloadAvailable() {
+      return false;
+    }
+
+    public void reset() {
+      begin = end = 0;
+    }
+    
+    @Override
+    public Object clone() {
+      try {
+        return super.clone();
+      } catch (CloneNotSupportedException e) {
+        throw new RuntimeException(); // should not happen
+      }
+    }
+    
+    @Override
+    public String toString() {
+      return "PositionInterval [begin=" + begin + ", end=" + end + "]";
+    }
+
+  }
+}
Index: lucene/src/java/org/apache/lucene/search/RangePositionsIterator.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/RangePositionsIterator.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/RangePositionsIterator.java	(revision 0)
@@ -0,0 +1,77 @@
+package org.apache.lucene.search;
+/**
+ * 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.PositionIntervalIterator;
+import org.apache.lucene.search.PositionIntervalIterator.PositionIntervalFilter;
+
+
+public class RangePositionsIterator extends PositionIntervalIterator implements PositionIntervalFilter {
+
+  private final PositionIntervalIterator iterator;
+  private int start;
+  private int end;
+  
+  public RangePositionsIterator(int start, int end) {
+    this(start, end, null);
+  }
+  
+  public RangePositionsIterator(int start, int end, PositionIntervalIterator iterator) {
+    this.iterator = iterator;
+    this.start = start;
+    this.end = end;
+  }
+
+  public PositionIntervalIterator filter(PositionIntervalIterator iter) {
+    return new RangePositionsIterator(start, end, iter);
+  }  
+  
+  @Override
+  public PositionInterval next() throws IOException {
+    PositionInterval interval = null;
+    while ((interval = iterator.next()) != null) {
+      if(interval.end > end) {
+        return null;
+      } else if (interval.begin >= start) {
+        return interval;
+      }
+    }
+    return null;
+  }
+
+
+  @Override
+  public int docID() {
+    return iterator.docID();
+  }
+
+  @Override
+  public Scorer getScorer() {
+    return iterator.getScorer();
+  }
+
+
+  @Override
+  public PositionIntervalIterator[] subs(boolean inOrder) {
+    return new PositionIntervalIterator[] { iterator };
+  }
+  
+  
+
+}
\ No newline at end of file
Index: lucene/src/java/org/apache/lucene/search/Scorer.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/Scorer.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/Scorer.java	(working copy)
@@ -101,6 +101,11 @@
   public float freq() throws IOException {
     throw new UnsupportedOperationException(this + " does not implement freq()");
   }
+  
+  public PositionIntervalIterator positions() throws IOException {
+    // TODO make abstract?
+    throw new UnsupportedOperationException();
+  }
 
   /**
    * A callback to gather information from a scorer and its sub-scorers. Each
Index: lucene/src/java/org/apache/lucene/search/TermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TermQuery.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/TermQuery.java	(working copy)
@@ -107,7 +107,7 @@
       final Terms terms = fields.terms(field);
       final TermsEnum termsEnum = terms.getThreadTermsEnum();
       termsEnum.seek(term.bytes(), state); // ignore return value -- asserted above!
-      final BulkPostingsEnum docs = termsEnum.bulkPostings(null, true, false);
+      final BulkPostingsEnum docs = termsEnum.bulkPostings(null, true, scorerContext.needsPositions);
       
       final int docFreq = termsEnum.docFreq();
       final BlockReader docDeltas = docs.getDocDeltasReader();
@@ -116,8 +116,16 @@
         return new MatchOnlyTermScorer(this, docs, docDeltas, docFreq,
             reader.getDeletedDocs(), similarity, reader.norms(term.field()));
       } else {
-        return new TermScorer(this, docs, docDeltas, frequencies, docFreq,
-            reader.getDeletedDocs(), similarity, reader.norms(term.field()));
+        if (scorerContext.needsPositions) {
+          BlockReader positionDeltasReader = docs.getPositionDeltasReader();
+          assert positionDeltasReader != null : docs;
+          return new TermScorer(this, docs, docDeltas,
+              frequencies, positionDeltasReader, scorerContext.needsPayloads, docFreq,
+              reader.getDeletedDocs(), similarity, reader.norms(term.field()));
+        } else {
+          return new TermScorer(this, docs, docDeltas, frequencies, docFreq,
+              reader.getDeletedDocs(), similarity, reader.norms(term.field()));
+        }
       }
     }
     
Index: lucene/src/java/org/apache/lucene/search/TermScorer.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TermScorer.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/TermScorer.java	(working copy)
@@ -21,7 +21,10 @@
 
 import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.BulkPostingsEnum.BlockReader;
+import org.apache.lucene.index.BulkPostingsEnum.BulkPayload;
+import org.apache.lucene.search.PositionIntervalIterator.PositionInterval;
 import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
 
 // nocommit -- break out aligned & not cases?
 // nocommit -- break out bulk vs doc-at-time scorer?
@@ -51,6 +54,12 @@
   private final int docFreq;
   private int count;
   private final Similarity similarity;
+  private final TermPositions positions;
+
+  
+  TermScorer(Weight weight, BulkPostingsEnum td, BlockReader docDeltaReader, BlockReader freqReader, int docFreq, Bits skipDocs, Similarity similarity, byte[] norms) throws IOException {
+    this(weight, td, docDeltaReader, freqReader, null, false, docFreq, skipDocs, similarity, norms);
+  }
   
   /**
    * Construct a <code>TermScorer</code>.
@@ -65,7 +74,8 @@
    * @param norms
    *          The field norms of the document fields for the <code>Term</code>.
    */
-  TermScorer(Weight weight, BulkPostingsEnum td, BlockReader docDeltaReader, BlockReader freqReader, int docFreq, Bits skipDocs, Similarity similarity, byte[] norms) throws IOException {
+  
+  TermScorer(Weight weight, BulkPostingsEnum td, BlockReader docDeltaReader, BlockReader freqReader, BlockReader posDeltaReader, boolean needsPayloads, int docFreq, Bits skipDocs, Similarity similarity, byte[] norms) throws IOException {
     super(weight);
     this.similarity = similarity;
     this.docsEnum = td;
@@ -74,6 +84,8 @@
     docDeltas = docDeltasReader.getBuffer();
     this.freqsReader = freqReader;
     freqs = freqsReader.getBuffer();
+    positions = posDeltaReader == null ? null :new TermPositions(posDeltaReader, needsPayloads && posDeltaReader.payloadsAvailable());   
+
     reset();
     this.skipDocs = skipDocs;
     this.norms = norms;
@@ -253,6 +265,9 @@
       assert freqPointerMax != 0;
       freqPointer = 0;
     }
+    if (positions != null) { // compiler should optimize this away 
+      positions.fill();
+    }
   }
   
   private void fillDeltas() throws IOException {
@@ -272,5 +287,124 @@
     assert freqPointerMax >= freqPointer: "fP=" + freqPointer + " fPMax=" + freqPointerMax;
     --docPointer;
     --freqPointer;
+    if (positions != null) { // compiler should optimize this away 
+      positions.reset();
+    }
+  }
+  
+  @Override
+  public PositionIntervalIterator positions() throws IOException {
+    return positions;
+  }
+  
+  private final class TermPositions extends PositionIntervalIterator {
+    private final PositionInterval interval;
+    private int positionsPending;
+    private int posPointerMax;
+    protected int posPointer;
+    private final int[] posBuffer;
+    private final BlockReader posReader;
+    public TermPositions(BlockReader blockReader, boolean doPayloads) {
+      posReader = blockReader;
+      posBuffer = blockReader.getBuffer();
+      this.interval = doPayloads ? new PayloadPosInterval(blockReader.getPayloadBuffer(), this) : new PositionInterval();
+      
+    }
+    
+    void fill() throws IOException {
+      if (positionsPending > 0) {
+        posPointer += positionsPending;
+        while (posPointer >= posPointerMax) { // we need while here if numPos > buffersize
+          posPointer -= posPointerMax; // add the pending positions from last round
+          posPointerMax = posReader.fill();
+          assert posPointerMax != 0;
+        }
+      } else if (posPointer + 1 >= posPointerMax) {
+        posPointerMax = posReader.fill();
+        assert posPointerMax != 0;
+        posPointer = -1;
+      }
+      interval.reset();
+      positionsPending =  freqs[freqPointer];
+    }
+    
+    void reset() {
+      interval.reset();
+      posPointer = posReader.offset();
+      posPointerMax = posReader.end();
+      assert posPointerMax >= posPointer: "pP=" + posPointer + " pPMax=" + posPointerMax;
+      --posPointer;
+      positionsPending = 0;
+    }
+
+    @Override
+    public PositionInterval next() throws IOException {
+      if (--positionsPending >= 0) {
+        if(++posPointer >= posPointerMax) {
+          posPointerMax = posReader.fill();
+          assert posPointerMax != 0;
+          posPointer = 0;
+        }
+        interval.begin += posBuffer[posPointer];
+        interval.end = interval.begin+1;
+        return interval;
+      }
+      interval.reset();
+      positionsPending = 0;
+      return null;
+    }
+
+    @Override
+    public int docID() {
+      return doc;
+    }
+
+    @Override
+    public Scorer getScorer() {
+      return TermScorer.this;
+    }
+
+    @Override
+    public PositionIntervalIterator[] subs(boolean inOrder) {
+      return EMPTY;
+    }
+  }
+  
+  private static final class PayloadPosInterval extends PositionInterval {
+    private int posPointer = -1;
+    private final BulkPayload payloads;
+    private final TermPositions termPos;
+
+    public PayloadPosInterval(BulkPayload payloads, TermPositions pos) {
+      this.payloads = payloads;
+      this.termPos = pos;
+    }
+    
+    @Override
+    public boolean payloadAvailable() {
+      return posPointer != termPos.posPointer;
+    }
+    
+    @Override
+    public boolean nextPayload(BytesRef ref) throws IOException {
+      final int pointer = termPos.posPointer;
+      if (pointer == posPointer) {
+        return false;
+      } else {
+        posPointer = pointer;
+        ref.bytes = payloads.bytes.bytes;
+        ref.offset = payloads.start[posPointer];
+        ref.length = payloads.length[posPointer];
+        return true;
+      }
+    }
+
+    @Override
+    public void reset() {
+      super.reset();
+      posPointer = -1;
+    }
+    
   }
+ 
 }
Index: lucene/src/java/org/apache/lucene/search/Weight.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/Weight.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/Weight.java	(working copy)
@@ -142,23 +142,39 @@
      */
     public final boolean topScorer;
     
+    /**
+     * if <code>true</code> Scorers / Queries supporting positional information will load
+     * positions.  
+     */
+    public final boolean needsPositions;
     
-    private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false);
+    /**
+     * if <code>true</code> Scorers / Queries supporting payload information will load
+     * payloads.  
+     */
+    public final boolean needsPayloads;
+    
+    
+    private static final ScorerContext DEFAULT_CONTEXT = new ScorerContext(true, false, false, false);
 
     /**
      * Returns a default {@link ScorerContext} template initialized with:
      * <ul>
      * <li>{@link #scoreDocsInOrder} = <code>true</code></li>
      * <li>{@link #topScorer} = <code>false</code></li>
+     * <li>{@link #needsPositions} = <code>false</code></li>
+     * <li>{@link #needsPayloads} = <code>false</code></li>
      * </ul>
      */
     public static ScorerContext def() {
       return DEFAULT_CONTEXT;
     }
     
-    private ScorerContext(boolean scoreDocsInOrder, boolean topScorer) {
+    private ScorerContext(boolean scoreDocsInOrder, boolean topScorer, boolean needsPostions, boolean needsPayloads) {
       this.scoreDocsInOrder = scoreDocsInOrder;
       this.topScorer = topScorer;
+      this.needsPositions = needsPostions;
+      this.needsPayloads = needsPayloads;
     }
 
     /**
@@ -172,7 +188,7 @@
       if (this.scoreDocsInOrder == scoreDocsInOrder) {
         return this;
       }
-      return new ScorerContext(scoreDocsInOrder, topScorer);
+      return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads);
     }
     
     /**
@@ -186,7 +202,36 @@
       if (this.topScorer == topScorer) {
         return this;
       }
-      return new ScorerContext(scoreDocsInOrder, topScorer);
+      return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads);
+    }
+    
+    
+    /**
+     * Creates and returns a copy of this context with the given value for
+     * {@link #needsPositions} and returns a new instance of
+     * {@link ScorerContext} iff the given value differs from the
+     * {@link #needsPositions}. Otherwise, this method has no effect and
+     * returns this instance.
+     */
+    public ScorerContext needsPositions(boolean needsPositions) {
+      if (this.needsPositions == needsPositions) {
+        return this;
+      }
+      return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads);
+    }
+    
+    /**
+     * Creates and returns a copy of this context with the given value for
+     * {@link #needsPayloads} and returns a new instance of
+     * {@link ScorerContext} iff the given value differs from the
+     * {@link #needsPayloads}. Otherwise, this method has no effect and
+     * returns this instance.
+     */
+    public ScorerContext needsPayloads(boolean needsPayloads) {
+      if (this.needsPayloads == needsPayloads) {
+        return this;
+      }
+      return new ScorerContext(scoreDocsInOrder, topScorer, needsPositions, needsPayloads);
     }
   }
 }
Index: lucene/src/java/org/apache/lucene/search/WithinPositionIterator.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/WithinPositionIterator.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/WithinPositionIterator.java	(revision 0)
@@ -0,0 +1,48 @@
+package org.apache.lucene.search;
+
+import java.io.IOException;
+import org.apache.lucene.search.PositionIntervalIterator.PositionIntervalFilter;
+
+
+public class WithinPositionIterator extends PositionIntervalIterator implements PositionIntervalFilter {
+  private int howMany;
+  private PositionIntervalIterator iterator;
+  public WithinPositionIterator(int howMany, PositionIntervalIterator iterator) {
+    this.howMany = howMany;
+    this.iterator = iterator;
+  }
+  
+  public WithinPositionIterator(int howMany) {
+    this(howMany, null);
+  }
+  @Override
+  public PositionInterval next() throws IOException {
+    PositionInterval interval = null;
+    while ((interval = iterator.next()) != null) {
+      if((interval.end - interval.begin) <= howMany){
+        return interval;
+      }
+    }
+    return null;
+  }
+
+  @Override
+  public PositionIntervalIterator[] subs(boolean inOrder) {
+    return new PositionIntervalIterator[] {iterator};
+  }
+
+  @Override
+  public int docID() {
+    return iterator.docID();
+  }
+
+  @Override
+  public Scorer getScorer() {
+    return iterator.getScorer();
+  }
+  
+  public PositionIntervalIterator filter(PositionIntervalIterator iter) {
+    return new WithinPositionIterator(howMany, iter);
+  }
+
+}
Index: lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java	(working copy)
@@ -21,15 +21,17 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PositionIntervalIterator;
+import org.apache.lucene.search.PositionIntervalIterator.PositionInterval;
 import org.apache.lucene.search.Scorer;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.ComplexExplanation;
-import org.apache.lucene.search.spans.TermSpans;
 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.SpansScorerWrapper;
 import org.apache.lucene.util.BytesRef;
 
 import java.io.IOException;
@@ -75,17 +77,17 @@
 
     @Override
     public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context),
+      return new PayloadTermSpanScorer((SpansScorerWrapper) query.getSpans(context),
           this, similarity, context.reader.norms(query.getField()));
     }
 
     protected class PayloadTermSpanScorer extends SpanScorer {
-      protected BytesRef payload;
+      protected final BytesRef payload = new BytesRef();
       protected float payloadScore;
       protected int payloadsSeen;
-      private final TermSpans termSpans;
+      private final SpansScorerWrapper termSpans;
 
-      public PayloadTermSpanScorer(TermSpans spans, Weight weight,
+      public PayloadTermSpanScorer(SpansScorerWrapper spans, Weight weight,
           Similarity similarity, byte[] norms) throws IOException {
         super(spans, weight, similarity, norms);
         termSpans = spans;
@@ -113,10 +115,9 @@
       }
 
       protected void processPayload(Similarity similarity) throws IOException {
-        final DocsAndPositionsEnum postings = termSpans.getPostings();
-        if (postings.hasPayload()) {
-          payload = postings.getPayload();
-          if (payload != null) {
+        if (termSpans.isPayloadAvailable()) {
+          final PositionInterval current = termSpans.current();
+          if (current.nextPayload(payload) && payload.length != 0) {
             payloadScore = function.currentScore(doc, term.field(),
                                                  spans.start(), spans.end(), payloadsSeen, payloadScore,
                                                  similarity.scorePayload(doc, term.field(), spans.start(),
Index: lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/spans/MockSpanQuery.java	(revision 0)
@@ -0,0 +1,154 @@
+package org.apache.lucene.search.spans;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Set;
+
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.PositionIntervalIterator.PositionIntervalFilter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.Weight.ScorerContext;
+
+public class MockSpanQuery extends SpanQuery {
+
+  private Query other;
+  private boolean needsPayloads;
+  private String field;
+  private PositionIntervalFilter filter;
+
+  public MockSpanQuery(Query other, boolean needsPayloads, String field, PositionIntervalFilter filter) {
+    this.other = other;
+    this.needsPayloads = needsPayloads;
+    this.field = field;
+    this.filter = filter;
+  }
+  
+  public MockSpanQuery(SpanQuery other, boolean needsPayloads) {
+    this(other, needsPayloads, other.getField(), null);
+  }
+  
+  @Override
+  public Spans getSpans(AtomicReaderContext context) throws IOException {
+    
+    if(other instanceof SpanQuery) {
+      return ((SpanQuery) other).getSpans(context);
+    }
+    
+    ReaderContext topReaderContext = context.reader.getTopReaderContext();
+
+    Weight weight = other.weight(new IndexSearcher(topReaderContext));
+    Scorer scorer = weight.scorer((AtomicReaderContext) topReaderContext, ScorerContext.def().needsPositions(true).needsPayloads(needsPayloads));
+    if (scorer == null) {
+      return EMPTY_SPANS;
+    }
+    return new SpansScorerWrapper(scorer, filter == null ? scorer.positions() : filter.filter(scorer.positions()));
+  }
+  
+  @Override
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
+    return new SpanWeight(this, searcher);
+  }
+  
+  
+
+  @Override
+  public void extractTerms(Set<Term> terms) {
+    other.extractTerms(terms);
+  }
+
+  @Override
+  public String getField() {
+    return field;
+  }
+
+  @Override
+  public String toString(String field) {
+    return other.toString();
+  }
+  
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + ((field == null) ? 0 : field.hashCode());
+    result = prime * result + (needsPayloads ? 1231 : 1237);
+    result = prime * result + ((other == null) ? 0 : other.hashCode());
+    return result;
+  }
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (!super.equals(obj))
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    MockSpanQuery other = (MockSpanQuery) obj;
+    if (field == null) {
+      if (other.field != null)
+        return false;
+    } else if (!field.equals(other.field))
+      return false;
+    if (needsPayloads != other.needsPayloads)
+      return false;
+    if (this.other == null) {
+      if (other.other != null)
+        return false;
+    } else if (!this.other.equals(other.other))
+      return false;
+    return true;
+  }
+
+
+
+
+  private static final class EmptySpans extends SpansScorerWrapper {
+
+    public EmptySpans() {
+      super(null, null);
+    }
+
+    @Override
+    public boolean next() {
+      return false;
+    }
+
+    @Override
+    public boolean skipTo(int target) {
+      return false;
+    }
+
+    @Override
+    public int doc() {
+      return DocIdSetIterator.NO_MORE_DOCS;
+    }
+
+    @Override
+    public int start() {
+      return -1;
+    }
+
+    @Override
+    public int end() {
+      return -1;
+    }
+
+    @Override
+    public Collection<byte[]> getPayload() {
+      return null;
+    }
+
+    @Override
+    public boolean isPayloadAvailable() {
+      return false;
+    }
+  }
+
+  public static final Spans EMPTY_SPANS = new EmptySpans();
+}
Index: lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java	(working copy)
@@ -17,86 +17,30 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.util.ToStringUtils;
-
-import java.io.IOException;
 import java.util.Set;
 
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+
 /** Matches spans containing a term. */
-public class SpanTermQuery extends SpanQuery {
+public class SpanTermQuery extends MockSpanQuery {
   protected Term term;
-
   /** Construct a SpanTermQuery matching the named term's spans. */
-  public SpanTermQuery(Term term) { this.term = term; }
-
+  public SpanTermQuery(Term term) { this(term, new TermQuery(term), true); }
+  public SpanTermQuery(Term term, boolean needsPayloads) { this(term, new TermQuery(term), needsPayloads); }
+  
+  private SpanTermQuery(Term term, TermQuery query, boolean needsPayloads) {
+    super(query, needsPayloads, term.field(), null);
+    this.term = term;
+  }
+  
   /** Return the term whose spans are matched. */
   public Term getTerm() { return term; }
 
-  @Override
-  public String getField() { return term.field(); }
   
   @Override
   public void extractTerms(Set<Term> terms) {
     terms.add(term);
   }
 
-  @Override
-  public String toString(String field) {
-    StringBuilder buffer = new StringBuilder();
-    if (term.field().equals(field))
-      buffer.append(term.text());
-    else
-      buffer.append(term.toString());
-    buffer.append(ToStringUtils.boost(getBoost()));
-    return buffer.toString();
-  }
-
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    result = prime * result + ((term == null) ? 0 : term.hashCode());
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (!super.equals(obj))
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    SpanTermQuery other = (SpanTermQuery) obj;
-    if (term == null) {
-      if (other.term != null)
-        return false;
-    } else if (!term.equals(other.term))
-      return false;
-    return true;
-  }
-
-  @Override
-  public Spans getSpans(final AtomicReaderContext context) throws IOException {
-    final IndexReader reader = context.reader;
-    final DocsAndPositionsEnum postings = reader.termPositionsEnum(reader.getDeletedDocs(),
-                                                                   term.field(),
-                                                                   term.bytes());
-
-    if (postings != null) {
-      return new TermSpans(postings, term);
-    } else {
-      if (reader.termDocsEnum(reader.getDeletedDocs(), term.field(), term.bytes()) != null) {
-        // term does exist, but has no positions
-        throw new IllegalStateException("field \"" + term.field() + "\" was indexed with Field.omitTermFreqAndPositions=true; cannot run SpanTermQuery (term=" + term.text() + ")");
-      } else {
-        // term does not exist
-        return TermSpans.EMPTY_TERM_SPANS;
-      }
-    }
-  }
 }
Index: lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/spans/SpansScorerWrapper.java	(revision 0)
@@ -0,0 +1,108 @@
+package org.apache.lucene.search.spans;
+
+/**
+ * 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.ArrayList;
+import java.util.Collection;
+
+import org.apache.lucene.search.PositionIntervalIterator;
+import org.apache.lucene.search.PositionIntervalIterator.PositionInterval;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.util.BytesRef;
+
+public class SpansScorerWrapper extends Spans {
+  private final Scorer scorer;
+  private final PositionIntervalIterator positions;
+  private PositionInterval current;
+  private int doc = -1;
+
+  public SpansScorerWrapper(Scorer scorer, PositionIntervalIterator positions) {
+    this.scorer = scorer;
+    this.positions = positions;
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    if (doc == -1) {
+      doc = scorer.nextDoc();
+    }
+
+    if (doc == Scorer.NO_MORE_DOCS) {
+      return false;
+    }
+
+    if ((current = positions.next()) == null) {
+      doc = scorer.nextDoc();
+      if (doc == Scorer.NO_MORE_DOCS) {
+        return false;
+      }
+      return (current = positions.next()) != null;
+    }
+    return true;
+  }
+
+  @Override
+  public boolean skipTo(int target) throws IOException {
+    doc = scorer.advance(target);
+    if (doc == Scorer.NO_MORE_DOCS) {
+      return false;
+    }
+    return (current = positions.next()) != null;
+  }
+
+  @Override
+  public int doc() {
+    return doc;
+  }
+
+  @Override
+  public int start() {
+    return current.begin;
+  }
+
+  @Override
+  public int end() {
+    return current.end;
+  }
+
+  @Override
+  public Collection<byte[]> getPayload() throws IOException {
+    BytesRef ref = new BytesRef();
+    final Collection<byte[]> payloads = new ArrayList<byte[]>();
+    while(current.nextPayload(ref)) {
+      if (ref.length > 0) {
+        byte[] retVal = new byte[ref.length];
+        System.arraycopy(ref.bytes, ref.offset, retVal, 0,
+            ref.length);
+        payloads.add(retVal);
+      }
+    }
+      
+    return payloads;
+  }
+
+  @Override
+  public boolean isPayloadAvailable() {
+    return current != null && current.payloadAvailable();
+  }
+
+  public PositionInterval current() {
+    return current;
+  }
+
+}
\ No newline at end of file
Index: lucene/src/java/org/apache/lucene/search/spans/TermSpans.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/TermSpans.java	(revision 1064068)
+++ lucene/src/java/org/apache/lucene/search/spans/TermSpans.java	(working copy)
@@ -1,169 +0,0 @@
-package org.apache.lucene.search.spans;
-/**
- * Copyright 2005 The Apache Software Foundation
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-import org.apache.lucene.index.Term;
-import org.apache.lucene.index.DocsAndPositionsEnum;
-import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.util.BytesRef;
-
-import java.io.IOException;
-import java.util.Collections;
-import java.util.Collection;
-
-/**
- * Expert:
- * Public for extension only
- */
-public class TermSpans extends Spans {
-  protected final DocsAndPositionsEnum postings;
-  protected final Term term;
-  protected int doc;
-  protected int freq;
-  protected int count;
-  protected int position;
-
-  public TermSpans(DocsAndPositionsEnum postings, Term term) throws IOException {
-    this.postings = postings;
-    this.term = term;
-    doc = -1;
-  }
-
-  // only for EmptyTermSpans (below)
-  TermSpans() {
-    term = null;
-    postings = null;
-  }
-
-  @Override
-  public boolean next() throws IOException {
-    if (count == freq) {
-      if (postings == null) {
-        return false;
-      }
-      doc = postings.nextDoc();
-      if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) {
-        return false;
-      }
-      freq = postings.freq();
-      count = 0;
-    }
-    position = postings.nextPosition();
-    count++;
-    return true;
-  }
-
-  @Override
-  public boolean skipTo(int target) throws IOException {
-    doc = postings.advance(target);
-    if (doc == DocsAndPositionsEnum.NO_MORE_DOCS) {
-      return false;
-    }
-
-    freq = postings.freq();
-    count = 0;
-    position = postings.nextPosition();
-    count++;
-
-    return true;
-  }
-
-  @Override
-  public int doc() {
-    return doc;
-  }
-
-  @Override
-  public int start() {
-    return position;
-  }
-
-  @Override
-  public int end() {
-    return position + 1;
-  }
-
-  // TODO: Remove warning after API has been finalized
-  @Override
-  public Collection<byte[]> getPayload() throws IOException {
-    final BytesRef payload = postings.getPayload();
-    final byte[] bytes;
-    if (payload != null) {
-      bytes = new byte[payload.length];
-      System.arraycopy(payload.bytes, payload.offset, bytes, 0, payload.length);
-    } else {
-      bytes = null;
-    }
-    return Collections.singletonList(bytes);
-  }
-
-  // TODO: Remove warning after API has been finalized
-  @Override
-  public boolean isPayloadAvailable() {
-    return postings.hasPayload();
-  }
-
-  @Override
-  public String toString() {
-    return "spans(" + term.toString() + ")@" +
-            (doc == -1 ? "START" : (doc == Integer.MAX_VALUE) ? "END" : doc + "-" + position);
-  }
-
-  public DocsAndPositionsEnum getPostings() {
-    return postings;
-  }
-
-  private static final class EmptyTermSpans extends TermSpans {
-
-    @Override
-    public boolean next() {
-      return false;
-    }
-
-    @Override
-    public boolean skipTo(int target) {
-      return false;
-    }
-
-    @Override
-    public int doc() {
-      return DocIdSetIterator.NO_MORE_DOCS;
-    }
-    
-    @Override
-    public int start() {
-      return -1;
-    }
-
-    @Override
-    public int end() {
-      return -1;
-    }
-
-    @Override
-    public Collection<byte[]> getPayload() {
-      return null;
-    }
-
-    @Override
-    public boolean isPayloadAvailable() {
-      return false;
-    }
-  }
-
-  public static final TermSpans EMPTY_TERM_SPANS = new EmptyTermSpans();
-}
Index: lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java	(revision 1064068)
+++ lucene/src/test/org/apache/lucene/search/TestBooleanQuery.java	(working copy)
@@ -17,21 +17,30 @@
  * limitations under the License.
  */
 
+import java.io.IOException;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryParser.QueryParser;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.PositionIntervalIterator.PositionInterval;
+import org.apache.lucene.search.Weight.ScorerContext;
+import org.apache.lucene.search.RangePositionsIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.lucene.util.ReaderUtil;
 
 public class TestBooleanQuery extends LuceneTestCase {
   
@@ -163,6 +172,102 @@
     dir1.close();
     dir2.close();
   }
+  
+  public void testConjunctionPositions() throws IOException {
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random, directory,
+        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(
+            MockTokenizer.WHITESPACE, true, false)));
+    {
+      Document doc = new Document();
+      doc.add(newField(
+          "field",
+          "Pease porridge hot! Pease porridge cold! Pease porridge in the pot nine days old! Some like it hot, some"
+              + " like it cold, Some like it in the pot nine days old! Pease porridge hot! Pease porridge cold!",
+          Field.Store.YES, Field.Index.ANALYZED));
+      writer.addDocument(doc);
+    }
+    
+    {
+      Document doc = new Document();
+      doc.add(newField(
+          "field",
+          "Pease porridge cold! Pease porridge hot! Pease porridge in the pot nine days old! Some like it cold, some"
+              + " like it hot, Some like it in the pot nine days old! Pease porridge cold! Pease porridge hot!",
+          Field.Store.YES, Field.Index.ANALYZED));
+      writer.addDocument(doc);
+    }
+    
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = new IndexSearcher(reader);
+    writer.close();
+    BooleanQuery query = new BooleanQuery();
+    query.add(new BooleanClause(new TermQuery(new Term("field", "porridge")), Occur.MUST));
+    query.add(new BooleanClause(new TermQuery(new Term("field", "pease")), Occur.MUST));
+    query.add(new BooleanClause(new TermQuery(new Term("field", "hot!")), Occur.MUST));
+    {
+      PositionFilterQuery filter = new PositionFilterQuery(query, new RangePositionsIterator(0,3));
+      TopDocs search = searcher.search(filter, 10);
+      ScoreDoc[] scoreDocs = search.scoreDocs;
+      assertEquals(1, search.totalHits);
+      assertEquals(0, scoreDocs[0].doc);
+    }
+    {
+      PositionFilterQuery filter = new PositionFilterQuery(query, new WithinPositionIterator(3));
+      TopDocs search = searcher.search(filter, 10);
+      ScoreDoc[] scoreDocs = search.scoreDocs;
+      assertEquals(2, search.totalHits);
+      assertEquals(0, scoreDocs[0].doc);
+      assertEquals(1, scoreDocs[1].doc);
+    }
+    
+    Weight weight = searcher.createWeight(query);
+    ReaderContext topReaderContext = searcher.getTopReaderContext();
+    AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+    assertEquals(1, leaves.length);
+    for (int i = 0; i < leaves.length; i++) {
+      Scorer scorer = weight.scorer(leaves[0], ScorerContext.def().topScorer(true).needsPositions(true));
+      {
+        int nextDoc = scorer.nextDoc();
+        assertEquals(0, nextDoc);
+        PositionIntervalIterator positions = scorer.positions();
+        PositionInterval interval = null;
+        int[] start = new int[] {0, 1, 2, 3, 4, 6, 7, 31, 32, 33};
+        int[] end = new int[] {3, 4, 5, 34, 34, 34, 34, 34, 35, 36};
+        // {start}term{end} - end is pos+1 
+        // {0}Pease {1}porridge {2}hot!{0} {3}Pease{1} {4}porridge{2} cold! {5}Pease {6}porridge in the pot nine days old! Some like it hot, some"
+        // like it cold, Some like it in the pot nine days old! {7}Pease {8}porridge {9}hot!{3,4,5,6,7} Pease{8} porridge{9} cold!",
+        for (int j = 0; j < end.length; j++) {
+          interval = positions.next();
+          assertNotNull(interval);
+          assertEquals(start[j], interval.begin);
+          assertEquals(end[j], interval.end);
+        }
+        assertNull(positions.next());
+      }
+      {
+        int nextDoc = scorer.nextDoc();
+        assertEquals(1, nextDoc);
+        PositionIntervalIterator positions = scorer.positions();
+        PositionInterval interval = null;
+        int[] start = new int[] {0, 1, 3, 4, 5, 6, 7, 31, 32, 34 };
+        int[] end = new int[] {6, 6, 6, 7, 8, 37, 37, 37, 37, 37 };
+        // {start}term{end} - end is pos+1
+        // {0}Pease {1}porridge cold! {2}Pease {3}porridge {4}hot!{0, 1, 2, 3} {5}Pease {4, 6}porridge in the pot nine days old! Some like it cold, some
+        // like it hot, Some like it in the pot nine days old! {7}Pease {8}porridge cold! {9}Pease porridge hot{5, 6, 7, 8, 9}!
+        for (int j = 0; j < end.length; j++) {
+          interval = positions.next();
+          assertNotNull(interval);
+          assertEquals(j + "", start[j], interval.begin);
+          assertEquals(j+ "", end[j], interval.end);
+        }
+        assertNull(positions.next());
+      }
+    }
+    searcher.close();
+    reader.close();
+    directory.close();
+  }
 }
  
 
Index: lucene/src/test/org/apache/lucene/search/TestTermQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestTermQuery.java	(revision 0)
+++ lucene/src/test/org/apache/lucene/search/TestTermQuery.java	(revision 0)
@@ -0,0 +1,300 @@
+package org.apache.lucene.search;
+
+/**
+ * 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.ArrayList;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.PositionIntervalIterator.PositionInterval;
+import org.apache.lucene.search.Weight.ScorerContext;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ReaderUtil;
+
+/**
+ * This class contains tests related to {@link TermQuery}
+ */
+public class TestTermQuery extends LuceneTestCase {
+
+  private String fieldName;
+  private boolean usePayload;
+
+  public void setUp() throws Exception {
+    super.setUp();
+    fieldName = "field" + random.nextInt();
+    usePayload = true;// random.nextBoolean();
+  }
+
+  /**
+   * Simple testcase for {@link TermScorer#positions()}
+   */
+  public void testPositionsSimple() throws IOException {
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random, directory,
+        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(
+            MockTokenizer.WHITESPACE, true, usePayload)));
+    for (int i = 0; i < 39; i++) {
+      Document doc = new Document();
+      doc.add(newField(fieldName, "1 2 3 4 5 6 7 8 9 10 "
+          + "1 2 3 4 5 6 7 8 9 10 " + "1 2 3 4 5 6 7 8 9 10 "
+          + "1 2 3 4 5 6 7 8 9 10", Field.Store.YES, Field.Index.ANALYZED));
+      writer.addDocument(doc);
+    }
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = new IndexSearcher(reader);
+    writer.close();
+
+    for (int i = 0; i < 39 * RANDOM_MULTIPLIER; i++) {
+      TermQuery one = new TermQuery(new Term(fieldName, "1"));
+      ReaderContext topReaderContext = reader.getTopReaderContext();
+      AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+      Weight weight = one.createWeight(searcher);
+      for (AtomicReaderContext atomicReaderContext : leaves) {
+        Scorer scorer = weight.scorer(atomicReaderContext, ScorerContext.def()
+            .needsPositions(true).needsPayloads(usePayload));
+        assertNotNull(scorer);
+        final int advance = scorer.advance(1 + random.nextInt(27));
+        PositionIntervalIterator positions = scorer.positions();
+
+        do {
+          PositionInterval interval = null;
+          String msg = "Advanced to: " + advance + " current doc: "
+              + scorer.docID() + " usePayloads: " + usePayload;
+          assertNotNull(msg, (interval = positions.next()));
+          assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f);
+
+          assertEquals(msg, 0, interval.begin);
+          assertEquals(msg, 1, interval.end);
+          checkPayload(0, interval);
+
+          assertNotNull(msg, (interval = positions.next()));
+          assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f);
+          assertEquals(msg, 10, interval.begin);
+          assertEquals(msg, 11, interval.end);
+          checkPayload(10, interval);
+
+          assertNotNull(msg, (interval = positions.next()));
+          assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f);
+          assertEquals(msg, 20, interval.begin);
+          assertEquals(msg, 21, interval.end);
+          checkPayload(20, interval);
+
+          assertNotNull(msg, (interval = positions.next()));
+          assertEquals(msg, 4.0f, positions.getScorer().freq(), 0.0f);
+          assertEquals(msg, 30, interval.begin);
+          assertEquals(msg, 31, interval.end);
+          checkPayload(30, interval);
+
+          assertNull(msg, (interval = positions.next()));
+
+        } while (scorer.nextDoc() != Scorer.NO_MORE_DOCS);
+      }
+    }
+    searcher.close();
+    reader.close();
+    directory.close();
+  }
+
+  public final void checkPayload(int pos, PositionInterval interval) throws IOException {
+    if (!CodecProvider.getDefault().getFieldCodec(fieldName).equals("Standard"))
+      return; // nocommit remove this once all codecs work with bulk payloads
+    if (usePayload) {
+      assertTrue(interval.payloadAvailable());
+      BytesRef bytes = new BytesRef();
+      assertTrue(interval.nextPayload(bytes));
+      assertFalse(interval.payloadAvailable());
+      assertEquals("pos: " + pos, bytes.utf8ToString());
+    } else {
+      assertFalse(interval.payloadAvailable());
+    }
+
+  }
+
+  /**
+   * this test indexes random numbers within a range into a field and checks
+   * their occurrences by searching for a number from that range selected at
+   * random. All positions for that number are safed up front and compared to
+   * the terms scorers positions.
+   * 
+   */
+  public void testRandomPositons() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random, dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(
+            MockTokenizer.WHITESPACE, true, usePayload)));
+    int numDocs = 131;
+    int max = 1051;
+    int term = random.nextInt(max);
+    Integer[][] positionsInDoc = new Integer[numDocs][];
+    for (int i = 0; i < numDocs; i++) {
+      Document doc = new Document();
+      ArrayList<Integer> positions = new ArrayList<Integer>();
+      StringBuilder builder = new StringBuilder();
+      for (int j = 0; j < 3049; j++) {
+        int nextInt = random.nextInt(max);
+        builder.append(nextInt).append(" ");
+        if (nextInt == term) {
+          positions.add(Integer.valueOf(j));
+        }
+      }
+      doc.add(newField(fieldName, builder.toString(), Field.Store.YES,
+          Field.Index.ANALYZED));
+      positionsInDoc[i] = positions.toArray(new Integer[0]);
+      writer.addDocument(doc);
+    }
+
+    IndexReader reader = writer.getReader();
+    writer.close();
+    IndexSearcher searcher = new IndexSearcher(reader);
+
+    for (int i = 0; i < 39 * RANDOM_MULTIPLIER; i++) {
+      TermQuery one = new TermQuery(new Term(fieldName, "" + term));
+      ReaderContext topReaderContext = reader.getTopReaderContext();
+      AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+      Weight weight = one.createWeight(searcher);
+      for (AtomicReaderContext atomicReaderContext : leaves) {
+        Scorer scorer = weight.scorer(atomicReaderContext, ScorerContext.def()
+            .needsPositions(true));
+        assertNotNull(scorer);
+        int initDoc = 0;
+        int maxDoc = atomicReaderContext.reader.maxDoc();
+        // initially advance or do next doc
+        if (random.nextBoolean()) {
+          initDoc = scorer.nextDoc();
+        } else {
+          initDoc = scorer.advance(random.nextInt(maxDoc));
+        }
+        // now run through the scorer and check if all positions are there...
+        do {
+          int docID = scorer.docID();
+          if (docID == Scorer.NO_MORE_DOCS) {
+            break;
+          }
+          PositionIntervalIterator positions = scorer.positions();
+          Integer[] pos = positionsInDoc[atomicReaderContext.docBase + docID];
+          
+          assertEquals((float)pos.length, positions.getScorer().freq(), 0.0f);
+          // number of positions read should be random - don't read all of them
+          // allways
+          final int howMany = random.nextInt(20) == 0 ? pos.length
+              - random.nextInt(pos.length) : pos.length;
+          PositionInterval interval = null;
+
+          for (int j = 0; j < howMany; j++) {
+            assertNotNull((interval = positions.next()));
+            assertEquals("iteration: " + i + " initDoc: " + initDoc + " doc: "
+                + docID + " base: " + atomicReaderContext.docBase
+                + " positions: " + Arrays.toString(pos) + " usePayloads: "
+                + usePayload, pos[j].intValue(), interval.begin);
+            assertEquals(pos[j].intValue() + 1, interval.end);
+          }
+          if (howMany == pos.length) {
+            assertNull((interval = positions.next()));
+          }
+
+          if (random.nextInt(10) == 0) { // once is a while advance
+            scorer.advance(docID + 1 + random.nextInt((maxDoc - docID)));
+          }
+
+        } while (scorer.nextDoc() != Scorer.NO_MORE_DOCS);
+      }
+
+    }
+    reader.close();
+    dir.close();
+  }
+
+  /**
+   * tests retrieval of positions for terms that have a large number of
+   * occurrences to force test of buffer refill during positions iteration.
+   */
+  public void testLargeNumberOfPositions() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random, dir,
+        newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(
+            MockTokenizer.WHITESPACE, true, usePayload)));
+    int howMany = 1000;
+    for (int i = 0; i < 39; i++) {
+      Document doc = new Document();
+      StringBuilder builder = new StringBuilder();
+      for (int j = 0; j < howMany; j++) {
+        if (j % 2 == 0) {
+          builder.append("even ");
+        } else {
+          builder.append("odd ");
+        }
+      }
+      doc.add(newField(fieldName, builder.toString(), Field.Store.YES,
+          Field.Index.ANALYZED));
+      writer.addDocument(doc);
+    }
+
+    // now do seaches
+    IndexReader reader = writer.getReader();
+    writer.close();
+    IndexSearcher searcher = new IndexSearcher(reader);
+
+    for (int i = 0; i < 39 * RANDOM_MULTIPLIER; i++) {
+      TermQuery one = new TermQuery(new Term(fieldName, "even"));
+      ReaderContext topReaderContext = reader.getTopReaderContext();
+      AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+      Weight weight = one.createWeight(searcher);
+      PositionInterval interval = null;
+      for (AtomicReaderContext atomicReaderContext : leaves) {
+        Scorer scorer = weight.scorer(atomicReaderContext, ScorerContext.def()
+            .needsPositions(true));
+        assertNotNull(scorer);
+
+        int initDoc = 0;
+        int maxDoc = atomicReaderContext.reader.maxDoc();
+        // initially advance or do next doc
+        if (random.nextBoolean()) {
+          initDoc = scorer.nextDoc();
+        } else {
+          initDoc = scorer.advance(random.nextInt(maxDoc));
+        }
+        String msg = "Iteration: " + i + " initDoc: " + initDoc + " payloads: "
+            + usePayload;
+        PositionIntervalIterator positions = scorer.positions();
+        assertEquals(howMany / 2.f, positions.getScorer().freq(), 0.0);
+        for (int j = 0; j < howMany; j += 2) {
+          assertNotNull("next returned nullat index: " + j + " with freq: "
+              + positions.getScorer().freq() + " -- " + msg,(interval = positions.next()));
+          assertEquals("position missmatch index: " + j + " with freq: "
+              + positions.getScorer().freq() + " -- " + msg, j, interval.begin);
+        }
+        assertNull("next returned nonNull -- " + msg,(interval = positions.next()));
+
+      }
+    }
+    reader.close();
+    dir.close();
+  }
+
+}
\ No newline at end of file
Index: lucene/src/test/org/apache/lucene/search/spans/TestSpanFirstQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/TestSpanFirstQuery.java	(revision 1064068)
+++ lucene/src/test/org/apache/lucene/search/spans/TestSpanFirstQuery.java	(working copy)
@@ -26,6 +26,8 @@
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.RangePositionsIterator;
+import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
@@ -64,4 +66,37 @@
     reader.close();
     dir.close();
   }
+  
+  public void testStartPositionsAlt() throws Exception {
+    Directory dir = newDirectory();
+    
+    // mimic StopAnalyzer
+    CharacterRunAutomaton stopSet = new CharacterRunAutomaton(new RegExp("the|a|of").toAutomaton());
+    Analyzer analyzer = new MockAnalyzer(MockTokenizer.SIMPLE, true, stopSet, true);
+    
+    RandomIndexWriter writer = new RandomIndexWriter(random, dir, analyzer);
+    Document doc = new Document();
+    doc.add(newField("field", "the quick brown fox", Field.Index.ANALYZED));
+    writer.addDocument(doc);
+    Document doc2 = new Document();
+    doc2.add(newField("field", "quick brown fox", Field.Index.ANALYZED));
+    writer.addDocument(doc2);
+    
+    IndexReader reader = writer.getReader();
+    IndexSearcher searcher = new IndexSearcher(reader);
+    
+    // user queries on "starts-with quick"
+    SpanQuery sfq = new MockSpanQuery(new TermQuery(new Term("field", "quick")), false,"field", new RangePositionsIterator(0, 1));
+    assertEquals(1, searcher.search(sfq, 10).totalHits);
+    
+    // user queries on "starts-with the quick"
+    SpanQuery include =  new MockSpanQuery(new TermQuery(new Term("field", "quick")), false,"field", new RangePositionsIterator(0, 2));
+    sfq = new SpanNotQuery(include, sfq);
+    assertEquals(1, searcher.search(sfq, 10).totalHits);
+    
+    writer.close();
+    searcher.close();
+    reader.close();
+    dir.close();
+  }
 }
