Index: src/test/org/apache/lucene/search/TestSort.java =================================================================== --- src/test/org/apache/lucene/search/TestSort.java (revision 962932) +++ src/test/org/apache/lucene/search/TestSort.java (working copy) @@ -747,7 +747,7 @@ Sort[] sort = new Sort[] { new Sort(SortField.FIELD_DOC), new Sort() }; for (int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, false, + TopFieldCollector tdc = TopFieldCollector.create(sort[i], 10, false, false, false, true); full.search(q, tdc); @@ -766,7 +766,7 @@ Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; for (int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, false, + TopFieldCollector tdc = TopFieldCollector.create(sort[i], 10, true, false, false, true); full.search(q, tdc); @@ -786,7 +786,7 @@ Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; for (int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, + TopFieldCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, false, true); full.search(q, tdc); @@ -806,7 +806,7 @@ Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; for (int i = 0; i < sort.length; i++) { Query q = new MatchAllDocsQuery(); - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, + TopFieldCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true); full.search(q, tdc); @@ -854,7 +854,7 @@ bq.setMinimumNumberShouldMatch(1); for (int i = 0; i < sort.length; i++) { for (int j = 0; j < tfcOptions.length; j++) { - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, + TopFieldCollector tdc = TopFieldCollector.create(sort[i], 10, tfcOptions[j][0], tfcOptions[j][1], tfcOptions[j][2], false); assertTrue(tdc.getClass().getName().endsWith("$"+actualTFCClasses[j])); @@ -873,7 +873,7 @@ // Two Sort criteria to instantiate the multi/single comparators. Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort() }; for (int i = 0; i < sort.length; i++) { - TopDocsCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true); + TopFieldCollector tdc = TopFieldCollector.create(sort[i], 10, true, true, true, true); TopDocs td = tdc.topDocs(); assertEquals(0, td.totalHits); assertTrue(Float.isNaN(td.getMaxScore())); Index: src/test/org/apache/lucene/search/TestTopDocsCollector.java =================================================================== --- src/test/org/apache/lucene/search/TestTopDocsCollector.java (revision 962932) +++ src/test/org/apache/lucene/search/TestTopDocsCollector.java (working copy) @@ -1,210 +0,0 @@ -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.analysis.KeywordAnalyzer; -import org.apache.lucene.document.Document; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriter.MaxFieldLength; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.RAMDirectory; -import org.apache.lucene.util.LuceneTestCase; - -public class TestTopDocsCollector extends LuceneTestCase { - - private static final class MyTopsDocCollector extends TopDocsCollector { - - private int idx = 0; - private int base = 0; - - public MyTopsDocCollector(int size) { - super(new HitQueue(size, false)); - } - - @Override - protected TopDocs newTopDocs(ScoreDoc[] results, int start) { - if (results == null) { - return EMPTY_TOPDOCS; - } - - float maxScore = Float.NaN; - if (start == 0) { - maxScore = results[0].score; - } else { - for (int i = pq.size(); i > 1; i--) { pq.pop(); } - maxScore = ((ScoreDoc) pq.pop()).score; - } - - return new TopDocs(totalHits, results, maxScore); - } - - @Override - public void collect(int doc) throws IOException { - ++totalHits; - pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++])); - } - - @Override - public void setNextReader(IndexReader reader, int docBase) - throws IOException { - base = docBase; - } - - @Override - public void setScorer(Scorer scorer) throws IOException { - // Don't do anything. Assign scores in random - } - - @Override - public boolean acceptsDocsOutOfOrder() { - return true; - } - - } - - // Scores array to be used by MyTopDocsCollector. If it is changed, MAX_SCORE - // must also change. - private static final float[] scores = new float[] { - 0.7767749f, 1.7839992f, 8.9925785f, 7.9608946f, 0.07948637f, 2.6356435f, - 7.4950366f, 7.1490803f, 8.108544f, 4.961808f, 2.2423935f, 7.285586f, 4.6699767f, - 2.9655676f, 6.953706f, 5.383931f, 6.9916306f, 8.365894f, 7.888485f, 8.723962f, - 3.1796896f, 0.39971232f, 1.3077754f, 6.8489285f, 9.17561f, 5.060466f, 7.9793315f, - 8.601509f, 4.1858315f, 0.28146625f - }; - - private static final float MAX_SCORE = 9.17561f; - - private Directory dir = new RAMDirectory(); - - private TopDocsCollector doSearch(int numResults) throws IOException { - Query q = new MatchAllDocsQuery(); - IndexSearcher searcher = new IndexSearcher(dir, true); - TopDocsCollector tdc = new MyTopsDocCollector(numResults); - searcher.search(q, tdc); - searcher.close(); - return tdc; - } - - @Override - protected void setUp() throws Exception { - super.setUp(); - - // populate an index with 30 documents, this should be enough for the test. - // The documents have no content - the test uses MatchAllDocsQuery(). - IndexWriter writer = new IndexWriter(dir, new KeywordAnalyzer(), MaxFieldLength.UNLIMITED); - for (int i = 0; i < 30; i++) { - writer.addDocument(new Document()); - } - writer.close(); - } - - @Override - protected void tearDown() throws Exception { - dir.close(); - dir = null; - super.tearDown(); - } - - public void testInvalidArguments() throws Exception { - int numResults = 5; - TopDocsCollector tdc = doSearch(numResults); - - // start < 0 - assertEquals(0, tdc.topDocs(-1).scoreDocs.length); - - // start > pq.size() - assertEquals(0, tdc.topDocs(numResults + 1).scoreDocs.length); - - // start == pq.size() - assertEquals(0, tdc.topDocs(numResults).scoreDocs.length); - - // howMany < 0 - assertEquals(0, tdc.topDocs(0, -1).scoreDocs.length); - - // howMany == 0 - assertEquals(0, tdc.topDocs(0, 0).scoreDocs.length); - - } - - public void testZeroResults() throws Exception { - TopDocsCollector tdc = new MyTopsDocCollector(5); - assertEquals(0, tdc.topDocs(0, 1).scoreDocs.length); - } - - public void testFirstResultsPage() throws Exception { - TopDocsCollector tdc = doSearch(15); - assertEquals(10, tdc.topDocs(0, 10).scoreDocs.length); - } - - public void testSecondResultsPages() throws Exception { - TopDocsCollector tdc = doSearch(15); - // ask for more results than are available - assertEquals(5, tdc.topDocs(10, 10).scoreDocs.length); - - // ask for 5 results (exactly what there should be - tdc = doSearch(15); - assertEquals(5, tdc.topDocs(10, 5).scoreDocs.length); - - // ask for less results than there are - tdc = doSearch(15); - assertEquals(4, tdc.topDocs(10, 4).scoreDocs.length); - } - - public void testGetAllResults() throws Exception { - TopDocsCollector tdc = doSearch(15); - assertEquals(15, tdc.topDocs().scoreDocs.length); - } - - public void testGetResultsFromStart() throws Exception { - TopDocsCollector tdc = doSearch(15); - // should bring all results - assertEquals(15, tdc.topDocs(0).scoreDocs.length); - - tdc = doSearch(15); - // get the last 5 only. - assertEquals(5, tdc.topDocs(10).scoreDocs.length); - } - - public void testMaxScore() throws Exception { - // ask for all results - TopDocsCollector tdc = doSearch(15); - TopDocs td = tdc.topDocs(); - assertEquals(MAX_SCORE, td.getMaxScore(), 0f); - - // ask for 5 last results - tdc = doSearch(15); - td = tdc.topDocs(10); - assertEquals(MAX_SCORE, td.getMaxScore(), 0f); - } - - // This does not test the PQ's correctness, but whether topDocs() - // implementations return the results in decreasing score order. - public void testResultsOrder() throws Exception { - TopDocsCollector tdc = doSearch(15); - ScoreDoc[] sd = tdc.topDocs().scoreDocs; - - assertEquals(MAX_SCORE, sd[0].score, 0f); - for (int i = 1; i < sd.length; i++) { - assertTrue(sd[i - 1].score >= sd[i].score); - } - } - -} Index: src/test/org/apache/lucene/search/JustCompileSearch.java =================================================================== --- src/test/org/apache/lucene/search/JustCompileSearch.java (revision 962932) +++ src/test/org/apache/lucene/search/JustCompileSearch.java (working copy) @@ -419,9 +419,9 @@ } } - static final class JustCompileTopDocsCollector extends TopDocsCollector { + static final class JustCompileTopDocsCollector extends PQTopDocsCollector { - protected JustCompileTopDocsCollector(PriorityQueue pq) { + protected JustCompileTopDocsCollector(PriorityQueue pq) { super(pq); } Index: src/test/org/apache/lucene/search/TestTopScoreDocCollector.java =================================================================== --- src/test/org/apache/lucene/search/TestTopScoreDocCollector.java (revision 962932) +++ src/test/org/apache/lucene/search/TestTopScoreDocCollector.java (working copy) @@ -59,7 +59,7 @@ bq.setMinimumNumberShouldMatch(1); IndexSearcher searcher = new IndexSearcher(dir, true); for (int i = 0; i < inOrder.length; i++) { - TopDocsCollector tdc = TopScoreDocCollector.create(3, inOrder[i]); + TopDocsCollector tdc = TopScoreDocCollector.create(3, inOrder[i]); assertEquals("org.apache.lucene.search.TopScoreDocCollector$" + actualTSDCClass[i], tdc.getClass().getName()); searcher.search(new MatchAllDocsQuery(), tdc); Index: src/test/org/apache/lucene/search/TestElevationComparator.java =================================================================== --- src/test/org/apache/lucene/search/TestElevationComparator.java (revision 962932) +++ src/test/org/apache/lucene/search/TestElevationComparator.java (working copy) @@ -71,7 +71,7 @@ new SortField(null, SortField.SCORE, reversed) ); - TopDocsCollector topCollector = TopFieldCollector.create(sort, 50, false, true, true, true); + TopFieldCollector topCollector = TopFieldCollector.create(sort, 50, false, true, true, true); searcher.search(newq, null, topCollector); TopDocs topDocs = topCollector.topDocs(0, 10); Index: src/test/org/apache/lucene/search/TestPQTopDocsCollector.java =================================================================== --- src/test/org/apache/lucene/search/TestPQTopDocsCollector.java (revision 0) +++ src/test/org/apache/lucene/search/TestPQTopDocsCollector.java (revision 0) @@ -0,0 +1,210 @@ +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.analysis.KeywordAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriter.MaxFieldLength; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMDirectory; +import org.apache.lucene.util.LuceneTestCase; + +public class TestPQTopDocsCollector extends LuceneTestCase { + + private static final class MyTopsDocCollector extends PQTopDocsCollector { + + private int idx = 0; + private int base = 0; + + public MyTopsDocCollector(int size) { + super(new HitQueue(size, false)); + } + + @Override + protected TopDocs newTopDocs(ScoreDoc[] results, int start) { + if (results == null) { + return EMPTY_TOPDOCS; + } + + float maxScore = Float.NaN; + if (start == 0) { + maxScore = results[0].score; + } else { + for (int i = pq.size(); i > 1; i--) { pq.pop(); } + maxScore = pq.pop().score; + } + + return new TopDocs(totalHits, results, maxScore); + } + + @Override + public void collect(int doc) throws IOException { + ++totalHits; + pq.insertWithOverflow(new ScoreDoc(doc + base, scores[idx++])); + } + + @Override + public void setNextReader(IndexReader reader, int docBase) + throws IOException { + base = docBase; + } + + @Override + public void setScorer(Scorer scorer) throws IOException { + // Don't do anything. Assign scores in random + } + + @Override + public boolean acceptsDocsOutOfOrder() { + return true; + } + + } + + // Scores array to be used by MyTopDocsCollector. If it is changed, MAX_SCORE + // must also change. + private static final float[] scores = new float[] { + 0.7767749f, 1.7839992f, 8.9925785f, 7.9608946f, 0.07948637f, 2.6356435f, + 7.4950366f, 7.1490803f, 8.108544f, 4.961808f, 2.2423935f, 7.285586f, 4.6699767f, + 2.9655676f, 6.953706f, 5.383931f, 6.9916306f, 8.365894f, 7.888485f, 8.723962f, + 3.1796896f, 0.39971232f, 1.3077754f, 6.8489285f, 9.17561f, 5.060466f, 7.9793315f, + 8.601509f, 4.1858315f, 0.28146625f + }; + + private static final float MAX_SCORE = 9.17561f; + + private Directory dir = new RAMDirectory(); + + private TopDocsCollector doSearch(int numResults) throws IOException { + Query q = new MatchAllDocsQuery(); + IndexSearcher searcher = new IndexSearcher(dir, true); + TopDocsCollector tdc = new MyTopsDocCollector(numResults); + searcher.search(q, tdc); + searcher.close(); + return tdc; + } + + @Override + protected void setUp() throws Exception { + super.setUp(); + + // populate an index with 30 documents, this should be enough for the test. + // The documents have no content - the test uses MatchAllDocsQuery(). + IndexWriter writer = new IndexWriter(dir, new KeywordAnalyzer(), MaxFieldLength.UNLIMITED); + for (int i = 0; i < 30; i++) { + writer.addDocument(new Document()); + } + writer.close(); + } + + @Override + protected void tearDown() throws Exception { + dir.close(); + dir = null; + super.tearDown(); + } + + public void testInvalidArguments() throws Exception { + int numResults = 5; + TopDocsCollector tdc = doSearch(numResults); + + // start < 0 + assertEquals(0, tdc.topDocs(-1).scoreDocs.length); + + // start > pq.size() + assertEquals(0, tdc.topDocs(numResults + 1).scoreDocs.length); + + // start == pq.size() + assertEquals(0, tdc.topDocs(numResults).scoreDocs.length); + + // howMany < 0 + assertEquals(0, tdc.topDocs(0, -1).scoreDocs.length); + + // howMany == 0 + assertEquals(0, tdc.topDocs(0, 0).scoreDocs.length); + + } + + public void testZeroResults() throws Exception { + TopDocsCollector tdc = new MyTopsDocCollector(5); + assertEquals(0, tdc.topDocs(0, 1).scoreDocs.length); + } + + public void testFirstResultsPage() throws Exception { + TopDocsCollector tdc = doSearch(15); + assertEquals(10, tdc.topDocs(0, 10).scoreDocs.length); + } + + public void testSecondResultsPages() throws Exception { + TopDocsCollector tdc = doSearch(15); + // ask for more results than are available + assertEquals(5, tdc.topDocs(10, 10).scoreDocs.length); + + // ask for 5 results (exactly what there should be + tdc = doSearch(15); + assertEquals(5, tdc.topDocs(10, 5).scoreDocs.length); + + // ask for less results than there are + tdc = doSearch(15); + assertEquals(4, tdc.topDocs(10, 4).scoreDocs.length); + } + + public void testGetAllResults() throws Exception { + TopDocsCollector tdc = doSearch(15); + assertEquals(15, tdc.topDocs().scoreDocs.length); + } + + public void testGetResultsFromStart() throws Exception { + TopDocsCollector tdc = doSearch(15); + // should bring all results + assertEquals(15, tdc.topDocs(0).scoreDocs.length); + + tdc = doSearch(15); + // get the last 5 only. + assertEquals(5, tdc.topDocs(10).scoreDocs.length); + } + + public void testMaxScore() throws Exception { + // ask for all results + TopDocsCollector tdc = doSearch(15); + TopDocs td = tdc.topDocs(); + assertEquals(MAX_SCORE, td.getMaxScore(), 0f); + + // ask for 5 last results + tdc = doSearch(15); + td = tdc.topDocs(10); + assertEquals(MAX_SCORE, td.getMaxScore(), 0f); + } + + // This does not test the PQ's correctness, but whether topDocs() + // implementations return the results in decreasing score order. + public void testResultsOrder() throws Exception { + TopDocsCollector tdc = doSearch(15); + ScoreDoc[] sd = tdc.topDocs().scoreDocs; + + assertEquals(MAX_SCORE, sd[0].score, 0f); + for (int i = 1; i < sd.length; i++) { + assertTrue(sd[i - 1].score >= sd[i].score); + } + } + +} Index: src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java =================================================================== --- src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java (revision 962932) +++ src/test/org/apache/lucene/search/TestPositiveScoresOnlyCollector.java (working copy) @@ -66,7 +66,7 @@ } Scorer s = new SimpleScorer(); - TopDocsCollector tdc = TopScoreDocCollector.create(scores.length, true); + TopScoreDocCollector tdc = TopScoreDocCollector.create(scores.length, true); Collector c = new PositiveScoresOnlyCollector(tdc); c.setScorer(s); while (s.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { Index: src/java/org/apache/lucene/search/TopScoreDocCollector.java =================================================================== --- src/java/org/apache/lucene/search/TopScoreDocCollector.java (revision 962932) +++ src/java/org/apache/lucene/search/TopScoreDocCollector.java (working copy) @@ -34,7 +34,7 @@ * collector will not properly collect hits with such * scores. */ -public abstract class TopScoreDocCollector extends TopDocsCollector { +public abstract class TopScoreDocCollector extends PQTopDocsCollector { // Assumes docs are scored in order. private static class InOrderTopScoreDocCollector extends TopScoreDocCollector { Index: src/java/org/apache/lucene/search/PQTopDocsCollector.java =================================================================== --- src/java/org/apache/lucene/search/PQTopDocsCollector.java (revision 0) +++ src/java/org/apache/lucene/search/PQTopDocsCollector.java (revision 0) @@ -0,0 +1,163 @@ +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.util.PriorityQueue; + +/** + * A base class for all collectors that return a {@link TopDocs} output. This + * collector allows easy extension by providing a single constructor which + * accepts a {@link PriorityQueue} as well as protected members for that + * priority queue and a counter of the number of total hits.
+ * Extending classes can override {@link #topDocs(int, int)} and + * {@link #getTotalHits()} in order to provide their own implementation. + */ +public abstract class PQTopDocsCollector extends TopDocsCollector { + + // This is used in case topDocs() is called with illegal parameters, or there + // simply aren't (enough) results. + protected static final TopDocs EMPTY_TOPDOCS = new TopDocs(0, new ScoreDoc[0], Float.NaN); + + /** + * The priority queue which holds the top documents. Note that different + * implementations of PriorityQueue give different meaning to 'top documents'. + * HitQueue for example aggregates the top scoring documents, while other PQ + * implementations may hold documents sorted by other criteria. + */ + protected PriorityQueue pq; + + /** The total number of documents that the collector encountered. */ + protected int totalHits; + + protected PQTopDocsCollector(PriorityQueue pq) { + this.pq = pq; + } + + /** + * Populates the results array with the ScoreDoc instaces. This can be + * overridden in case a different ScoreDoc type should be returned. + */ + protected void populateResults(ScoreDoc[] results, int howMany) { + for (int i = howMany - 1; i >= 0; i--) { + results[i] = pq.pop(); + } + } + + /** + * Returns a {@link TopDocs} instance containing the given results. If + * results is null it means there are no results to return, + * either because there were 0 calls to collect() or because the arguments to + * topDocs were invalid. + * Subclass MUST override if using a subclass of TopDocs. + * + * @param results ScoreDocs to wrap + * @param start unused in default impl, but is the start of the requested range + * @return an appropriately typed TopDocs + */ + @SuppressWarnings("unchecked") + protected T newTopDocs(ScoreDoc[] results, int start) { + return (T)(results == null ? EMPTY_TOPDOCS : new TopDocs(totalHits, results)); + } + + @Override + public int size() { + return pq.size(); + } + + /** The total number of documents that matched this query. */ + @Override + public int getTotalHits() { + return totalHits; + } + + /** Returns the top docs that were collected by this collector. */ + @Override + public final T 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()); + } + + /** + * Returns the documents in the rage [start .. pq.size()) that were collected + * by this collector. Note that if start >= pq.size(), an empty TopDocs is + * returned.
+ * This method is convenient to call if the application always asks for the + * last results, starting from the last 'page'.
+ * NOTE: you cannot call this method more than once for each search + * execution. If you need to call it more than once, passing each time a + * different start, you should call {@link #topDocs()} and work + * with the returned {@link TopDocs} object, which will contain all the + * results this search execution collected. + */ + @Override + public final T 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. + return topDocs(start, totalHits < pq.size() ? totalHits : pq.size()); + } + + /** + * Returns the documents in the rage [start .. start+howMany) that were + * collected by this collector. Note that if start >= pq.size(), an empty + * TopDocs is returned, and if pq.size() - start < howMany, then only the + * available documents in [start .. pq.size()) are returned.
+ * This method is useful to call in case pagination of search results is + * allowed by the search application, as well as it attempts to optimize the + * memory used by allocating only as much as requested by howMany.
+ * NOTE: you cannot call this method more than once for each search + * execution. If you need to call it more than once, passing each time a + * different range, you should call {@link #topDocs()} and work with the + * returned {@link TopDocs} object, which will contain all the results this + * search execution collected. + */ + @Override + public final T 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 + // pq.size() or totalHits. + int size = totalHits < pq.size() ? totalHits : pq.size(); + + // Don't bother to throw an exception, just return an empty TopDocs in case + // the parameters are invalid or out of range. + if (start < 0 || start >= size || howMany <= 0) { + return newTopDocs(null, start); + } + + // We know that start < pqsize, so just fix howMany. + howMany = Math.min(size - start, howMany); + 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() - start - howMany; i > 0; i--) { pq.pop(); } + + // Get the requested results from pq. + populateResults(results, howMany); + + return newTopDocs(results, start); + } + +} Index: src/java/org/apache/lucene/search/TopFieldCollector.java =================================================================== --- src/java/org/apache/lucene/search/TopFieldCollector.java (revision 962932) +++ src/java/org/apache/lucene/search/TopFieldCollector.java (working copy) @@ -33,7 +33,7 @@ *

NOTE: This API is experimental and might change in * incompatible ways in the next release.

*/ -public abstract class TopFieldCollector extends TopDocsCollector { +public abstract class TopFieldCollector extends PQTopDocsCollector { // TODO: one optimization we could do is to pre-fill // the queue with sentinel value that guaranteed to @@ -978,7 +978,7 @@ } @Override - protected TopDocs newTopDocs(ScoreDoc[] results, int start) { + protected TopFieldDocs newTopDocs(ScoreDoc[] results, int start) { if (results == null) { results = EMPTY_SCOREDOCS; // Set maxScore to NaN, in case this is a maxScore tracking collector. Index: src/java/org/apache/lucene/search/TopDocsCollector.java =================================================================== --- src/java/org/apache/lucene/search/TopDocsCollector.java (revision 962932) +++ src/java/org/apache/lucene/search/TopDocsCollector.java (working copy) @@ -17,74 +17,23 @@ * limitations under the License. */ - -import org.apache.lucene.util.PriorityQueue; - /** - * A base class for all collectors that return a {@link TopDocs} output. This - * collector allows easy extension by providing a single constructor which - * accepts a {@link PriorityQueue} as well as protected members for that - * priority queue and a counter of the number of total hits.
- * Extending classes can override {@link #topDocs(int, int)} and - * {@link #getTotalHits()} in order to provide their own implementation. + * An interface for all collectors that return a {@link TopDocs} output. */ -public abstract class TopDocsCollector extends Collector { +public abstract class TopDocsCollector extends Collector { - // This is used in case topDocs() is called with illegal parameters, or there - // simply aren't (enough) results. - protected static final TopDocs EMPTY_TOPDOCS = new TopDocs(0, new ScoreDoc[0], Float.NaN); + /** @return The size of the TopDocs. */ + public abstract int size(); - /** - * The priority queue which holds the top documents. Note that different - * implementations of PriorityQueue give different meaning to 'top documents'. - * HitQueue for example aggregates the top scoring documents, while other PQ - * implementations may hold documents sorted by other criteria. - */ - protected PriorityQueue pq; - - /** The total number of documents that the collector encountered. */ - protected int totalHits; + /** @return The total number of documents that matched this query. */ + public abstract int getTotalHits(); - protected TopDocsCollector(PriorityQueue pq) { - this.pq = pq; - } - - /** - * Populates the results array with the ScoreDoc instaces. This can be - * overridden in case a different ScoreDoc type should be returned. - */ - protected void populateResults(ScoreDoc[] results, int howMany) { - for (int i = howMany - 1; i >= 0; i--) { - results[i] = pq.pop(); - } - } + /** @return The top docs that were collected by this collector. */ + public abstract T topDocs(); /** - * Returns a {@link TopDocs} instance containing the given results. If - * results is null it means there are no results to return, - * either because there were 0 calls to collect() or because the arguments to - * topDocs were invalid. - */ - protected TopDocs newTopDocs(ScoreDoc[] results, int start) { - return results == null ? EMPTY_TOPDOCS : new TopDocs(totalHits, results); - } - - /** The total number of documents that matched this query. */ - public int getTotalHits() { - return totalHits; - } - - /** Returns the top docs that were collected by this collector. */ - public final 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()); - } - - /** - * Returns the documents in the rage [start .. pq.size()) that were collected - * by this collector. Note that if start >= pq.size(), an empty TopDocs is + * Returns the documents in the rage [start .. size) that were collected + * by this collector. Note that if start >= size, an empty TopDocs is * returned.
* This method is convenient to call if the application always asks for the * last results, starting from the last 'page'.
@@ -94,18 +43,13 @@ * with the returned {@link TopDocs} object, which will contain all the * results this search execution collected. */ - public final 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. - return topDocs(start, totalHits < pq.size() ? totalHits : pq.size()); - } + public abstract T topDocs(int start); /** * Returns the documents in the rage [start .. start+howMany) that were - * collected by this collector. Note that if start >= pq.size(), an empty - * TopDocs is returned, and if pq.size() - start < howMany, then only the - * available documents in [start .. pq.size()) are returned.
+ * collected by this collector. Note that if start >= size, an empty + * TopDocs is returned, and if size - start < howMany, then only the + * available documents in [start .. size) are returned.
* This method is useful to call in case pagination of search results is * allowed by the search application, as well as it attempts to optimize the * memory used by allocating only as much as requested by howMany.
@@ -115,34 +59,6 @@ * returned {@link TopDocs} object, which will contain all the results this * search execution collected. */ - public final 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 - // pq.size() or totalHits. - int size = totalHits < pq.size() ? totalHits : pq.size(); + public abstract T topDocs(int start, int howMany); - // Don't bother to throw an exception, just return an empty TopDocs in case - // the parameters are invalid or out of range. - if (start < 0 || start >= size || howMany <= 0) { - return newTopDocs(null, start); - } - - // We know that start < pqsize, so just fix howMany. - howMany = Math.min(size - start, howMany); - 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() - start - howMany; i > 0; i--) { pq.pop(); } - - // Get the requested results from pq. - populateResults(results, howMany); - - return newTopDocs(results, start); - } - -} +} \ No newline at end of file