Index: lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java
===================================================================
--- lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java	(revision 1061682)
+++ lucene/contrib/highlighter/src/java/org/apache/lucene/search/highlight/WeightedSpanTermExtractor.java	(working copy)
@@ -30,6 +30,7 @@
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.memory.MemoryIndex;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
@@ -50,7 +51,7 @@
 
   private String fieldName;
   private TokenStream tokenStream;
-  private Map<String,IndexReader> readers = new HashMap<String,IndexReader>(10); 
+  private Map<String,AtomicReaderContext> readers = new HashMap<String,AtomicReaderContext>(10); 
   private String defaultField;
   private boolean expandMultiTermQuery;
   private boolean cachedTokenStream;
@@ -66,11 +67,11 @@
   }
 
   private void closeReaders() {
-    Collection<IndexReader> readerSet = readers.values();
+    Collection<AtomicReaderContext> ctxSet = readers.values();
 
-    for (final IndexReader reader : readerSet) {
+    for (final AtomicReaderContext ctx : ctxSet) {
       try {
-        reader.close();
+        ctx.reader.close();
       } catch (IOException e) {
         // alert?
       }
@@ -149,7 +150,7 @@
         query = mtq;
       }
       if (mtq.getField() != null) {
-        IndexReader ir = getReaderForField(mtq.getField());
+        IndexReader ir = getLeafContextForField(mtq.getField()).reader;
         extract(query.rewrite(ir), terms);
       }
     } else if (query instanceof MultiPhraseQuery) {
@@ -234,7 +235,7 @@
     final boolean mustRewriteQuery = mustRewriteQuery(spanQuery);
     if (mustRewriteQuery) {
       for (final String field : fieldNames) {
-        final SpanQuery rewrittenQuery = (SpanQuery) spanQuery.rewrite(getReaderForField(field));
+        final SpanQuery rewrittenQuery = (SpanQuery) spanQuery.rewrite(getLeafContextForField(field).reader);
         queries.put(field, rewrittenQuery);
         rewrittenQuery.extractTerms(nonWeightedTerms);
       }
@@ -246,12 +247,12 @@
 
     for (final String field : fieldNames) {
 
-      IndexReader reader = getReaderForField(field);
+      AtomicReaderContext context = getLeafContextForField(field);
       final Spans spans;
       if (mustRewriteQuery) {
-        spans = queries.get(field).getSpans(reader);
+        spans = queries.get(field).getSpans(context);
       } else {
-        spans = spanQuery.getSpans(reader);
+        spans = spanQuery.getSpans(context);
       }
 
 
@@ -317,22 +318,23 @@
     return rv;
   }
 
-  private IndexReader getReaderForField(String field) throws IOException {
+  private AtomicReaderContext getLeafContextForField(String field) throws IOException {
     if(wrapToCaching && !cachedTokenStream && !(tokenStream instanceof CachingTokenFilter)) {
       tokenStream = new CachingTokenFilter(tokenStream);
       cachedTokenStream = true;
     }
-    IndexReader reader = readers.get(field);
-    if (reader == null) {
+    AtomicReaderContext context = readers.get(field);
+    if (context == null) {
       MemoryIndex indexer = new MemoryIndex();
       indexer.addField(field, tokenStream);
       tokenStream.reset();
       IndexSearcher searcher = indexer.createSearcher();
-      reader = searcher.getIndexReader();
-      readers.put(field, reader);
+      // MEM index has only atomic ctx
+      context = (AtomicReaderContext) searcher.getTopReaderContext();
+      readers.put(field, context);
     }
 
-    return reader;
+    return context;
   }
 
   /**
Index: lucene/src/java/org/apache/lucene/index/BulkPostingsEnum.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/BulkPostingsEnum.java	(revision 1061682)
+++ 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/codecs/standard/StandardPostingsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(revision 1061682)
+++ 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;
@@ -1040,6 +1038,7 @@
       final int[] buffer = new int[BULK_BUFFER_SIZE];
       int limit;
       int pending;
+      BulkPayload payloads;
   
       @Override
       public int[] getBuffer() {
@@ -1055,15 +1054,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 {
@@ -1084,6 +1093,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/CachingSpanFilter.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/CachingSpanFilter.java	(working copy)
@@ -62,14 +62,15 @@
 
   @Override
   public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
-    SpanFilterResult result = getCachedResult(context.reader);
+    SpanFilterResult result = getCachedResult(context);
     return result != null ? result.getDocIdSet() : null;
   }
   
   // for testing
   int hitCount, missCount;
 
-  private SpanFilterResult getCachedResult(IndexReader reader) throws IOException {
+  private SpanFilterResult getCachedResult(AtomicReaderContext context) throws IOException {
+    final IndexReader reader = context.reader;
 
     final Object coreKey = reader.getCoreCacheKey();
     final Object delCoreKey = reader.hasDeletions() ? reader.getDeletedDocs() : coreKey;
@@ -81,7 +82,7 @@
     }
 
     missCount++;
-    result = filter.bitSpans(reader);
+    result = filter.bitSpans(context);
 
     cache.put(coreKey, delCoreKey, result);
     return result;
@@ -89,8 +90,8 @@
 
 
   @Override
-  public SpanFilterResult bitSpans(IndexReader reader) throws IOException {
-    return getCachedResult(reader);
+  public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
+    return getCachedResult(context);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/search/Positions.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/Positions.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/Positions.java	(revision 0)
@@ -0,0 +1,43 @@
+package org.apache.lucene.search;
+/**
+ * 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 java.io.IOException;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * 
+ * TODO add documentation
+ */
+public abstract class Positions {
+  public int freq;
+  public int start;
+  public int end;
+  public final boolean hasPayloads;
+  public final BytesRef payload;
+  
+  public Positions() {
+    this(null);
+  }
+  
+  public Positions(BytesRef ref) {
+    payload = ref;
+    hasPayloads = ref != null;
+  }
+  
+  public abstract boolean next() throws IOException;
+  
+}
Index: lucene/src/java/org/apache/lucene/search/Scorer.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/Scorer.java	(revision 1061682)
+++ 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 Positions 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/SpanFilter.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/SpanFilter.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/SpanFilter.java	(working copy)
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 import java.io.IOException;
 
@@ -30,9 +30,9 @@
 public abstract class SpanFilter extends Filter{
   /** Returns a SpanFilterResult with true for documents which should be permitted in
     search results, and false for those that should not and Spans for where the true docs match.
-   * @param reader The {@link org.apache.lucene.index.IndexReader} to load position and DocIdSet information from
+   * @param context The {@link AtomicReaderContext} to load position and DocIdSet information from
    * @return A {@link SpanFilterResult}
    * @throws java.io.IOException if there was an issue accessing the necessary information
    * */
-  public abstract SpanFilterResult bitSpans(IndexReader reader) throws IOException;
+  public abstract SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException;
 }
Index: lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/SpanQueryFilter.java	(working copy)
@@ -16,7 +16,6 @@
  */
 
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.Spans;
@@ -54,15 +53,15 @@
 
   @Override
   public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
-    SpanFilterResult result = bitSpans(context.reader);
+    SpanFilterResult result = bitSpans(context);
     return result.getDocIdSet();
   }
 
   @Override
-  public SpanFilterResult bitSpans(IndexReader reader) throws IOException {
+  public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
 
-    final OpenBitSet bits = new OpenBitSet(reader.maxDoc());
-    Spans spans = query.getSpans(reader);
+    final OpenBitSet bits = new OpenBitSet(context.reader.maxDoc());
+    Spans spans = query.getSpans(context);
     List<SpanFilterResult.PositionInfo> tmp = new ArrayList<SpanFilterResult.PositionInfo>(20);
     int currentDoc = -1;
     SpanFilterResult.PositionInfo currentInfo = null;
Index: lucene/src/java/org/apache/lucene/search/TermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TermQuery.java	(revision 1061682)
+++ 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 1061682)
+++ lucene/src/java/org/apache/lucene/search/TermScorer.java	(working copy)
@@ -21,7 +21,9 @@
 
 import org.apache.lucene.index.BulkPostingsEnum;
 import org.apache.lucene.index.BulkPostingsEnum.BlockReader;
+import org.apache.lucene.index.BulkPostingsEnum.BulkPayload;
 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 +53,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 +73,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 +83,7 @@
     docDeltas = docDeltasReader.getBuffer();
     this.freqsReader = freqReader;
     freqs = freqsReader.getBuffer();
+    positions = posDeltaReader == null ? null : getPositions(posDeltaReader, needsPayloads); 
     reset();
     this.skipDocs = skipDocs;
     this.norms = norms;
@@ -253,6 +263,9 @@
       assert freqPointerMax != 0;
       freqPointer = 0;
     }
+    if (positions != null) { // compiler should optimize this away 
+      positions.fill();
+    }
   }
   
   private void fillDeltas() throws IOException {
@@ -272,5 +285,114 @@
     assert freqPointerMax >= freqPointer: "fP=" + freqPointer + " fPMax=" + freqPointerMax;
     --docPointer;
     --freqPointer;
+    if (positions != null) { // compiler should optimize this away 
+      positions.reset();
+    }
+  }
+  
+  @Override
+  public Positions positions() throws IOException {
+    return positions;
   }
+  
+  class TermPositions extends Positions {
+    private int positionsPending;
+    private int posPointerMax;
+    protected int posPointer;
+    private final int[] posBuffer;
+    private final BlockReader posReader;
+
+    public TermPositions(BlockReader blockReader) {
+      this(blockReader, null); // no payloads here
+      
+    }
+    
+    protected TermPositions(BlockReader blockReader, BytesRef payloads) {
+      super(payloads);
+      posReader = blockReader;
+      posBuffer = blockReader.getBuffer();
+    }
+    
+    void fill() throws IOException {
+      freq = freqs[freqPointer];
+      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;
+      }
+      end = start = 0;
+      positionsPending = freq;
+    }
+    
+    void reset() {
+      start = 0;
+      posPointer = posReader.offset();
+      posPointerMax = posReader.end();
+      assert posPointerMax >= posPointer: "pP=" + posPointer + " pPMax=" + posPointerMax;
+      --posPointer;
+      positionsPending = freq = 0;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (--positionsPending >= 0) {
+        if(++posPointer >= posPointerMax) {
+          posPointerMax = posReader.fill();
+          assert posPointerMax != 0;
+          posPointer = 0;
+        }
+        start += posBuffer[posPointer];
+        end = start+1;
+        return true;
+      }
+      end = start = 0;
+      positionsPending = 0;
+      return false;
+    }
+  }
+  
+  final class TermPositionsAndPayloads extends TermPositions {
+    private final BulkPayload payloads;
+
+    TermPositionsAndPayloads(BlockReader blockReader,
+        BulkPayload payloads) {
+      super(blockReader, payloads.bytes);
+      this.payloads = payloads;
+    }
+    @Override
+    void reset() {
+      super.reset();
+      payload.offset = 0;
+      payload.length = 0;
+    }
+
+    @Override
+    public boolean next() throws IOException {
+      if (super.next()) {
+        payload.offset = this.payloads.start[posPointer];
+        payload.length = this.payloads.length[posPointer];
+        return true;
+      }
+      
+      payload.offset = 0;
+      payload.length = 0;
+      return false;
+    }
+  }
+
+  private final TermPositions getPositions(BlockReader blockReader,
+      boolean needsPayload) {
+    if (needsPayload && blockReader.payloadsAvailable()) {
+      return new TermPositionsAndPayloads(blockReader,  blockReader.getPayloadBuffer());
+    }
+    return new TermPositions(blockReader);
+  }
+ 
 }
Index: lucene/src/java/org/apache/lucene/search/Weight.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/Weight.java	(revision 1061682)
+++ 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/payloads/PayloadNearQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java	(working copy)
@@ -144,7 +144,7 @@
 
     @Override
     public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new PayloadNearSpanScorer(query.getSpans(context.reader), this,
+      return new PayloadNearSpanScorer(query.getSpans(context), this,
           similarity, context.reader.norms(query.getField()));
     }
   }
Index: lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/payloads/PayloadSpanUtil.java	(working copy)
@@ -23,8 +23,9 @@
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DisjunctionMaxQuery;
@@ -33,6 +34,7 @@
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.spans.MultiSpansWrapper;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanOrQuery;
 import org.apache.lucene.search.spans.SpanQuery;
@@ -48,14 +50,14 @@
  * 
  */
 public class PayloadSpanUtil {
-  private IndexReader reader;
+  private ReaderContext context;
 
   /**
-   * @param reader
+   * @param context
    *          that contains doc with payloads to extract
    */
-  public PayloadSpanUtil(IndexReader reader) {
-    this.reader = reader;
+  public PayloadSpanUtil(ReaderContext context) {
+    this.context = context;
   }
 
   /**
@@ -169,7 +171,7 @@
 
   private void getPayloads(Collection<byte []> payloads, SpanQuery query)
       throws IOException {
-    Spans spans = query.getSpans(reader);
+    Spans spans = MultiSpansWrapper.wrap(context, query);
 
     while (spans.next() == true) {
       if (spans.isPayloadAvailable()) {
Index: lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java	(working copy)
@@ -21,12 +21,12 @@
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Positions;
 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;
@@ -75,7 +75,7 @@
 
     @Override
     public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-      return new PayloadTermSpanScorer((TermSpans) query.getSpans(context.reader),
+      return new PayloadTermSpanScorer((SpansScorerWrapper) query.getSpans(context),
           this, similarity, context.reader.norms(query.getField()));
     }
 
@@ -83,9 +83,9 @@
       protected BytesRef payload;
       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,9 +113,9 @@
       }
 
       protected void processPayload(Similarity similarity) throws IOException {
-        final DocsAndPositionsEnum postings = termSpans.getPostings();
-        if (postings.hasPayload()) {
-          payload = postings.getPayload();
+        final Positions postings = termSpans.getPositions();
+        if (termSpans.isPayloadAvailable()) {
+          payload = postings.payload;
           if (payload != null) {
             payloadScore = function.currentScore(doc, term.field(),
                                                  spans.start(), spans.end(), payloadsSeen, payloadScore,
Index: lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/FieldMaskingSpanQuery.java	(working copy)
@@ -21,6 +21,7 @@
 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.Query;
 import org.apache.lucene.search.Weight;
@@ -91,8 +92,8 @@
   // ...this is done to be more consistent with things like SpanFirstQuery
   
   @Override
-  public Spans getSpans(IndexReader reader) throws IOException {
-    return maskedQuery.getSpans(reader);
+  public Spans getSpans(AtomicReaderContext context) throws IOException {
+    return maskedQuery.getSpans(context);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/search/spans/MultiSpansWrapper.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/MultiSpansWrapper.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/spans/MultiSpansWrapper.java	(revision 0)
@@ -0,0 +1,139 @@
+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 java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+
+import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
+import org.apache.lucene.util.ReaderUtil;
+
+public class MultiSpansWrapper extends Spans {
+
+  private SpanQuery query;
+  private AtomicReaderContext[] leaves;
+  private int leafOrd = 0;
+  private Spans current;
+
+  private MultiSpansWrapper(AtomicReaderContext[] leaves, SpanQuery query) {
+    this.query = query;
+    this.leaves = leaves;
+
+  }
+  
+  public static Spans wrap(ReaderContext topLevelReaderContext, SpanQuery query) throws IOException {
+    AtomicReaderContext[] leaves = ReaderUtil.leaves(topLevelReaderContext);
+    if(leaves.length == 1) {
+      return query.getSpans(leaves[0]);
+    }
+    return new MultiSpansWrapper(leaves, query);
+  }
+
+  @Override
+  public boolean next() throws IOException {
+    if (leafOrd >= leaves.length) {
+      return false;
+    }
+    if (current == null) {
+      current = query.getSpans(leaves[leafOrd]);
+    }
+    while(true) {
+      if (current.next()) {
+        return true;
+      }
+      if (++leafOrd < leaves.length) {
+        current = query.getSpans(leaves[leafOrd]);
+      } else {
+        current = null;
+        break;
+      }
+    }
+    return false;
+  }
+
+  @Override
+  public boolean skipTo(int target) throws IOException {
+    if (leafOrd >= leaves.length) {
+      return false;
+    }
+
+    int subIndex = ReaderUtil.subIndex(target, leaves);
+    assert subIndex >= leafOrd;
+    if (subIndex != leafOrd) {
+      current = query.getSpans(leaves[subIndex]);
+      leafOrd = subIndex;
+    } else if (current == null) {
+      current = query.getSpans(leaves[leafOrd]);
+    }
+    while (true) {
+      if (current.skipTo(target - leaves[leafOrd].docBase)) {
+        return true;
+      }
+      if (++leafOrd < leaves.length) {
+        current = query.getSpans(leaves[leafOrd]);
+      } else {
+          current = null;
+          break;
+      }
+    }
+
+    return false;
+  }
+
+  @Override
+  public int doc() {
+    if (current == null) {
+      return DocsEnum.NO_MORE_DOCS;
+    }
+    return current.doc() + leaves[leafOrd].docBase;
+  }
+
+  @Override
+  public int start() {
+    if (current == null) {
+      return DocsEnum.NO_MORE_DOCS;
+    }
+    return current.start();
+  }
+
+  @Override
+  public int end() {
+    if (current == null) {
+      return DocsEnum.NO_MORE_DOCS;
+    }
+    return current.end();
+  }
+
+  @Override
+  public Collection<byte[]> getPayload() throws IOException {
+    if (current == null) {
+      return Collections.emptyList();
+    }
+    return current.getPayload();
+  }
+
+  @Override
+  public boolean isPayloadAvailable() {
+    if (current == null) {
+      return false;
+    }
+    return current.isPayloadAvailable();
+  }
+
+}
Index: lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/NearSpansOrdered.java	(working copy)
@@ -17,7 +17,7 @@
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.ArrayUtil;
 
 import java.io.IOException;
@@ -28,6 +28,7 @@
 import java.util.List;
 import java.util.Collection;
 import java.util.Set;
+import java.util.TreeSet;
 
 /** A Spans that is formed from the ordered subspans of a SpanNearQuery
  * where the subspans do not overlap and have a maximum slop between them.
@@ -77,11 +78,11 @@
   private SpanNearQuery query;
   private boolean collectPayloads = true;
   
-  public NearSpansOrdered(SpanNearQuery spanNearQuery, IndexReader reader) throws IOException {
-    this(spanNearQuery, reader, true);
+  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context) throws IOException {
+    this(spanNearQuery, context, true);
   }
 
-  public NearSpansOrdered(SpanNearQuery spanNearQuery, IndexReader reader, boolean collectPayloads)
+  public NearSpansOrdered(SpanNearQuery spanNearQuery, AtomicReaderContext context, boolean collectPayloads)
   throws IOException {
     if (spanNearQuery.getClauses().length < 2) {
       throw new IllegalArgumentException("Less than 2 clauses: "
@@ -94,7 +95,7 @@
     matchPayload = new LinkedList<byte[]>();
     subSpansByDoc = new Spans[clauses.length];
     for (int i = 0; i < clauses.length; i++) {
-      subSpans[i] = clauses[i].getSpans(reader);
+      subSpans[i] = clauses[i].getSpans(context);
       subSpansByDoc[i] = subSpans[i]; // used in toSameDoc()
     }
     query = spanNearQuery; // kept for toString() only.
Index: lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/NearSpansUnordered.java	(working copy)
@@ -18,6 +18,7 @@
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.PriorityQueue;
 
 import java.io.IOException;
@@ -131,7 +132,7 @@
   }
 
 
-  public NearSpansUnordered(SpanNearQuery query, IndexReader reader)
+  public NearSpansUnordered(SpanNearQuery query, AtomicReaderContext context)
     throws IOException {
     this.query = query;
     this.slop = query.getSlop();
@@ -141,7 +142,7 @@
     subSpans = new Spans[clauses.length];    
     for (int i = 0; i < clauses.length; i++) {
       SpansCell cell =
-        new SpansCell(clauses[i].getSpans(reader), i);
+        new SpansCell(clauses[i].getSpans(context), i);
       ordered.add(cell);
       subSpans[i] = cell.spans;
     }
Index: lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanFirstQuery.java	(working copy)
@@ -38,7 +38,7 @@
 
   @Override
   protected AcceptStatus acceptPosition(Spans spans) throws IOException {
-    assert spans.start() != spans.end();
+    assert spans.start() != spans.end() : "start equals end: " + spans.start();
     if (spans.start() >= end)
       return AcceptStatus.NO_AND_ADVANCE;
     else if (spans.end() <= end)
Index: lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanMultiTermQueryWrapper.java	(working copy)
@@ -20,6 +20,7 @@
 import java.io.IOException;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.MultiTermQuery;
 import org.apache.lucene.search.Query;
@@ -88,7 +89,7 @@
   }
   
   @Override
-  public Spans getSpans(IndexReader reader) throws IOException {
+  public Spans getSpans(AtomicReaderContext context) throws IOException {
     throw new UnsupportedOperationException("Query should have been rewritten");
   }
 
Index: lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanNearQuery.java	(working copy)
@@ -27,6 +27,7 @@
 
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.ToStringUtils;
@@ -116,16 +117,16 @@
   }
 
   @Override
-  public Spans getSpans(final IndexReader reader) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context) throws IOException {
     if (clauses.size() == 0)                      // optimize 0-clause case
-      return new SpanOrQuery(getClauses()).getSpans(reader);
+      return new SpanOrQuery(getClauses()).getSpans(context);
 
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return clauses.get(0).getSpans(reader);
+      return clauses.get(0).getSpans(context);
 
     return inOrder
-            ? (Spans) new NearSpansOrdered(this, reader, collectPayloads)
-            : (Spans) new NearSpansUnordered(this, reader);
+            ? (Spans) new NearSpansOrdered(this, context, collectPayloads)
+            : (Spans) new NearSpansUnordered(this, context);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanNotQuery.java	(working copy)
@@ -18,6 +18,7 @@
  */
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.ToStringUtils;
@@ -74,12 +75,12 @@
   }
 
   @Override
-  public Spans getSpans(final IndexReader reader) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context) throws IOException {
     return new Spans() {
-        private Spans includeSpans = include.getSpans(reader);
+        private Spans includeSpans = include.getSpans(context);
         private boolean moreInclude = true;
 
-        private Spans excludeSpans = exclude.getSpans(reader);
+        private Spans excludeSpans = exclude.getSpans(context);
         private boolean moreExclude = excludeSpans.next();
 
         @Override
Index: lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanOrQuery.java	(working copy)
@@ -26,6 +26,7 @@
 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.util.PriorityQueue;
 import org.apache.lucene.util.ToStringUtils;
@@ -162,9 +163,9 @@
   }
 
   @Override
-  public Spans getSpans(final IndexReader reader) throws IOException {
+  public Spans getSpans(final AtomicReaderContext context) throws IOException {
     if (clauses.size() == 1)                      // optimize 1-clause case
-      return (clauses.get(0)).getSpans(reader);
+      return (clauses.get(0)).getSpans(context);
 
     return new Spans() {
         private SpanQueue queue = null;
@@ -173,7 +174,7 @@
           queue = new SpanQueue(clauses.size());
           Iterator<SpanQuery> i = clauses.iterator();
           while (i.hasNext()) {
-            Spans spans = i.next().getSpans(reader);
+            Spans spans = i.next().getSpans(context);
             if (   ((target == -1) && spans.next())
                 || ((target != -1) && spans.skipTo(target))) {
               queue.add(spans);
Index: lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanPositionCheckQuery.java	(working copy)
@@ -18,6 +18,7 @@
 
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
 
@@ -80,8 +81,8 @@
   protected abstract AcceptStatus acceptPosition(Spans spans) throws IOException;
 
   @Override
-  public Spans getSpans(final IndexReader reader) throws IOException {
-    return new PositionCheckSpan(reader);
+  public Spans getSpans(final AtomicReaderContext context) throws IOException {
+    return new PositionCheckSpan(context);
   }
 
 
@@ -105,8 +106,8 @@
   protected class PositionCheckSpan extends Spans {
     private Spans spans;
 
-    public PositionCheckSpan(IndexReader reader) throws IOException {
-      spans = match.getSpans(reader);
+    public PositionCheckSpan(AtomicReaderContext context) throws IOException {
+      spans = match.getSpans(context);
     }
 
     @Override
Index: lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanQuery.java	(working copy)
@@ -19,7 +19,7 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Weight;
@@ -28,7 +28,7 @@
 public abstract class SpanQuery extends Query {
   /** Expert: Returns the matches for this query in an index.  Used internally
    * to search for spans. */
-  public abstract Spans getSpans(IndexReader reader) throws IOException;
+  public abstract Spans getSpans(AtomicReaderContext context) throws IOException;
 
   /** Returns the name of the field matched by this query.*/
   public abstract String getField();
Index: lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanTermQuery.java	(working copy)
@@ -17,21 +17,42 @@
  * limitations under the License.
  */
 
-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.Term;
 import org.apache.lucene.index.DocsAndPositionsEnum;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Positions;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.search.Weight.ScorerContext;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ToStringUtils;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Set;
 
 /** Matches spans containing a term. */
 public class SpanTermQuery extends SpanQuery {
   protected Term term;
-
+  private TermQuery termQuery;
+  private final boolean needsPayloads;
   /** 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) {
+    this.term = term;
+    this.termQuery = query;
+    this.needsPayloads = needsPayloads;
+  }
+  
   /** Return the term whose spans are matched. */
   public Term getTerm() { return term; }
 
@@ -78,23 +99,157 @@
       return false;
     return true;
   }
-
+  
   @Override
-  public Spans getSpans(final IndexReader reader) throws IOException {
-    final DocsAndPositionsEnum postings = reader.termPositionsEnum(reader.getDeletedDocs(),
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
+    return new SpanWeight(this, searcher);
+  }
+  
+  
+ 
+  @Override
+  public Spans getSpans(final AtomicReaderContext context) throws IOException {
+    final DocsAndPositionsEnum postings = context.reader.termPositionsEnum(context.reader.getDeletedDocs(),
                                                                    term.field(),
                                                                    term.bytes());
-
+    ReaderContext topReaderContext = context.reader.getTopReaderContext();
+    Weight weight = termQuery.weight(new IndexSearcher(topReaderContext));
+    
     if (postings != null) {
-      return new TermSpans(postings, term);
+      Scorer scorer = weight.scorer((AtomicReaderContext) topReaderContext, ScorerContext.def().needsPositions(true).needsPayloads(needsPayloads));
+      return new SpansScorerWrapper(scorer, scorer.positions());
     } else {
-      if (reader.termDocsEnum(reader.getDeletedDocs(), term.field(), term.bytes()) != null) {
+      if (context.reader.termPositionsEnum(context.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;
+        return EMPTY_SPANS;
+      }
+    }
+  }
+  
+  
+  //public due to PayloadTermQuery
+  public static class SpansScorerWrapper extends Spans {
+    private final Scorer scorer;
+    private final Positions positions;
+    // only load each payload once
+    private boolean loadPayload = true;
+    private int doc = -1;
+    public SpansScorerWrapper(Scorer scorer, Positions positions) {
+      this.scorer = scorer;
+      this.positions = positions;
+    }
+    @Override
+    public boolean next() throws IOException {
+      loadPayload = true;
+      if(doc == -1) {
+        doc = scorer.nextDoc();
+      }
+      
+      if (doc == Scorer.NO_MORE_DOCS) {
+        return false;
+      }
+      
+      if(!positions.next()) {
+        doc = scorer.nextDoc();
+        if (doc == Scorer.NO_MORE_DOCS) {
+          return false;
+        }
+        return positions.next();
+      } 
+      return true;
+    }
+
+    @Override
+    public boolean skipTo(int target) throws IOException {
+      loadPayload = true;
+      doc = scorer.advance(target);
+      if(doc == Scorer.NO_MORE_DOCS) {
+        return false;
+      }
+      return positions.next();
+    }
+
+    @Override
+    public int doc() {
+      return doc;
+    }
+
+    @Override
+    public int start() {
+      return positions.start;
+    }
+
+    @Override
+    public int end() {
+      assert positions.start == positions.end-1;
+      return positions.end;
+    }
+
+    @Override
+    public Collection<byte[]> getPayload() throws IOException {
+      loadPayload = false;
+      BytesRef payload = positions.payload;
+      if (payload.length == 0) {
+        return Collections.emptyList();
       }
+      byte[] retVal = new byte[payload.length];
+      System.arraycopy(payload.bytes, payload.offset, retVal, 0, payload.length);
+      return Collections.singletonList(retVal);
+    }
+
+    @Override
+    public boolean isPayloadAvailable() {
+      return positions.hasPayloads && loadPayload;
+    }
+    
+    public Positions getPositions() {
+      return positions;
     }
   }
+  
+  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/SpanWeight.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/SpanWeight.java	(working copy)
@@ -73,7 +73,7 @@
 
   @Override
   public Scorer scorer(AtomicReaderContext context, ScorerContext scorerContext) throws IOException {
-    return new SpanScorer(query.getSpans(context.reader), this, similarity, context.reader
+    return new SpanScorer(query.getSpans(context), this, similarity, context.reader
         .norms(query.getField()));
   }
 
Index: lucene/src/java/org/apache/lucene/search/spans/Spans.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/Spans.java	(revision 1061682)
+++ lucene/src/java/org/apache/lucene/search/spans/Spans.java	(working copy)
@@ -20,6 +20,8 @@
 import java.io.IOException;
 import java.util.Collection;
 
+import org.apache.lucene.search.DocIdSetIterator;
+
 /** Expert: an enumeration of span matches.  Used to implement span searching.
  * Each span represents a range of term positions within a document.  Matches
  * are enumerated in order, by increasing document number, within that by
@@ -83,4 +85,5 @@
    * @return true if there is a payload available at this position that can be loaded
    */
   public abstract boolean isPayloadAvailable();
+  
 }
Index: lucene/src/java/org/apache/lucene/search/spans/TermSpans.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/spans/TermSpans.java	(revision 1061682)
+++ 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/JustCompileSearch.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/JustCompileSearch.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/JustCompileSearch.java	(working copy)
@@ -19,7 +19,6 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.FieldInvertState;
@@ -278,7 +277,7 @@
   static final class JustCompileSpanFilter extends SpanFilter {
 
     @Override
-    public SpanFilterResult bitSpans(IndexReader reader) throws IOException {
+    public SpanFilterResult bitSpans(AtomicReaderContext context) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
     
Index: lucene/src/test/org/apache/lucene/search/TestPositionIncrement.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestPositionIncrement.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/TestPositionIncrement.java	(working copy)
@@ -40,11 +40,13 @@
 import org.apache.lucene.queryParser.QueryParser;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.search.payloads.PayloadSpanUtil;
+import org.apache.lucene.search.spans.MultiSpansWrapper;
 import org.apache.lucene.search.spans.SpanNearQuery;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.search.spans.Spans;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.util.automaton.CharacterRunAutomaton;
 import org.apache.lucene.util.automaton.RegExp;
 import org.apache.lucene.util.BytesRef;
@@ -243,7 +245,8 @@
         "a a b c d e a f g h i j a b k k")));
     writer.addDocument(doc);
 
-    IndexReader r = new SlowMultiReaderWrapper(writer.getReader());
+    IndexReader readerFromWriter = writer.getReader();
+    SlowMultiReaderWrapper r = new SlowMultiReaderWrapper(readerFromWriter);
 
     DocsAndPositionsEnum tp = r.termPositionsEnum(r.getDeletedDocs(),
                                                      "content",
@@ -262,7 +265,7 @@
     // only one doc has "a"
     assertEquals(tp.NO_MORE_DOCS, tp.nextDoc());
 
-    IndexSearcher is = new IndexSearcher(r);
+    IndexSearcher is = new IndexSearcher(readerFromWriter);
   
     SpanTermQuery stq1 = new SpanTermQuery(new Term("content", "a"));
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
@@ -274,7 +277,7 @@
     if (VERBOSE) {
       System.out.println("\ngetPayloadSpans test");
     }
-    Spans pspans = snq.getSpans(is.getIndexReader());
+    Spans pspans = snq.getSpans(ReaderUtil.leaves(is.getTopReaderContext())[0]);//MultiSpansWrapper.wrap(is.getTopReaderContext(), snq);
     while (pspans.next()) {
       if (VERBOSE) {
         System.out.println("doc " + pspans.doc() + ": span " + pspans.start()
@@ -289,11 +292,11 @@
         }
       }
     }
-    assertEquals(5, count);
     assertTrue(sawZero);
+    assertEquals(5, count);
 
     // System.out.println("\ngetSpans test");
-    Spans spans = snq.getSpans(is.getIndexReader());
+    Spans spans = MultiSpansWrapper.wrap(is.getTopReaderContext(), snq);
     count = 0;
     sawZero = false;
     while (spans.next()) {
@@ -308,7 +311,7 @@
     // System.out.println("\nPayloadSpanUtil test");
 
     sawZero = false;
-    PayloadSpanUtil psu = new PayloadSpanUtil(is.getIndexReader());
+    PayloadSpanUtil psu = new PayloadSpanUtil(is.getTopReaderContext());
     Collection<byte[]> pls = psu.getPayloadsForQuery(snq);
     count = pls.size();
     for (byte[] bytes : pls) {
Index: lucene/src/test/org/apache/lucene/search/TestSpanQueryFilter.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestSpanQueryFilter.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/TestSpanQueryFilter.java	(working copy)
@@ -21,13 +21,14 @@
 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.RandomIndexWriter;
-import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.spans.SpanTermQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.English;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ReaderUtil;
 
 public class TestSpanQueryFilter extends LuceneTestCase {
 
@@ -40,12 +41,15 @@
               Field.Store.NO, Field.Index.ANALYZED));
       writer.addDocument(document);
     }
+    writer.optimize(); // nocommit BulkPostins don't work with Multi so far!
+    writer.commit(); 
     IndexReader reader = writer.getReader();
     writer.close();
-
-    SpanTermQuery query = new SpanTermQuery(new Term("field", English.intToEnglish(10).trim()));
+    AtomicReaderContext[] atomicCtx = ReaderUtil.leaves(reader.getTopReaderContext());
+    assertEquals(1, atomicCtx.length);
+    SpanTermQuery query = new SpanTermQuery(new Term("field", English.intToEnglish(10).trim()), false);
     SpanQueryFilter filter = new SpanQueryFilter(query);
-    SpanFilterResult result = filter.bitSpans(new SlowMultiReaderWrapper(reader));
+    SpanFilterResult result = filter.bitSpans(atomicCtx[0]);
     DocIdSet docIdSet = result.getDocIdSet();
     assertTrue("docIdSet is null and it shouldn't be", docIdSet != null);
     assertContainsDocId("docIdSet doesn't contain docId 10", docIdSet, 10);
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,289 @@
+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.Weight.ScorerContext;
+import org.apache.lucene.store.Directory;
+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));
+        Positions positions = scorer.positions();
+        
+        do {
+          String msg = "Advanced to: " + advance + " current doc: "
+              + scorer.docID() + " usePayloads: " + usePayload;
+          assertTrue(msg, positions.next());
+          assertEquals(msg, 4, positions.freq);
+
+          assertEquals(msg, 0, positions.start);
+          assertEquals(msg, 1, positions.end);
+          checkPayload(0, positions);
+
+          assertTrue(msg, positions.next());
+          assertEquals(msg, 4, positions.freq);
+          assertEquals(msg, 10, positions.start);
+          assertEquals(msg, 11, positions.end);
+          checkPayload(10, positions);
+          
+          assertTrue(msg, positions.next());
+          assertEquals(msg, 4, positions.freq);
+
+          assertEquals(msg, 20, positions.start);
+          assertEquals(msg, 21, positions.end);
+          checkPayload(20, positions);
+          
+          assertTrue(msg, positions.next());
+          assertEquals(msg, 4, positions.freq);
+          assertEquals(msg, 30, positions.start);
+          assertEquals(msg, 31, positions.end);
+          checkPayload(30, positions);
+          
+          assertFalse(msg, positions.next());
+
+        } while (scorer.nextDoc() != Scorer.NO_MORE_DOCS);
+      }
+    }
+    searcher.close();
+    reader.close();
+    directory.close();
+  }
+  
+  public final void checkPayload(int pos, Positions positions) {
+    if(!CodecProvider.getDefault().getFieldCodec(fieldName).equals("Standard"))
+      return; // nocommit remove this once all codecs work with bulk payloads 
+    if (usePayload) {
+      assertTrue(positions.hasPayloads);
+      assertEquals("pos: "+ pos, positions.payload.utf8ToString());
+    } else {
+      assertFalse(positions.hasPayloads);
+    }
+    
+    
+  }
+
+  /**
+   * 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;
+          }
+          Positions positions = scorer.positions();
+          Integer[] pos = positionsInDoc[atomicReaderContext.docBase + docID];
+          assertEquals(pos.length, positions.freq);
+          // 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;
+          for (int j = 0; j < howMany; j++) {
+            assertTrue(positions.next());
+            assertEquals("iteration: " + i + " initDoc: " + initDoc + " doc: "
+                + docID + " base: " + atomicReaderContext.docBase
+                + " positions: " + Arrays.toString(pos) + " usePayloads: " + usePayload, pos[j].intValue(),
+                positions.start);
+            assertEquals(pos[j].intValue() + 1, positions.end);
+          }
+          if (howMany == pos.length) {
+            assertFalse(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);
+      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;
+        Positions positions = scorer.positions();
+        assertEquals(howMany / 2, positions.freq);
+        for (int j = 0; j < howMany; j += 2) {
+          assertTrue("next returned false at index: "+j + " with freq: " + positions.freq + " -- "+  msg, positions.next());
+          assertEquals("position missmatch index: "+j + " with freq: " + positions.freq + " -- "+  msg, j, positions.start);
+        }
+        assertFalse("next returned true -- "+  msg, positions.next());
+      }
+    }
+    reader.close();
+    dir.close();
+  }
+
+}
Index: lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/payloads/TestPayloadNearQuery.java	(working copy)
@@ -301,7 +301,7 @@
 
     @Override public float scorePayload(int docId, String fieldName, int start, int end, byte[] payload, int offset, int length) {
       //we know it is size 4 here, so ignore the offset/length
-      return payload[0];
+      return payload[offset];
     }
     //!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
     //Make everything else 1 so we see the effect of the payload
Index: lucene/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/payloads/TestPayloadTermQuery.java	(working copy)
@@ -18,6 +18,7 @@
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.English;
+import org.apache.lucene.util.ReaderUtil;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.QueryUtils;
 import org.apache.lucene.search.TopDocs;
@@ -26,9 +27,10 @@
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DefaultSimilarity;
+import org.apache.lucene.search.spans.MultiSpansWrapper;
 import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.search.spans.SpanTermQuery.SpansScorerWrapper;
 import org.apache.lucene.search.spans.Spans;
-import org.apache.lucene.search.spans.TermSpans;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.analysis.TokenStream;
@@ -36,6 +38,7 @@
 import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.Payload;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
@@ -122,7 +125,7 @@
       doc.add(newField("multiField", English.intToEnglish(i) + "  " + English.intToEnglish(i), Field.Store.YES, Field.Index.ANALYZED));
       writer.addDocument(doc);
     }
-    reader = new SlowMultiReaderWrapper(writer.getReader());
+    reader = writer.getReader();
     writer.close();
 
     searcher = new IndexSearcher(reader);
@@ -153,9 +156,8 @@
       assertTrue(doc.score + " does not equal: " + 1, doc.score == 1);
     }
     CheckHits.checkExplanations(query, PayloadHelper.FIELD, searcher, true);
-    Spans spans = query.getSpans(searcher.getIndexReader());
+    Spans spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), query);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    assertTrue("spans is not an instanceof " + TermSpans.class, spans instanceof TermSpans);
     /*float score = hits.score(0);
     for (int i =1; i < hits.length(); i++)
     {
@@ -205,9 +207,8 @@
     }
     assertTrue(numTens + " does not equal: " + 10, numTens == 10);
     CheckHits.checkExplanations(query, "field", searcher, true);
-    Spans spans = query.getSpans(searcher.getIndexReader());
+    Spans spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), query);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    assertTrue("spans is not an instanceof " + TermSpans.class, spans instanceof TermSpans);
     //should be two matches per document
     int count = 0;
     //100 hits times 2 matches per hit, we should have 200 in count
@@ -247,9 +248,8 @@
     }
     assertTrue(numTens + " does not equal: " + 10, numTens == 10);
     CheckHits.checkExplanations(query, "field", searcher, true);
-    Spans spans = query.getSpans(searcher.getIndexReader());
+    Spans spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), query);
     assertTrue("spans is null and it shouldn't be", spans != null);
-    assertTrue("spans is not an instanceof " + TermSpans.class, spans instanceof TermSpans);
     //should be two matches per document
     int count = 0;
     //100 hits times 2 matches per hit, we should have 200 in count
@@ -293,7 +293,7 @@
     @Override
     public float scorePayload(int docId, String fieldName, int start, int end, byte[] payload, int offset, int length) {
       //we know it is size 4 here, so ignore the offset/length
-      return payload[0];
+      return payload[offset];
     }
 
     //!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
@@ -333,7 +333,7 @@
   static class FullSimilarity extends DefaultSimilarity{
     public float scorePayload(int docId, String fieldName, byte[] payload, int offset, int length) {
       //we know it is size 4 here, so ignore the offset/length
-      return payload[0];
+      return payload[offset];
     }
   }
 
Index: lucene/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/spans/JustCompileSearchSpans.java	(working copy)
@@ -20,7 +20,7 @@
 import java.io.IOException;
 import java.util.Collection;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.search.Weight;
 import org.apache.lucene.search.Similarity;
 
@@ -82,7 +82,7 @@
     }
 
     @Override
-    public Spans getSpans(IndexReader reader) throws IOException {
+    public Spans getSpans(AtomicReaderContext context) throws IOException {
       throw new UnsupportedOperationException(UNSUPPORTED_MSG);
     }
 
Index: lucene/src/test/org/apache/lucene/search/spans/TestBasics.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/TestBasics.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/spans/TestBasics.java	(working copy)
@@ -30,7 +30,6 @@
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Payload;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.SlowMultiReaderWrapper;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -176,6 +175,27 @@
     QueryUtils.check(term2);
     QueryUtils.checkUnequal(term1,term2);
   }
+  
+  public void testSpanTermQuery() throws Exception {
+    SpanTermQuery term1 = new SpanTermQuery(new Term("field", "seventy"));
+    checkHits(term1, new int[]
+                             { 70, 71, 72, 73, 74, 75, 76, 77, 78, 79, 170,
+        171, 172, 173, 174, 175, 176, 177, 178, 179, 270, 271, 272, 273, 274,
+        275, 276, 277, 278, 279, 370, 371, 372, 373, 374, 375, 376, 377, 378,
+        379, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 570, 571, 572,
+        573, 574, 575, 576, 577, 578, 579, 670, 671, 672, 673, 674, 675, 676,
+        677, 678, 679, 770, 771, 772, 773, 774, 775, 776, 777, 778, 779, 870,
+        871, 872, 873, 874, 875, 876, 877, 878, 879, 970, 971, 972, 973, 974,
+        975, 976, 977, 978, 979, 1070, 1071, 1072, 1073, 1074, 1075, 1076,
+        1077, 1078, 1079, 1170, 1270, 1370, 1470, 1570, 1670, 1770, 1870, 1970,
+        1171, 1172, 1173, 1174, 1175, 1176, 1177, 1178, 1179, 1271, 1272, 1273,
+        1274, 1275, 1276, 1277, 1278, 1279, 1371, 1372, 1373, 1374, 1375, 1376,
+        1377, 1378, 1379, 1471, 1472, 1473, 1474, 1475, 1476, 1477, 1478, 1479,
+        1571, 1572, 1573, 1574, 1575, 1576, 1577, 1578, 1579, 1671, 1672, 1673,
+        1674, 1675, 1676, 1677, 1678, 1679, 1771, 1772, 1773, 1774, 1775, 1776,
+        1777, 1778, 1779, 1871, 1872, 1873, 1874, 1875, 1876, 1877, 1878, 1879,
+        1971, 1972, 1973, 1974, 1975, 1976, 1977, 1978, 1979 });
+  }
 
   @Test
   public void testSpanNearUnordered() throws Exception {
@@ -522,8 +542,8 @@
   public void testSpansSkipTo() throws Exception {
 	  SpanTermQuery t1 = new SpanTermQuery(new Term("field", "seventy"));
 	  SpanTermQuery t2 = new SpanTermQuery(new Term("field", "seventy"));
-	  Spans s1 = t1.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
-	  Spans s2 = t2.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+	  Spans s1 = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), t1);
+	  Spans s2 = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), t2);
 	  
 	  assertTrue(s1.next());
 	  assertTrue(s2.next());
Index: lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/spans/TestFieldMaskingSpanQuery.java	(working copy)
@@ -254,7 +254,7 @@
     SpanQuery q  = new SpanOrQuery(q1, new FieldMaskingSpanQuery(q2, "gender"));
     check(q, new int[] { 0, 1, 2, 3, 4 });
   
-    Spans span = q.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     
     assertEquals(true, span.next());
     assertEquals(s(0,0,1), s(span));
@@ -295,8 +295,8 @@
     check(qA, new int[] { 0, 1, 2, 4 });
     check(qB, new int[] { 0, 1, 2, 4 });
   
-    Spans spanA = qA.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
-    Spans spanB = qB.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    Spans spanA = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), qA);
+    Spans spanB = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), qB);
     
     while (spanA.next()) {
       assertTrue("spanB not still going", spanB.next());
@@ -316,7 +316,7 @@
         new FieldMaskingSpanQuery(qB, "id") }, -1, false );
     check(q, new int[] { 0, 1, 2, 3 });
   
-    Spans span = q.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     
     assertEquals(true, span.next());
     assertEquals(s(0,0,1), s(span));
Index: lucene/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/spans/TestNearSpansOrdered.java	(working copy)
@@ -22,7 +22,7 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
-import org.apache.lucene.index.SlowMultiReaderWrapper;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.RandomIndexWriter;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryParser.QueryParser;
@@ -34,6 +34,7 @@
 import org.apache.lucene.search.Weight.ScorerContext;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ReaderUtil;
 
 public class TestNearSpansOrdered extends LuceneTestCase {
   protected IndexSearcher searcher;
@@ -62,7 +63,7 @@
       doc.add(newField(FIELD, docFields[i], Field.Store.NO, Field.Index.ANALYZED));
       writer.addDocument(doc);
     }
-    reader = new SlowMultiReaderWrapper(writer.getReader());
+    reader = writer.getReader();
     writer.close();
     searcher = new IndexSearcher(reader);
   }
@@ -102,7 +103,7 @@
   
   public void testNearSpansNext() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(true, span.next());
     assertEquals(s(0,0,3), s(span));
     assertEquals(true, span.next());
@@ -117,7 +118,7 @@
    */
   public void testNearSpansSkipToLikeNext() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(true, span.skipTo(0));
     assertEquals(s(0,0,3), s(span));
     assertEquals(true, span.skipTo(1));
@@ -127,7 +128,7 @@
   
   public void testNearSpansNextThenSkipTo() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(true, span.next());
     assertEquals(s(0,0,3), s(span));
     assertEquals(true, span.skipTo(1));
@@ -137,7 +138,7 @@
   
   public void testNearSpansNextThenSkipPast() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(true, span.next());
     assertEquals(s(0,0,3), s(span));
     assertEquals(false, span.skipTo(2));
@@ -145,20 +146,20 @@
   
   public void testNearSpansSkipPast() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(false, span.skipTo(2));
   }
   
   public void testNearSpansSkipTo0() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(true, span.skipTo(0));
     assertEquals(s(0,0,3), s(span));
   }
 
   public void testNearSpansSkipTo1() throws Exception {
     SpanNearQuery q = makeQuery();
-    Spans span = q.getSpans(searcher.getIndexReader());
+    Spans span =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), q);
     assertEquals(true, span.skipTo(1));
     assertEquals(s(1,0,4), s(span));
   }
@@ -170,8 +171,9 @@
   public void testSpanNearScorerSkipTo1() throws Exception {
     SpanNearQuery q = makeQuery();
     Weight w = q.weight(searcher);
-    assertTrue(searcher.getTopReaderContext().isAtomic);
-    Scorer s = w.scorer((AtomicReaderContext) searcher.getTopReaderContext(), ScorerContext.def());
+    ReaderContext topReaderContext = searcher.getTopReaderContext();
+    AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+    Scorer s = w.scorer(leaves[0], ScorerContext.def());
     assertEquals(1, s.advance(1));
   }
   /**
@@ -180,8 +182,10 @@
    */
   public void testSpanNearScorerExplain() throws Exception {
     SpanNearQuery q = makeQuery();
-    assertTrue(searcher.getTopReaderContext().isAtomic);
-    Explanation e = q.weight(searcher).explain((AtomicReaderContext) searcher.getTopReaderContext(), 1);
+    ReaderContext topReaderContext = searcher.getTopReaderContext();
+    AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+
+    Explanation e = q.weight(searcher).explain(leaves[0], 1);
     assertTrue("Scorer explanation value for doc#1 isn't positive: "
                + e.toString(),
                0.0f < e.getValue());
Index: lucene/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/spans/TestPayloadSpans.java	(working copy)
@@ -68,12 +68,12 @@
     SpanTermQuery stq;
     Spans spans;
     stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "seventy"));
-    spans = stq.getSpans(indexReader);
+    spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), stq);
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 100, 1, 1, 1);
 
     stq = new SpanTermQuery(new Term(PayloadHelper.NO_PAYLOAD_FIELD, "seventy"));  
-    spans = stq.getSpans(indexReader);
+    spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), stq);
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 100, 0, 0, 0);
   }
@@ -84,7 +84,7 @@
     SpanFirstQuery sfq;
     match = new SpanTermQuery(new Term(PayloadHelper.FIELD, "one"));
     sfq = new SpanFirstQuery(match, 2);
-    Spans spans = sfq.getSpans(indexReader);
+    Spans spans = MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq);
     checkSpans(spans, 109, 1, 1, 1);
     //Test more complicated subclause
     SpanQuery[] clauses = new SpanQuery[2];
@@ -92,11 +92,11 @@
     clauses[1] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "hundred"));
     match = new SpanNearQuery(clauses, 0, true);
     sfq = new SpanFirstQuery(match, 2);
-    checkSpans(sfq.getSpans(indexReader), 100, 2, 1, 1);
+    checkSpans(MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq), 100, 2, 1, 1);
 
     match = new SpanNearQuery(clauses, 0, false);
     sfq = new SpanFirstQuery(match, 2);
-    checkSpans(sfq.getSpans(indexReader), 100, 2, 1, 1);
+    checkSpans(MultiSpansWrapper.wrap(indexReader.getTopReaderContext(), sfq), 100, 2, 1, 1);
     
   }
   
@@ -119,8 +119,9 @@
     writer.addDocument(doc);
     IndexReader reader = writer.getReader();
     writer.close();
+    
 
-    checkSpans(snq.getSpans(new SlowMultiReaderWrapper(reader)), 1,new int[]{2});
+    checkSpans(MultiSpansWrapper.wrap(reader.getTopReaderContext(), snq), 1,new int[]{2});
     reader.close();
     directory.close();
   }
@@ -130,7 +131,7 @@
     Spans spans;
     IndexSearcher searcher = getSearcher();
     stq = new SpanTermQuery(new Term(PayloadHelper.FIELD, "mark"));
-    spans = stq.getSpans(searcher.getIndexReader());
+    spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), stq);
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 0, null);
 
@@ -141,7 +142,7 @@
     clauses[2] = new SpanTermQuery(new Term(PayloadHelper.FIELD, "xx"));
     SpanNearQuery spanNearQuery = new SpanNearQuery(clauses, 12, false);
 
-    spans = spanNearQuery.getSpans(searcher.getIndexReader());
+    spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), spanNearQuery);
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 2, new int[]{3,3});
 
@@ -152,8 +153,8 @@
 
     spanNearQuery = new SpanNearQuery(clauses, 6, true);
    
-    
-    spans = spanNearQuery.getSpans(searcher.getIndexReader());
+    spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), spanNearQuery);
+
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 1, new int[]{3});
      
@@ -175,7 +176,7 @@
     
     // yy within 6 of xx within 6 of rr
 
-    spans = nestedSpanNearQuery.getSpans(searcher.getIndexReader());
+    spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), nestedSpanNearQuery);
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 2, new int[]{3,3});
     closeIndexReader.close();
@@ -206,8 +207,8 @@
     clauses3[1] = snq;
      
     SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses3, 6, false);
+    spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), nestedSpanNearQuery);
 
-    spans = nestedSpanNearQuery.getSpans(searcher.getIndexReader());
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 1, new int[]{3});
     closeIndexReader.close();
@@ -244,7 +245,7 @@
      
     SpanNearQuery nestedSpanNearQuery = new SpanNearQuery(clauses3, 6, false);
 
-    spans = nestedSpanNearQuery.getSpans(searcher.getIndexReader());
+    spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), nestedSpanNearQuery);
     assertTrue("spans is null and it shouldn't be", spans != null);
     checkSpans(spans, 2, new int[]{8, 8});
     closeIndexReader.close();
@@ -269,7 +270,7 @@
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
     SpanQuery[] sqs = { stq1, stq2 };
     SpanNearQuery snq = new SpanNearQuery(sqs, 1, true);
-    Spans spans = snq.getSpans(is.getIndexReader());
+    Spans spans = MultiSpansWrapper.wrap(is.getTopReaderContext(), snq);
 
     TopDocs topDocs = is.search(snq, 1);
     Set<String> payloadSet = new HashSet<String>();
@@ -306,7 +307,7 @@
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
     SpanQuery[] sqs = { stq1, stq2 };
     SpanNearQuery snq = new SpanNearQuery(sqs, 0, true);
-    Spans spans = snq.getSpans(is.getIndexReader());
+    Spans spans =  MultiSpansWrapper.wrap(is.getTopReaderContext(), snq);
 
     TopDocs topDocs = is.search(snq, 1);
     Set<String> payloadSet = new HashSet<String>();
@@ -342,7 +343,7 @@
     SpanTermQuery stq2 = new SpanTermQuery(new Term("content", "k"));
     SpanQuery[] sqs = { stq1, stq2 };
     SpanNearQuery snq = new SpanNearQuery(sqs, 0, true);
-    Spans spans = snq.getSpans(is.getIndexReader());
+    Spans spans =  MultiSpansWrapper.wrap(is.getTopReaderContext(), snq);
 
     TopDocs topDocs = is.search(snq, 1);
     Set<String> payloadSet = new HashSet<String>();
@@ -376,11 +377,11 @@
     doc.add(newField(PayloadHelper.FIELD,"xx rr yy mm  pp", Field.Store.YES, Field.Index.ANALYZED));
     writer.addDocument(doc);
   
-    IndexReader reader = new SlowMultiReaderWrapper(writer.getReader());
+    IndexReader reader = writer.getReader();
     writer.close();
     IndexSearcher searcher = new IndexSearcher(reader);
 
-    PayloadSpanUtil psu = new PayloadSpanUtil(searcher.getIndexReader());
+    PayloadSpanUtil psu = new PayloadSpanUtil(searcher.getTopReaderContext());
     
     Collection<byte[]> payloads = psu.getPayloadsForQuery(new TermQuery(new Term(PayloadHelper.FIELD, "rr")));
     if(VERBOSE)
Index: lucene/src/test/org/apache/lucene/search/spans/TestSpans.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/spans/TestSpans.java	(revision 1061682)
+++ lucene/src/test/org/apache/lucene/search/spans/TestSpans.java	(working copy)
@@ -28,6 +28,7 @@
 import org.apache.lucene.search.Weight.ScorerContext;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexReader.AtomicReaderContext;
@@ -38,6 +39,8 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.ReaderUtil;
+
 import java.io.IOException;
 
 public class TestSpans extends LuceneTestCase {
@@ -197,7 +200,7 @@
                                 makeSpanTermQuery("t3") },
                               slop,
                               ordered);
-    Spans spans = snq.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    Spans spans = MultiSpansWrapper.wrap(searcher.getTopReaderContext(), snq);
 
     assertTrue("first range", spans.next());
     assertEquals("first doc", 11, spans.doc());
@@ -223,7 +226,7 @@
                                 makeSpanTermQuery("u2") },
                               0,
                               false);
-    Spans spans = snq.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    Spans spans =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), snq);
     assertTrue("Does not have next and it should", spans.next());
     assertEquals("doc", 4, spans.doc());
     assertEquals("start", 1, spans.start());
@@ -259,7 +262,7 @@
                               },
                               1,
                               false);
-    spans = snq.getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    spans =  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), snq);
     assertTrue("Does not have next and it should", spans.next());
     assertEquals("doc", 4, spans.doc());
     assertEquals("start", 0, spans.start());
@@ -317,7 +320,7 @@
     for (int i = 0; i < terms.length; i++) {
       sqa[i] = makeSpanTermQuery(terms[i]);
     }
-    return (new SpanOrQuery(sqa)).getSpans(new SlowMultiReaderWrapper(searcher.getIndexReader()));
+    return  MultiSpansWrapper.wrap(searcher.getTopReaderContext(), new SpanOrQuery(sqa));
   }
 
   private void tstNextSpans(Spans spans, int doc, int start, int end)
@@ -402,34 +405,43 @@
   public void testSpanScorerZeroSloppyFreq() throws Exception {
     boolean ordered = true;
     int slop = 1;
-
-    final Similarity sim = new DefaultSimilarity() {
-      @Override
-      public float sloppyFreq(int distance) {
-        return 0.0f;
+    ReaderContext topReaderContext = searcher.getTopReaderContext();
+    AtomicReaderContext[] leaves = ReaderUtil.leaves(topReaderContext);
+    int subIndex = ReaderUtil.subIndex(11, leaves);
+    for (int i = 0; i < leaves.length; i++) {
+      
+     
+      final Similarity sim = new DefaultSimilarity() {
+        @Override
+        public float sloppyFreq(int distance) {
+          return 0.0f;
+        }
+      };
+  
+      final Similarity oldSim = searcher.getSimilarity();
+      Scorer spanScorer;
+      try {
+        searcher.setSimilarity(sim);
+        SpanNearQuery snq = new SpanNearQuery(
+                                new SpanQuery[] {
+                                  makeSpanTermQuery("t1"),
+                                  makeSpanTermQuery("t2") },
+                                slop,
+                                ordered);
+  
+        spanScorer = snq.weight(searcher).scorer(leaves[i], ScorerContext.def());
+      } finally {
+        searcher.setSimilarity(oldSim);
       }
-    };
-
-    final Similarity oldSim = searcher.getSimilarity();
-    Scorer spanScorer;
-    try {
-      searcher.setSimilarity(sim);
-      SpanNearQuery snq = new SpanNearQuery(
-                              new SpanQuery[] {
-                                makeSpanTermQuery("t1"),
-                                makeSpanTermQuery("t2") },
-                              slop,
-                              ordered);
-
-      spanScorer = snq.weight(searcher).scorer(new AtomicReaderContext(new SlowMultiReaderWrapper(searcher.getIndexReader())), ScorerContext.def());
-    } finally {
-      searcher.setSimilarity(oldSim);
+      if (i == subIndex) {
+        assertTrue("first doc", spanScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
+        assertEquals("first doc number", spanScorer.docID() + leaves[i].docBase, 11);
+        float score = spanScorer.score();
+        assertTrue("first doc score should be zero, " + score, score == 0.0f);
+      }  else {
+        assertTrue("no second doc", spanScorer.nextDoc() == DocIdSetIterator.NO_MORE_DOCS);
+      }
     }
-    assertTrue("first doc", spanScorer.nextDoc() != DocIdSetIterator.NO_MORE_DOCS);
-    assertEquals("first doc number", spanScorer.docID(), 11);
-    float score = spanScorer.score();
-    assertTrue("first doc score should be zero, " + score, score == 0.0f);
-    assertTrue("no second doc", spanScorer.nextDoc() == DocIdSetIterator.NO_MORE_DOCS);
   }
 
   // LUCENE-1404
