Index: lucene/CHANGES.txt
===================================================================
--- lucene/CHANGES.txt	(revision 926765)
+++ lucene/CHANGES.txt	(working copy)
@@ -216,6 +216,8 @@
   convenient classes in case you want to disable segment merges by IndexWriter
   without tweaking a particular MergePolicy parameters, such as mergeFactor.
   MergeScheduler's methods are now public. (Shai Erera via Mike McCandless)
+
+* LUCENE-2215: Added a PagingCollector that is more efficient with PriorityQueue operations (Aaron McCurry, Grant Ingersoll)   
   
 Optimizations
 
Index: lucene/src/test/org/apache/lucene/search/TestPagingCollector.java
===================================================================
--- lucene/src/test/org/apache/lucene/search/TestPagingCollector.java	(revision 0)
+++ lucene/src/test/org/apache/lucene/search/TestPagingCollector.java	(revision 0)
@@ -0,0 +1,205 @@
+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.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Index;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.util.English;
+import org.apache.lucene.util.LuceneTestCaseJ4;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import static org.junit.Assert.*;
+
+import java.util.Set;
+import java.util.HashSet;
+
+/**
+ * Testing the paging collector.
+ */
+public class TestPagingCollector extends LuceneTestCaseJ4 {
+
+  protected static RAMDirectory directory;
+  private static final int NUM_DOCS = 1000;
+  protected static IndexSearcher searcher;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    directory = new RAMDirectory();
+    IndexWriterConfig config = new IndexWriterConfig(TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT));
+    IndexWriter indexWriter = new IndexWriter(directory, config);
+    for (int i = 0; i < NUM_DOCS; i++) {
+      Document document = new Document();
+      document.add(new Field("english", English.intToEnglish(i), Store.NO, Index.ANALYZED_NO_NORMS));
+      indexWriter.addDocument(document);
+    }
+    indexWriter.close();
+    searcher = new IndexSearcher(directory, true);
+  }
+
+  @AfterClass
+  public static void afterClass() throws Exception {
+    searcher.close();
+    searcher = null;
+    directory.close();
+    directory = null;
+  }
+
+  @Test
+  public void testPagingSameScore() throws Exception {
+
+    Query query = new MatchAllDocsQuery();
+    PagingCollector collector = new PagingCollector(10);
+    searcher.search(query, collector);
+    TopDocs topDocs = collector.topDocs();
+    assertEquals(topDocs.totalHits, NUM_DOCS);
+    assertEquals(topDocs.scoreDocs.length, 10);
+    int lastSeen = 0;
+    float lastScore = Float.MAX_VALUE;
+    Set<Integer> seen = new HashSet<Integer>();
+    ScoreDoc lastSD = checkPagingOrdered(topDocs, seen, lastSeen, lastScore);
+    assertEquals(lastSD.doc, 9);
+    collector = new PagingCollector(10, topDocs.scoreDocs.length,topDocs.scoreDocs[9]);
+    searcher.search(query, collector);
+    topDocs = collector.topDocs();
+    assertEquals(topDocs.totalHits, NUM_DOCS);
+    assertEquals(topDocs.scoreDocs.length, 10);
+    lastSD = checkPagingOrdered(topDocs, seen, lastSD.doc, lastSD.score);
+    assertEquals(lastSD.doc, 19);
+  }
+
+  @Test
+  public void testDiffScore() throws Exception {
+    Query query = new TermQuery(new Term("english", "one"));
+    PagingCollector collector = new PagingCollector(10);
+    searcher.search(query, collector);
+    TopDocs topDocs = collector.topDocs();
+    assertEquals(topDocs.totalHits, 109);
+    assertEquals(topDocs.scoreDocs.length, 10);
+    int lastSeen = 0;
+    float lastScore = Float.MAX_VALUE;
+    Set<Integer> seen = new HashSet<Integer>();
+    ScoreDoc lastSD = checkPagingUnOrdered(topDocs, seen, lastSeen, lastScore);
+    assertEquals(lastSD.doc, 108);
+    collector = new PagingCollector(10, topDocs.scoreDocs.length,topDocs.scoreDocs[9]);
+    searcher.search(query, collector);
+    topDocs = collector.topDocs();
+    assertEquals(topDocs.totalHits, 109);
+    assertEquals(topDocs.scoreDocs.length, 10);
+    lastSD = checkPagingUnOrdered(topDocs, seen, lastSD.doc, lastSD.score);
+    assertEquals(lastSD.doc, 118);
+  }
+
+  @Test
+  public void testEdgeCases() throws Exception {
+    Query query = new TermQuery(new Term("english", "one"));
+    PagingCollector collector = new PagingCollector(10);
+    searcher.search(query, collector);
+    TopDocs topDocs = collector.topDocs();
+    assertEquals(topDocs.totalHits, 109);
+    assertEquals(topDocs.scoreDocs.length, 10);
+    int lastSeen = 0;
+    float lastScore = Float.MAX_VALUE;
+    Set<Integer> seen = new HashSet<Integer>();
+    ScoreDoc lastSD = checkPagingUnOrdered(topDocs, seen, lastSeen, lastScore);
+    assertEquals(lastSD.doc, 108);
+    //get more than available
+    collector = new PagingCollector(200, topDocs.scoreDocs.length, topDocs.scoreDocs[9]);
+    searcher.search(query, collector);
+    topDocs = collector.topDocs();
+    assertEquals(topDocs.totalHits, 109);
+    assertEquals(topDocs.scoreDocs.length, 99);
+    lastSD = checkPagingUnOrdered(topDocs, seen, lastSD.doc, lastSD.score);
+    assertEquals(lastSD.doc, 901);
+    //try past the end
+    collector = new PagingCollector(10, 109, topDocs.scoreDocs[topDocs.scoreDocs.length - 1]);
+    searcher.search(query, collector);
+    topDocs = collector.topDocs();
+    assertEquals(109, topDocs.totalHits);
+    assertEquals(0, topDocs.scoreDocs.length);
+    lastSD = checkPagingUnOrdered(topDocs, seen, lastSD.doc, lastSD.score);
+    assertEquals(lastSD.doc, 901);
+
+  }
+
+  private ScoreDoc checkPagingOrdered(TopDocs topDocs, Set<Integer> seen, int lastSeen, float lastScore) {
+    for (int i = 0; i < topDocs.scoreDocs.length; i++) {
+      ScoreDoc sd = topDocs.scoreDocs[i];
+      assertFalse(seen.contains(sd.doc));
+      assertTrue(sd.doc + " is not >= " + lastSeen, sd.doc >= lastSeen);
+      assertTrue(sd.score + " is not <= " + lastScore, sd.score <= lastScore);
+      lastSeen = sd.doc;
+      lastScore = sd.score;
+      seen.add(sd.doc);
+    }
+    return new ScoreDoc(lastSeen, lastScore);
+  }
+
+  private ScoreDoc checkPagingUnOrdered(TopDocs topDocs, Set<Integer> seen, int lastSeen, float lastScore) {
+    for (int i = 0; i < topDocs.scoreDocs.length; i++) {
+      ScoreDoc sd = topDocs.scoreDocs[i];
+      //System.out.println("id: " + sd.doc + " sc: " + sd.score);
+      assertFalse("Seen before: " + sd.doc, seen.contains(sd.doc));
+      assertTrue(sd.score + " is not <= " + lastScore, sd.score <= lastScore);
+      lastSeen = sd.doc;
+      lastScore = sd.score;
+      seen.add(sd.doc);
+    }
+    return new ScoreDoc(lastSeen, lastScore);
+  }
+
+
+  @Test
+  public void testPagingIterable() throws Exception {
+
+		PagingIterable.TotalHitsRef totalHitsRef = new PagingIterable.TotalHitsRef();
+		PagingIterable.ProgressRef progressRef = new PagingIterable.ProgressRef();
+
+		TermQuery query = new TermQuery(new Term("english", "one"));
+		PagingIterable paging = new PagingIterable(searcher, query, 100);
+
+		for (ScoreDoc sd : 	paging.skipTo(90).
+							gather(20).
+							totalHits(totalHitsRef).
+							progress(progressRef)) {
+
+			System.out.println(
+					"time [" + progressRef.queryTime() + "] " +
+					"total hits [" + totalHitsRef.totalHits() + "] " +
+					"searches [" + progressRef.searchesPerformed() + "] " +
+					"position [" + progressRef.currentHitPosition() + "] " +
+					"doc id [" + sd.doc + "] " +
+					"score [" + sd.score + "]");
+		}
+    assertTrue(false);//TODO: put in a proper test
+  }
+
+
+}

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

Index: lucene/src/java/org/apache/lucene/search/PagingCollector.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/PagingCollector.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/PagingCollector.java	(revision 0)
@@ -0,0 +1,165 @@
+package org.apache.lucene.search;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.util.PriorityQueue;
+
+/**
+ * The {@link PagingCollector} allows for paging through lucene hits.
+
+ */
+public class PagingCollector extends TopDocsCollector<ScoreDoc> {
+
+	private ScoreDoc pqTop;
+	private int docBase;
+	private Scorer scorer;
+	//private ScoreDoc previousPassLowest;
+  private float prvLowScore;
+  private int prvLowDoc;
+	private int numHits;
+  private int numSeen;
+
+  public PagingCollector(int numHits) {
+		// creates an empty score doc so that i don't have to check for null
+		// each time.
+		this(numHits, 0, -1, Float.MAX_VALUE);
+	}
+
+  /**
+   *
+   * @param numHits The number of hits to collect
+   * @param prvLowDoc The previous id of the lowest scoring doc
+   * @param prvLowScore The previous low score
+   */
+  public PagingCollector(int numHits, int numSeen, int prvLowDoc, float prvLowScore) {
+    super(new HitQueue(numHits, true));
+		this.pqTop = pq.top();
+    this.numHits = numHits;
+    this.numSeen = numSeen;
+    this.prvLowDoc = prvLowDoc;
+    this.prvLowScore = prvLowScore;
+  }
+
+  /**
+   *
+   * @param numHits The number of hits to collect
+   * @param previousPassLowest The last ScoreDoc that was hit on the previous page
+   *
+   * @see #PagingCollector(int, int, int, float)
+   */
+  public PagingCollector(int numHits, int numSeen, ScoreDoc previousPassLowest) {
+		this(numHits, numSeen, previousPassLowest.doc,  previousPassLowest.score);
+	}
+
+  /** 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.
+    // 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 - numSeen < pq.size() ? totalHits - numSeen : pq.size();
+
+    if (size == 0){
+      //just return empty
+      //TODO: Do something smarter here?
+      return new TopDocs(totalHits, new ScoreDoc[0], Float.NaN);
+    }
+
+    // We know that start < pqsize, so just fix howMany. 
+    int howMany = Math.min(size, numHits);
+    ScoreDoc[] results = new ScoreDoc[howMany];
+
+    // pq's pop() returns the 'least' element in the queue, therefore need
+    // to discard the first ones, until we reach the requested range.
+    // Note that this loop will usually not be executed, since the common usage
+    // should be that the caller asks for the last howMany results. However it's
+    // needed here for completeness.
+    for (int i = pq.size() - howMany; i > 0; i--) { pq.pop(); }
+
+    // Get the requested results from pq.
+    populateResults(results, howMany);
+
+    return newTopDocs(results, 0);
+  }
+
+  /**
+   * throw new UnsupportedOperationException
+   */
+  public TopDocs topDocs(int start) {
+    // 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.
+    throw new UnsupportedOperationException("Not needed for Paging Collector, just use topDocs()");
+  }
+
+  /**
+   * throw new UnsupportedOperationException
+   */
+  public TopDocs topDocs(int start, int howMany) {
+
+    throw new UnsupportedOperationException("Not needed for Paging Collector, just use topDocs()");
+  }
+
+
+  @Override
+	public boolean acceptsDocsOutOfOrder() {
+		return true;
+	}
+
+	@Override
+	public void collect(int doc) throws IOException {
+		float score = scorer.score();
+		totalHits++;
+		doc += docBase;
+		if (score > prvLowScore /*// this hit was gathered on a previous page.*/
+            || score == prvLowScore && doc <= prvLowDoc /*  if the scores are the same and the doc is less than or equal to the previous pass lowest hit doc then skip because this collector favors lower number documents.*/
+            || (score < pqTop.score || (score == pqTop.score && doc > pqTop.doc))
+            ) {
+
+			return;
+		}
+		pqTop.doc = doc;
+		pqTop.score = score;
+		pqTop = pq.updateTop();
+	}
+
+	@Override
+	public void setNextReader(IndexReader reader, int docBase) throws IOException {
+		this.docBase = docBase;
+	}
+
+	@Override
+	public void setScorer(Scorer scorer) throws IOException {
+		this.scorer = scorer;
+	}
+
+	public ScoreDoc getLastScoreDoc(TopDocs topDocs) {
+		return topDocs.scoreDocs[(totalHits < numHits ? totalHits : numHits)-1];
+	}
+
+	public ScoreDoc getLastScoreDoc(ScoreDoc[] scoreDocs) {
+		return scoreDocs[(totalHits < numHits ? totalHits : numHits)-1];
+	}
+
+}

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

Index: lucene/src/java/org/apache/lucene/search/PagingIterable.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/PagingIterable.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/search/PagingIterable.java	(revision 0)
@@ -0,0 +1,211 @@
+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.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * The {@link PagingIterable} class allows for easy paging through lucene hits.
+
+ */
+public class PagingIterable implements Iterable<ScoreDoc> {
+
+	private static int DEFAULT_NUMBER_OF_HITS_TO_COLLECT = 1000;
+	private Searcher searcher;
+	private Query query;
+	private TotalHitsRef totalHitsRef = new TotalHitsRef();
+	private ProgressRef progressRef = new ProgressRef();
+	private int skipTo;
+	private int numHitsToCollect = DEFAULT_NUMBER_OF_HITS_TO_COLLECT;
+	private int gather = -1;
+
+	public PagingIterable(Searcher searcher, Query query) throws IOException {
+		this(searcher,query,DEFAULT_NUMBER_OF_HITS_TO_COLLECT,null,null);
+	}
+	
+	public PagingIterable(Searcher searcher, Query query, int numHitsToCollect) throws IOException {
+		this(searcher,query,numHitsToCollect,null,null);
+	}
+	
+	public PagingIterable(Searcher searcher, Query query, int numHitsToCollect, TotalHitsRef totalHitsRef, ProgressRef progressRef) throws IOException {
+		this.query = searcher.rewrite(query);
+		this.searcher = searcher;
+		this.numHitsToCollect = numHitsToCollect;
+		this.totalHitsRef = totalHitsRef == null ? this.totalHitsRef : totalHitsRef;
+		this.progressRef = progressRef == null ? this.progressRef : progressRef;
+	}
+
+  public static class TotalHitsRef {
+		//This is an atomic integer because more than likely if there is
+		//any status sent to the user, it will be done in another thread.
+		protected AtomicInteger totalHits = new AtomicInteger(0);
+		public int totalHits() {
+			return totalHits.get();
+		}
+	}
+	
+	public static class ProgressRef {
+		//These are atomic integers because more than likely if there is
+		//any status sent to the user, it will be done in another thread. 
+		protected AtomicInteger skipTo = new AtomicInteger(0);
+		protected AtomicInteger currentHitPosition = new AtomicInteger(0);
+		protected AtomicInteger searchesPerformed = new AtomicInteger(0);
+		protected AtomicLong queryTime = new AtomicLong(0);
+		
+		public int skipTo() {
+			return skipTo.get();
+		}
+		public int currentHitPosition() {
+			return currentHitPosition.get();
+		}
+		public int searchesPerformed() {
+			return searchesPerformed.get();
+		}
+		public long queryTime() {
+			return queryTime.get();
+		}
+	}
+	
+	/**
+	 * Gets the total hits of the search.
+	 * @return the total hits.
+	 */
+	public int getTotalHits() {
+		return totalHitsRef.totalHits();
+	}
+	
+	/**
+	 * Allows for gathering of the total hits of this search.
+	 * @param ref {@link TotalHitsRef}.
+	 * @return this.
+	 */
+	public PagingIterable totalHits(TotalHitsRef ref) {
+		totalHitsRef = ref;
+		return this;
+	}
+	
+	/**
+	 * Skips the first x number of hits.
+	 * @param skipTo the number hits to skip.
+	 * @return this.
+	 */
+	public PagingIterable skipTo(int skipTo) {
+		this.skipTo = skipTo;
+		return this;
+	}
+	
+	/**
+	 * Only gather up to x number of hits.
+	 * @param gather the number of hits to gather.
+	 * @return this.
+	 */
+	public PagingIterable gather(int gather) {
+		this.gather = gather;
+		return this;
+	}
+	
+	/**
+	 * Allows for gathering the progress of the paging.
+	 * @param ref the {@link ProgressRef}.
+	 * @return this.
+	 */
+	public PagingIterable progress(ProgressRef ref) {
+		this.progressRef = ref;
+		return this;
+	}
+	
+	/**
+	 * The {@link ScoreDoc} iterator.
+	 */
+
+	public Iterator<ScoreDoc> iterator() {
+		return skipHits(new PagingIterator());
+	}
+	
+	class PagingIterator implements Iterator<ScoreDoc> {
+		private PagingCollector collector;
+		private ScoreDoc[] scoreDocs;
+		private int counter = 0;
+		private int offset = 0;
+		private int endPosition = gather == -1 ? Integer.MAX_VALUE : skipTo + gather;
+		
+		PagingIterator() {
+			search();
+		}
+		
+		void search() {
+			long s = System.currentTimeMillis();
+			progressRef.searchesPerformed.incrementAndGet();
+			if (collector == null) {
+				collector = new PagingCollector(numHitsToCollect);
+			} else {
+				collector = new PagingCollector(numHitsToCollect,counter, scoreDocs[scoreDocs.length - 1]);
+			}
+			try {
+				searcher.search(query, collector);
+			} catch (IOException e) {
+				throw new RuntimeException(e);
+			}
+			
+			totalHitsRef.totalHits.set(collector.getTotalHits());
+			scoreDocs = collector.topDocs().scoreDocs;
+			long e = System.currentTimeMillis();
+			progressRef.queryTime.addAndGet(e-s);
+		}
+
+		public boolean hasNext() {
+			return counter < totalHitsRef.totalHits() && counter < endPosition ? true : false;
+		}
+
+		public ScoreDoc next() {
+			if (isCurrentCollectorExhausted()) {
+				search();
+				offset = 0;
+			}
+			progressRef.currentHitPosition.set(counter);
+			counter++;
+			return scoreDocs[offset++];
+		}
+
+		private boolean isCurrentCollectorExhausted() {
+			return offset < scoreDocs.length ? false : true;
+		}
+
+
+		public void remove() {
+			throw new RuntimeException("read only");
+		}
+	}
+
+	private Iterator<ScoreDoc> skipHits(Iterator<ScoreDoc> iterator) {
+		progressRef.skipTo.set(skipTo);
+		for (int i = 0; i < skipTo && iterator.hasNext(); i++) {
+			//eats the hits, and moves the iterator to the desired skip to position.
+			progressRef.currentHitPosition.set(i);
+			iterator.next();
+		}
+		return iterator;
+	}
+	
+	public static void setDefaultNumberOfHitsToCollect(int num) {
+		DEFAULT_NUMBER_OF_HITS_TO_COLLECT = num;
+	}
+}
\ No newline at end of file

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

Index: lucene/src/java/org/apache/lucene/search/TopDocsCollector.java
===================================================================
--- lucene/src/java/org/apache/lucene/search/TopDocsCollector.java	(revision 926765)
+++ lucene/src/java/org/apache/lucene/search/TopDocsCollector.java	(working copy)
@@ -75,7 +75,7 @@
   }
   
   /** Returns the top docs that were collected by this collector. */
-  public final TopDocs topDocs() {
+  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.
@@ -94,7 +94,7 @@
    * with the returned {@link TopDocs} object, which will contain all the
    * results this search execution collected.
    */
-  public final TopDocs topDocs(int start) {
+  public TopDocs topDocs(int start) {
     // 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.
@@ -115,7 +115,7 @@
    * returned {@link TopDocs} object, which will contain all the results this
    * search execution collected.
    */
-  public final TopDocs topDocs(int start, int howMany) {
+  public TopDocs topDocs(int start, int howMany) {
     
     // In case pq was populated with sentinel values, there might be less
     // results than pq.size(). Therefore return all results until either
