mirror of https://github.com/apache/lucene.git
LUCENE-2215: add IndexSearcher.searchAfter
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1173423 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
0fd96b2526
commit
e182c59b9d
|
@ -532,6 +532,10 @@ New features
|
||||||
components between all fields, and PerFieldReuseStrategy which shares per field.
|
components between all fields, and PerFieldReuseStrategy which shares per field.
|
||||||
(Chris Male)
|
(Chris Male)
|
||||||
|
|
||||||
|
* LUCENE-2215: Added IndexSearcher.searchAfter which returns results after a specified
|
||||||
|
ScoreDoc (e.g. last document on the previous page) to support deep paging use cases.
|
||||||
|
(Aaron McCurry, Grant Ingersoll, Robert Muir)
|
||||||
|
|
||||||
Optimizations
|
Optimizations
|
||||||
|
|
||||||
* LUCENE-2588: Don't store unnecessary suffixes when writing the terms
|
* LUCENE-2588: Don't store unnecessary suffixes when writing the terms
|
||||||
|
|
|
@ -275,6 +275,34 @@ public class IndexSearcher implements Closeable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Finds the top <code>n</code>
|
||||||
|
* hits for <code>query</code>, applying <code>filter</code> if non-null,
|
||||||
|
* where all results are after a previous result (<code>after</code>).
|
||||||
|
* <p>
|
||||||
|
* By passing the bottom result from a previous page as <code>after</code>,
|
||||||
|
* this method can be used for efficient 'deep-paging' across potentially
|
||||||
|
* large result sets.
|
||||||
|
*
|
||||||
|
* @throws BooleanQuery.TooManyClauses
|
||||||
|
*/
|
||||||
|
public TopDocs searchAfter(ScoreDoc after, Query query, int n) throws IOException {
|
||||||
|
return searchAfter(after, query, null, n);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Finds the top <code>n</code>
|
||||||
|
* hits for <code>query</code>, applying <code>filter</code> if non-null,
|
||||||
|
* where all results are after a previous result (<code>after</code>).
|
||||||
|
* <p>
|
||||||
|
* By passing the bottom result from a previous page as <code>after</code>,
|
||||||
|
* this method can be used for efficient 'deep-paging' across potentially
|
||||||
|
* large result sets.
|
||||||
|
*
|
||||||
|
* @throws BooleanQuery.TooManyClauses
|
||||||
|
*/
|
||||||
|
public TopDocs searchAfter(ScoreDoc after, Query query, Filter filter, int n) throws IOException {
|
||||||
|
return search(createNormalizedWeight(query), filter, after, n);
|
||||||
|
}
|
||||||
|
|
||||||
/** Finds the top <code>n</code>
|
/** Finds the top <code>n</code>
|
||||||
* hits for <code>query</code>.
|
* hits for <code>query</code>.
|
||||||
*
|
*
|
||||||
|
@ -293,7 +321,7 @@ public class IndexSearcher implements Closeable {
|
||||||
*/
|
*/
|
||||||
public TopDocs search(Query query, Filter filter, int n)
|
public TopDocs search(Query query, Filter filter, int n)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return search(createNormalizedWeight(query), filter, n);
|
return search(createNormalizedWeight(query), filter, null, n);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Lower-level search API.
|
/** Lower-level search API.
|
||||||
|
@ -371,9 +399,9 @@ public class IndexSearcher implements Closeable {
|
||||||
* {@link IndexSearcher#search(Query,Filter,int)} instead.
|
* {@link IndexSearcher#search(Query,Filter,int)} instead.
|
||||||
* @throws BooleanQuery.TooManyClauses
|
* @throws BooleanQuery.TooManyClauses
|
||||||
*/
|
*/
|
||||||
protected TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
|
protected TopDocs search(Weight weight, Filter filter, ScoreDoc after, int nDocs) throws IOException {
|
||||||
if (executor == null) {
|
if (executor == null) {
|
||||||
return search(leafContexts, weight, filter, nDocs);
|
return search(leafContexts, weight, filter, after, nDocs);
|
||||||
} else {
|
} else {
|
||||||
final HitQueue hq = new HitQueue(nDocs, false);
|
final HitQueue hq = new HitQueue(nDocs, false);
|
||||||
final Lock lock = new ReentrantLock();
|
final Lock lock = new ReentrantLock();
|
||||||
|
@ -381,7 +409,7 @@ public class IndexSearcher implements Closeable {
|
||||||
|
|
||||||
for (int i = 0; i < leafSlices.length; i++) { // search each sub
|
for (int i = 0; i < leafSlices.length; i++) { // search each sub
|
||||||
runner.submit(
|
runner.submit(
|
||||||
new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, nDocs, hq));
|
new SearcherCallableNoSort(lock, this, leafSlices[i], weight, filter, after, nDocs, hq));
|
||||||
}
|
}
|
||||||
|
|
||||||
int totalHits = 0;
|
int totalHits = 0;
|
||||||
|
@ -408,14 +436,14 @@ public class IndexSearcher implements Closeable {
|
||||||
* {@link IndexSearcher#search(Query,Filter,int)} instead.
|
* {@link IndexSearcher#search(Query,Filter,int)} instead.
|
||||||
* @throws BooleanQuery.TooManyClauses
|
* @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 after, int nDocs) throws IOException {
|
||||||
// single thread
|
// single thread
|
||||||
int limit = reader.maxDoc();
|
int limit = reader.maxDoc();
|
||||||
if (limit == 0) {
|
if (limit == 0) {
|
||||||
limit = 1;
|
limit = 1;
|
||||||
}
|
}
|
||||||
nDocs = Math.min(nDocs, limit);
|
nDocs = Math.min(nDocs, limit);
|
||||||
TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
|
TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, after, !weight.scoresDocsOutOfOrder());
|
||||||
search(leaves, weight, filter, collector);
|
search(leaves, weight, filter, collector);
|
||||||
return collector.topDocs();
|
return collector.topDocs();
|
||||||
}
|
}
|
||||||
|
@ -704,23 +732,25 @@ public class IndexSearcher implements Closeable {
|
||||||
private final IndexSearcher searcher;
|
private final IndexSearcher searcher;
|
||||||
private final Weight weight;
|
private final Weight weight;
|
||||||
private final Filter filter;
|
private final Filter filter;
|
||||||
|
private final ScoreDoc after;
|
||||||
private final int nDocs;
|
private final int nDocs;
|
||||||
private final HitQueue hq;
|
private final HitQueue hq;
|
||||||
private final LeafSlice slice;
|
private final LeafSlice slice;
|
||||||
|
|
||||||
public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
|
public SearcherCallableNoSort(Lock lock, IndexSearcher searcher, LeafSlice slice, Weight weight,
|
||||||
Filter filter, int nDocs, HitQueue hq) {
|
Filter filter, ScoreDoc after, int nDocs, HitQueue hq) {
|
||||||
this.lock = lock;
|
this.lock = lock;
|
||||||
this.searcher = searcher;
|
this.searcher = searcher;
|
||||||
this.weight = weight;
|
this.weight = weight;
|
||||||
this.filter = filter;
|
this.filter = filter;
|
||||||
|
this.after = after;
|
||||||
this.nDocs = nDocs;
|
this.nDocs = nDocs;
|
||||||
this.hq = hq;
|
this.hq = hq;
|
||||||
this.slice = slice;
|
this.slice = slice;
|
||||||
}
|
}
|
||||||
|
|
||||||
public TopDocs call() throws IOException {
|
public TopDocs call() throws IOException {
|
||||||
final TopDocs docs = searcher.search (slice.leaves, weight, filter, nDocs);
|
final TopDocs docs = searcher.search (slice.leaves, weight, filter, after, nDocs);
|
||||||
final ScoreDoc[] scoreDocs = docs.scoreDocs;
|
final ScoreDoc[] scoreDocs = docs.scoreDocs;
|
||||||
for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
|
for (int j = 0; j < scoreDocs.length; j++) { // merge scoreDocs into hq
|
||||||
final ScoreDoc scoreDoc = scoreDocs[j];
|
final ScoreDoc scoreDoc = scoreDocs[j];
|
||||||
|
|
|
@ -77,12 +77,20 @@ public abstract class TopDocsCollector<T extends ScoreDoc> extends Collector {
|
||||||
return totalHits;
|
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. */
|
/** Returns the top docs that were collected by this collector. */
|
||||||
public TopDocs topDocs() {
|
public TopDocs topDocs() {
|
||||||
// In case pq was populated with sentinel values, there might be less
|
// In case pq was populated with sentinel values, there might be less
|
||||||
// results than pq.size(). Therefore return all results until either
|
// results than pq.size(). Therefore return all results until either
|
||||||
// pq.size() or totalHits.
|
// pq.size() or totalHits.
|
||||||
return topDocs(0, totalHits < pq.size() ? totalHits : pq.size());
|
return topDocs(0, topDocsSize());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -101,7 +109,7 @@ public abstract class TopDocsCollector<T extends ScoreDoc> extends Collector {
|
||||||
// In case pq was populated with sentinel values, there might be less
|
// In case pq was populated with sentinel values, there might be less
|
||||||
// results than pq.size(). Therefore return all results until either
|
// results than pq.size(). Therefore return all results until either
|
||||||
// pq.size() or totalHits.
|
// pq.size() or totalHits.
|
||||||
return topDocs(start, totalHits < pq.size() ? totalHits : pq.size());
|
return topDocs(start, topDocsSize());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -123,10 +131,12 @@ public abstract class TopDocsCollector<T extends ScoreDoc> extends Collector {
|
||||||
// In case pq was populated with sentinel values, there might be less
|
// In case pq was populated with sentinel values, there might be less
|
||||||
// results than pq.size(). Therefore return all results until either
|
// results than pq.size(). Therefore return all results until either
|
||||||
// pq.size() or totalHits.
|
// 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
|
// Don't bother to throw an exception, just return an empty TopDocs in case
|
||||||
// the parameters are invalid or out of range.
|
// 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) {
|
if (start < 0 || start >= size || howMany <= 0) {
|
||||||
return newTopDocs(null, start);
|
return newTopDocs(null, start);
|
||||||
}
|
}
|
||||||
|
|
|
@ -68,6 +68,67 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Assumes docs are scored in order.
|
||||||
|
private static class InOrderPagingScoreDocCollector extends TopScoreDocCollector {
|
||||||
|
private final ScoreDoc after;
|
||||||
|
// this is always after.doc - docBase, to save an add when score == after.score
|
||||||
|
private int afterDoc;
|
||||||
|
private int collectedHits;
|
||||||
|
|
||||||
|
private InOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
|
||||||
|
super(numHits);
|
||||||
|
this.after = after;
|
||||||
|
}
|
||||||
|
|
||||||
|
@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 > after.score || (score == after.score && doc <= afterDoc)) {
|
||||||
|
// 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);
|
||||||
|
afterDoc = after.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.
|
// Assumes docs are scored out of order.
|
||||||
private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
|
private static class OutOfOrderTopScoreDocCollector extends TopScoreDocCollector {
|
||||||
private OutOfOrderTopScoreDocCollector(int numHits) {
|
private OutOfOrderTopScoreDocCollector(int numHits) {
|
||||||
|
@ -102,6 +163,67 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Assumes docs are scored out of order.
|
||||||
|
private static class OutOfOrderPagingScoreDocCollector extends TopScoreDocCollector {
|
||||||
|
private final ScoreDoc after;
|
||||||
|
// this is always after.doc - docBase, to save an add when score == after.score
|
||||||
|
private int afterDoc;
|
||||||
|
private int collectedHits;
|
||||||
|
|
||||||
|
private OutOfOrderPagingScoreDocCollector(ScoreDoc after, int numHits) {
|
||||||
|
super(numHits);
|
||||||
|
this.after = after;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(int doc) throws IOException {
|
||||||
|
float score = scorer.score();
|
||||||
|
|
||||||
|
// This collector cannot handle NaN
|
||||||
|
assert !Float.isNaN(score);
|
||||||
|
|
||||||
|
totalHits++;
|
||||||
|
if (score > after.score || (score == after.score && doc <= afterDoc)) {
|
||||||
|
// 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);
|
||||||
|
afterDoc = after.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
|
* Creates a new {@link TopScoreDocCollector} given the number of hits to
|
||||||
* collect and whether documents are scored in order by the input
|
* collect and whether documents are scored in order by the input
|
||||||
|
@ -113,15 +235,33 @@ public abstract class TopScoreDocCollector extends TopDocsCollector<ScoreDoc> {
|
||||||
* objects.
|
* objects.
|
||||||
*/
|
*/
|
||||||
public static TopScoreDocCollector create(int numHits, boolean docsScoredInOrder) {
|
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 after, boolean docsScoredInOrder) {
|
||||||
|
|
||||||
if (numHits <= 0) {
|
if (numHits <= 0) {
|
||||||
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
|
throw new IllegalArgumentException("numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
|
||||||
}
|
}
|
||||||
|
|
||||||
if (docsScoredInOrder) {
|
if (docsScoredInOrder) {
|
||||||
return new InOrderTopScoreDocCollector(numHits);
|
return after == null
|
||||||
|
? new InOrderTopScoreDocCollector(numHits)
|
||||||
|
: new InOrderPagingScoreDocCollector(after, numHits);
|
||||||
} else {
|
} else {
|
||||||
return new OutOfOrderTopScoreDocCollector(numHits);
|
return after == null
|
||||||
|
? new OutOfOrderTopScoreDocCollector(numHits)
|
||||||
|
: new OutOfOrderPagingScoreDocCollector(after, numHits);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,99 @@
|
||||||
|
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 searchAfter() method
|
||||||
|
*/
|
||||||
|
public class TestSearchAfter 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);
|
||||||
|
BooleanQuery bq = new BooleanQuery();
|
||||||
|
bq.add(new TermQuery(new Term("english", "one")), BooleanClause.Occur.SHOULD);
|
||||||
|
bq.add(new TermQuery(new Term("oddeven", "even")), BooleanClause.Occur.SHOULD);
|
||||||
|
assertQuery(bq, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
void assertQuery(Query query, Filter filter) throws Exception {
|
||||||
|
TopDocs all = searcher.search(query, filter, searcher.maxDoc());
|
||||||
|
int pageSize = _TestUtil.nextInt(random, 1, searcher.maxDoc()*2);
|
||||||
|
int pageStart = 0;
|
||||||
|
ScoreDoc lastBottom = null;
|
||||||
|
while (pageStart < all.totalHits) {
|
||||||
|
TopDocs paged = searcher.searchAfter(lastBottom, query, filter, 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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -48,7 +48,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public TopDocs search(Weight weight, int topN) throws IOException {
|
public TopDocs search(Weight weight, int topN) throws IOException {
|
||||||
return search(ctx, weight, null, topN);
|
return search(ctx, weight, null, null, topN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -1034,7 +1034,7 @@ public class TestGrouping extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public TopDocs search(Weight weight, int topN) throws IOException {
|
public TopDocs search(Weight weight, int topN) throws IOException {
|
||||||
return search(ctx, weight, null, topN);
|
return search(ctx, weight, null, null, topN);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
Loading…
Reference in New Issue