Index: modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java
===================================================================
--- modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java	(revision 1171698)
+++ modules/grouping/src/test/org/apache/lucene/search/grouping/TestGrouping.java	(working copy)
@@ -1034,7 +1034,7 @@
     }
 
     public TopDocs search(Weight weight, int topN) throws IOException {
-      return search(ctx, weight, null, topN);
+      return search(ctx, weight, null, null, topN);
     }
 
     @Override
Index: lucene/src/test/org/apache/lucene/search/TestPagedSearch.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestPagedSearch.java	(revision 0)
+++ lucene/src/test/org/apache/lucene/search/TestPagedSearch.java	(revision 0)
@@ -0,0 +1,95 @@
+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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util._TestUtil;
+
+/**
+ * Tests IndexSearcher's pagedSearch() method
+ */
+public class TestPagedSearch extends LuceneTestCase {
+  private Directory dir;
+  private IndexReader reader;
+  private IndexSearcher searcher;
+   
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random, dir);
+    int numDocs = atLeast(200);
+    for (int i = 0; i < numDocs; i++) {
+      Document document = new Document();
+      document.add(newField("english", English.intToEnglish(i), StringField.TYPE_UNSTORED));
+      document.add(newField("oddeven", (i % 2 == 0) ? "even" : "odd", StringField.TYPE_UNSTORED));
+      iw.addDocument(document);
+    }
+    reader = iw.getReader();
+    iw.close();
+    searcher = newSearcher(reader);
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    searcher.close();
+    reader.close();
+    dir.close();
+    super.tearDown();
+  }
+  
+  public void testQueries() throws Exception {
+    Filter odd = new QueryWrapperFilter(new TermQuery(new Term("oddeven", "odd")));
+    assertQuery(new MatchAllDocsQuery(), null);
+    assertQuery(new TermQuery(new Term("english", "one")), null);
+    assertQuery(new MatchAllDocsQuery(), odd);
+    assertQuery(new TermQuery(new Term("english", "four")), odd);
+  }
+  
+  void assertQuery(Query query, Filter filter) throws Exception {
+    TopDocs all = searcher.search(query, filter, searcher.maxDoc());
+    int pageSize = _TestUtil.nextInt(random, 1, searcher.maxDoc());
+    int pageStart = 0;
+    ScoreDoc lastBottom = null;
+    while (pageStart < all.totalHits) {
+      TopDocs paged = searcher.pagedSearch(query, filter, lastBottom, pageSize);
+      if (paged.scoreDocs.length == 0) {
+        break;
+      }
+      assertPage(pageStart, all, paged);
+      pageStart += paged.scoreDocs.length;
+      lastBottom = paged.scoreDocs[paged.scoreDocs.length - 1];
+    }
+    assertEquals(all.scoreDocs.length, pageStart);
+  }
+
+  static void assertPage(int pageStart, TopDocs all, TopDocs paged) {
+    assertEquals(all.totalHits, paged.totalHits);
+    for (int i = 0; i < paged.scoreDocs.length; i++) {
+      assertEquals(all.scoreDocs[pageStart + i].doc, paged.scoreDocs[i].doc);
+      assertEquals(all.scoreDocs[pageStart + i].score, paged.scoreDocs[i].score, 0f);
+    }
+  }
+}

Property changes on: lucene/src/test/org/apache/lucene/search/TestPagedSearch.java
___________________________________________________________________
Added: svn:eol-style
   + native

Index: lucene/src/test/org/apache/lucene/search/TestTopDocsMerge.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestTopDocsMerge.java	(revision 1171698)
+++ lucene/src/test/org/apache/lucene/search/TestTopDocsMerge.java	(working copy)
@@ -48,7 +48,7 @@
     }
 
     public TopDocs search(Weight weight, int topN) throws IOException {
-      return search(ctx, weight, null, topN);
+      return search(ctx, weight, null, null, topN);
     }
 
     @Override
Index: lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java	(revision 1171698)
+++ lucene/src/java/org/apache/lucene/search/TopScoreDocCollector.java	(working copy)
@@ -67,7 +67,68 @@
       return false;
     }
   }
+  
+  // Assumes docs are scored in order.
+  private static class InOrderPagingScoreDocCollector extends TopScoreDocCollector {
+    private final ScoreDoc lastBottom;
+    // this is always lastBottom.doc - docBase, to save an add when score == lastBottom.score
+    private int lastBottomDoc;
+    private int collectedHits;
 
+    private InOrderPagingScoreDocCollector(ScoreDoc lastBottom, int numHits) {
+      super(numHits);
+      this.lastBottom = lastBottom;
+    }
+    
+    @Override
+    public void collect(int doc) throws IOException {
+      float score = scorer.score();
+
+      // This collector cannot handle these scores:
+      assert score != Float.NEGATIVE_INFINITY;
+      assert !Float.isNaN(score);
+
+      totalHits++;
+      
+      if (score > lastBottom.score || (score == lastBottom.score && doc <= lastBottomDoc)) {
+        // hit was collected on a previous page
+        return;
+      }
+      
+      if (score <= pqTop.score) {
+        // Since docs are returned in-order (i.e., increasing doc Id), a document
+        // with equal score to pqTop.score cannot compete since HitQueue favors
+        // documents with lower doc Ids. Therefore reject those docs too.
+        return;
+      }
+      collectedHits++;
+      pqTop.doc = doc + docBase;
+      pqTop.score = score;
+      pqTop = pq.updateTop();
+    }
+
+    @Override
+    public boolean acceptsDocsOutOfOrder() {
+      return false;
+    }
+
+    @Override
+    public void setNextReader(AtomicReaderContext context) {
+      super.setNextReader(context);
+      lastBottomDoc = lastBottom.doc - docBase;
+    }
+
+    @Override
+    protected int topDocsSize() {
+      return collectedHits < pq.size() ? collectedHits : pq.size();
+    }
+    
+    @Override
+    protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
+      return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
+    }
+  }
+
   // Assumes docs are scored out of order.
   private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
     private OutOfOrderTopScoreDocCollector(int numHits) {
@@ -101,7 +162,68 @@
       return true;
     }
   }
+  
+  // Assumes docs are scored out of order.
+  private static class OutOfOrderPagingScoreDocCollector extends TopScoreDocCollector {
+    private final ScoreDoc lastBottom;
+    // this is always lastBottom.doc - docBase, to save an add when score == lastBottom.score
+    private int lastBottomDoc;
+    private int collectedHits;
 
+    private OutOfOrderPagingScoreDocCollector(ScoreDoc lastBottom, int numHits) {
+      super(numHits);
+      this.lastBottom = lastBottom;
+    }
+    
+    @Override
+    public void collect(int doc) throws IOException {
+      float score = scorer.score();
+
+      // This collector cannot handle NaN
+      assert !Float.isNaN(score);
+
+      totalHits++;
+      if (score > lastBottom.score || (score == lastBottom.score && doc <= lastBottomDoc)) {
+        // hit was collected on a previous page
+        return;
+      }
+      if (score < pqTop.score) {
+        // Doesn't compete w/ bottom entry in queue
+        return;
+      }
+      doc += docBase;
+      if (score == pqTop.score && doc > pqTop.doc) {
+        // Break tie in score by doc ID:
+        return;
+      }
+      collectedHits++;
+      pqTop.doc = doc;
+      pqTop.score = score;
+      pqTop = pq.updateTop();
+    }
+    
+    @Override
+    public boolean acceptsDocsOutOfOrder() {
+      return true;
+    }
+    
+    @Override
+    public void setNextReader(AtomicReaderContext context) {
+      super.setNextReader(context);
+      lastBottomDoc = lastBottom.doc - docBase;
+    }
+    
+    @Override
+    protected int topDocsSize() {
+      return collectedHits < pq.size() ? collectedHits : pq.size();
+    }
+    
+    @Override
+    protected TopDocs newTopDocs(ScoreDoc[] results, int start) {
+      return results == null ? new TopDocs(totalHits, new ScoreDoc[0], Float.NaN) : new TopDocs(totalHits, results);
+    }
+  }
+
   /**
    * Creates a new {@link TopScoreDocCollector} given the number of hits to
    * collect and whether documents are scored in order by the input
@@ -113,15 +235,33 @@
    * objects.
    */
   public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
+    return create(numHits, null, docsScoredInOrder);
+  }
+  
+  /**
+   * Creates a new {@link TopScoreDocCollector} given the number of hits to
+   * collect, the bottom of the previous page, and whether documents are scored in order by the input
+   * {@link Scorer} to {@link #setScorer(Scorer)}.
+   *
+   * <p><b>NOTE</b>: The instances returned by this method
+   * pre-allocate a full array of length
+   * <code>numHits</code>, and fill the array with sentinel
+   * objects.
+   */
+  public static TopScoreDocCollector create(int numHits, ScoreDoc lastBottom, boolean docsScoredInOrder) {
     
     if (numHits <= 0) {
       throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
     }
-
+    
     if (docsScoredInOrder) {
-      return new InOrderTopScoreDocCollector(numHits);
+      return lastBottom == null 
+        ? new InOrderTopScoreDocCollector(numHits) 
+        : new InOrderPagingScoreDocCollector(lastBottom, numHits);
     } else {
-      return new OutOfOrderTopScoreDocCollector(numHits);
+      return lastBottom == null
+        ? new OutOfOrderTopScoreDocCollector(numHits)
+        : new OutOfOrderPagingScoreDocCollector(lastBottom, numHits);
     }
     
   }
Index: lucene/src/java/org/apache/lucene/search/TopDocsCollector.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TopDocsCollector.java	(revision 1171698)
+++ lucene/src/java/org/apache/lucene/search/TopDocsCollector.java	(working copy)
@@ -77,12 +77,20 @@
     return totalHits;
   }
   
+  /** The number of valid PQ entries */
+  protected int topDocsSize() {
+    // In case pq was populated with sentinel values, there might be less
+    // results than pq.size(). Therefore return all results until either
+    // pq.size() or totalHits.
+    return totalHits < pq.size() ? totalHits : pq.size();
+  }
+  
   /** Returns the top docs that were collected by this collector. */
   public TopDocs topDocs() {
     // In case pq was populated with sentinel values, there might be less
     // results than pq.size(). Therefore return all results until either
     // pq.size() or totalHits.
-    return topDocs(0, totalHits < pq.size() ? totalHits : pq.size());
+    return topDocs(0, topDocsSize());
   }
 
   /**
@@ -101,7 +109,7 @@
     // In case pq was populated with sentinel values, there might be less
     // results than pq.size(). Therefore return all results until either
     // pq.size() or totalHits.
-    return topDocs(start, totalHits < pq.size() ? totalHits : pq.size());
+    return topDocs(start, topDocsSize());
   }
 
   /**
@@ -123,10 +131,12 @@
     // In case pq was populated with sentinel values, there might be less
     // results than pq.size(). Therefore return all results until either
     // pq.size() or totalHits.
-    int size = totalHits < pq.size() ? totalHits : pq.size();
+    int size = topDocsSize();
 
     // Don't bother to throw an exception, just return an empty TopDocs in case
     // the parameters are invalid or out of range.
+    // TODO: shouldn't we throw IAE if apps give bad params here so they dont
+    // have sneaky silent bugs?
     if (start < 0 || start >= size || howMany <= 0) {
       return newTopDocs(null, start);
     }
Index: lucene/src/java/org/apache/lucene/search/IndexSearcher.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/IndexSearcher.java	(revision 1171698)
+++ lucene/src/java/org/apache/lucene/search/IndexSearcher.java	(working copy)
@@ -276,6 +276,26 @@
   }
 
   /** Finds the top <code>n</code>
+   * hits for <code>query</code>, applying <code>filter</code> if non-null,
+   * where all results are after the bottom of the previous page (<code>lastBottom</code>)
+   *
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public TopDocs pagedSearch(Query query, ScoreDoc lastBottom, int n) throws IOException {
+    return pagedSearch(query, null, lastBottom, n);
+  }
+  
+  /** Finds the top <code>n</code>
+   * hits for <code>query</code>, applying <code>filter</code> if non-null,
+   * where all results are after the bottom of the previous page (<code>lastBottom</code>)
+   *
+   * @throws BooleanQuery.TooManyClauses
+   */
+  public TopDocs pagedSearch(Query query, Filter filter, ScoreDoc lastBottom, int n) throws IOException {
+    return search(createNormalizedWeight(query), filter, lastBottom, n);
+  }
+  
+  /** Finds the top <code>n</code>
    * hits for <code>query</code>.
    *
    * @throws BooleanQuery.TooManyClauses
@@ -293,7 +313,7 @@
    */
   public TopDocs search(Query query, Filter filter, int n)
     throws IOException {
-    return search(createNormalizedWeight(query), filter, n);
+    return search(createNormalizedWeight(query), filter, null, n);
   }
 
   /** Lower-level search API.
@@ -371,9 +391,9 @@
    * {@link IndexSearcher#search(Query,Filter,int)} instead.
    * @throws BooleanQuery.TooManyClauses
    */
-  protected TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
+  protected TopDocs search(Weight weight, Filter filter, ScoreDoc lastBottom, int nDocs) throws IOException {
     if (executor == null) {
-      return search(leafContexts, weight, filter, nDocs);
+      return search(leafContexts, weight, filter, lastBottom, nDocs);
     } else {
       final HitQueue hq = new HitQueue(nDocs, false);
       final Lock lock = new ReentrantLock();
@@ -381,7 +401,7 @@
     
       for (int i = 0; i < leafSlices.length; i++) { // search each sub
         runner.submit(
-                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, nDocs, hq));
+                      new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, lastBottom, nDocs, hq));
       }
 
       int totalHits = 0;
@@ -408,14 +428,14 @@
    * {@link IndexSearcher#search(Query,Filter,int)} instead.
    * @throws BooleanQuery.TooManyClauses
    */
-  protected TopDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, int nDocs) throws IOException {
+  protected TopDocs search(AtomicReaderContext[] leaves, Weight weight, Filter filter, ScoreDoc lastBottom, int nDocs) throws IOException {
     // single thread
     int limit = reader.maxDoc();
     if (limit == 0) {
       limit = 1;
     }
     nDocs = Math.min(nDocs, limit);
-    TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
+    TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, lastBottom, !weight.scoresDocsOutOfOrder());
     search(leaves, weight, filter, collector);
     return collector.topDocs();
   }
@@ -704,23 +724,25 @@
     private final IndexSearcher searcher;
     private final Weight weight;
     private final Filter filter;
+    private final ScoreDoc lastBottom;
     private final int nDocs;
     private final HitQueue hq;
     private final LeafSlice slice;
 
     public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice,  Weight weight,
-        Filter filter, int nDocs, HitQueue hq) {
+        Filter filter, ScoreDoc lastBottom, int nDocs, HitQueue hq) {
       this.lock = lock;
       this.searcher = searcher;
       this.weight = weight;
       this.filter = filter;
+      this.lastBottom = lastBottom;
       this.nDocs = nDocs;
       this.hq = hq;
       this.slice = slice;
     }
 
     public TopDocs call() throws IOException {
-      final TopDocs docs = searcher.search (slice.leaves, weight, filter, nDocs);
+      final TopDocs docs = searcher.search (slice.leaves, weight, filter, lastBottom, nDocs);
       final ScoreDoc[] scoreDocs = docs.scoreDocs;
       for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
         final ScoreDoc scoreDoc = scoreDocs[j];
