mirror of https://github.com/apache/lucene.git
Replace TopFieldCollector usages in tests with collector manager (#761)
This commit replaces some usages of TopFieldCollector in tests with a corresponding collector manager created through TopFieldCollector#createSharedManager
This commit is contained in:
parent
deb6170107
commit
ea52a84c7e
|
@ -2643,14 +2643,14 @@ public class TestIndexSorting extends LuceneTestCase {
|
|||
System.out.println("TEST: iter=" + iter + " numHits=" + numHits);
|
||||
}
|
||||
|
||||
TopFieldCollector c1 = TopFieldCollector.create(sort, numHits, Integer.MAX_VALUE);
|
||||
s1.search(new MatchAllDocsQuery(), c1);
|
||||
TopDocs hits1 = c1.topDocs();
|
||||
|
||||
TopFieldCollector c2 = TopFieldCollector.create(sort, numHits, 1);
|
||||
s2.search(new MatchAllDocsQuery(), c2);
|
||||
|
||||
TopDocs hits2 = c2.topDocs();
|
||||
TopDocs hits1 =
|
||||
s1.search(
|
||||
new MatchAllDocsQuery(),
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, Integer.MAX_VALUE));
|
||||
TopDocs hits2 =
|
||||
s2.search(
|
||||
new MatchAllDocsQuery(),
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, 1));
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println(" topDocs query-time sort: totalHits=" + hits1.totalHits.value);
|
||||
|
|
|
@ -386,27 +386,30 @@ public class TestBoolean2 extends LuceneTestCase {
|
|||
}
|
||||
|
||||
// check diff (randomized) scorers (from AssertingSearcher) produce the same results
|
||||
TopFieldCollector collector = TopFieldCollector.create(sort, 1000, 1);
|
||||
searcher.search(q1, collector);
|
||||
ScoreDoc[] hits1 = collector.topDocs().scoreDocs;
|
||||
collector = TopFieldCollector.create(sort, 1000, 1);
|
||||
searcher.search(q1, collector);
|
||||
ScoreDoc[] hits2 = collector.topDocs().scoreDocs;
|
||||
ScoreDoc[] hits1 =
|
||||
searcher.search(q1, TopFieldCollector.createSharedManager(sort, 1000, null, 1))
|
||||
.scoreDocs;
|
||||
TopDocs topDocs =
|
||||
searcher.search(q1, TopFieldCollector.createSharedManager(sort, 1000, null, 1));
|
||||
ScoreDoc[] hits2 = topDocs.scoreDocs;
|
||||
CheckHits.checkEqual(q1, hits1, hits2);
|
||||
|
||||
BooleanQuery.Builder q3 = new BooleanQuery.Builder();
|
||||
q3.add(q1, BooleanClause.Occur.SHOULD);
|
||||
q3.add(new PrefixQuery(new Term("field2", "b")), BooleanClause.Occur.SHOULD);
|
||||
assertEquals(
|
||||
mulFactor * collector.totalHits + NUM_EXTRA_DOCS / 2, bigSearcher.count(q3.build()));
|
||||
mulFactor * topDocs.totalHits.value + NUM_EXTRA_DOCS / 2,
|
||||
bigSearcher.count(q3.build()));
|
||||
|
||||
// test diff (randomized) scorers produce the same results on bigSearcher as well
|
||||
collector = TopFieldCollector.create(sort, 1000 * mulFactor, 1);
|
||||
bigSearcher.search(q1, collector);
|
||||
hits1 = collector.topDocs().scoreDocs;
|
||||
collector = TopFieldCollector.create(sort, 1000 * mulFactor, 1);
|
||||
bigSearcher.search(q1, collector);
|
||||
hits2 = collector.topDocs().scoreDocs;
|
||||
hits1 =
|
||||
bigSearcher.search(
|
||||
q1, TopFieldCollector.createSharedManager(sort, 1000 * mulFactor, null, 1))
|
||||
.scoreDocs;
|
||||
hits2 =
|
||||
bigSearcher.search(
|
||||
q1, TopFieldCollector.createSharedManager(sort, 1000 * mulFactor, null, 1))
|
||||
.scoreDocs;
|
||||
CheckHits.checkEqual(q1, hits1, hits2);
|
||||
}
|
||||
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.lucene.index.IndexWriter;
|
|||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.SortedDocValues;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.FieldValueHitQueue.Entry;
|
||||
import org.apache.lucene.search.similarities.BM25Similarity;
|
||||
import org.apache.lucene.search.similarities.ClassicSimilarity;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
@ -85,10 +84,9 @@ public class TestElevationComparator extends LuceneTestCase {
|
|||
new SortField("id", new ElevationComparatorSource(priority), false),
|
||||
new SortField(null, SortField.Type.SCORE, reversed));
|
||||
|
||||
TopDocsCollector<Entry> topCollector = TopFieldCollector.create(sort, 50, Integer.MAX_VALUE);
|
||||
searcher.search(newq.build(), topCollector);
|
||||
|
||||
TopDocs topDocs = topCollector.topDocs(0, 10);
|
||||
TopDocs topDocs =
|
||||
searcher.search(
|
||||
newq.build(), TopFieldCollector.createSharedManager(sort, 50, null, Integer.MAX_VALUE));
|
||||
int nDocsReturned = topDocs.scoreDocs.length;
|
||||
|
||||
assertEquals(4, nDocsReturned);
|
||||
|
|
|
@ -61,48 +61,46 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher = newSearcher(reader, true, true, false);
|
||||
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
|
||||
final Sort sort = new Sort(sortField);
|
||||
final int numHits = 3;
|
||||
final int totalHitsThreshold = 3;
|
||||
|
||||
{ // simple sort
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
assertEquals(i, ((Long) fieldDoc.fields[0]).intValue());
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
||||
{ // paging sort with after
|
||||
long afterValue = 2;
|
||||
FieldDoc after = new FieldDoc(2, Float.NaN, new Long[] {afterValue});
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
assertEquals(afterValue + 1 + i, fieldDoc.fields[0]);
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
||||
{ // test that if there is the secondary sort on _score, scores are filled correctly
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(
|
||||
new Sort(sortField, FIELD_SCORE), numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
|
@ -110,16 +108,15 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
float score = (float) fieldDoc.fields[1];
|
||||
assertEquals(1.0, score, 0.001);
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
||||
{ // test that if numeric field is a secondary sort, no optimization is run
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(
|
||||
new Sort(FIELD_SCORE, sortField), numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
assertEquals(
|
||||
topDocs.totalHits.value,
|
||||
|
@ -146,16 +143,17 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
|
||||
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
|
||||
final Sort sort = new Sort(sortField);
|
||||
final int numHits = 3;
|
||||
final int totalHitsThreshold = 3;
|
||||
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(
|
||||
topDocs.scoreDocs.length, numHits); // sort still works and returns expected number of docs
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
|
@ -185,7 +183,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
|
||||
final int numHits = 3;
|
||||
final int totalHitsThreshold = 3;
|
||||
|
||||
|
@ -193,10 +193,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
|
||||
sortField.setMissingValue(0L); // set a competitive missing value
|
||||
final Sort sort = new Sort(sortField);
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
assertEquals(
|
||||
topDocs.totalHits.value,
|
||||
|
@ -206,10 +205,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
final SortField sortField = new SortField("my_field", SortField.Type.LONG);
|
||||
sortField.setMissingValue(100L); // set a NON competitive missing value
|
||||
final Sort sort = new Sort(sortField);
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
@ -235,17 +233,18 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
|
||||
final int numHits = 3;
|
||||
final int totalHitsThreshold = 3;
|
||||
|
||||
{ // test that sorting on a single field with equal values uses the optimization
|
||||
final SortField sortField = new SortField("my_field1", SortField.Type.INT);
|
||||
final Sort sort = new Sort(sortField);
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
|
@ -261,10 +260,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
final SortField sortField = new SortField("my_field1", SortField.Type.INT);
|
||||
final Sort sort = new Sort(sortField);
|
||||
FieldDoc after = new FieldDoc(afterDocID, Float.NaN, new Integer[] {afterValue});
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
|
@ -279,10 +277,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
final SortField sortField1 = new SortField("my_field1", SortField.Type.INT);
|
||||
final SortField sortField2 = new SortField("my_field2", SortField.Type.INT);
|
||||
final Sort sort = new Sort(sortField1, sortField2);
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
|
@ -312,23 +309,24 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
|
||||
final SortField sortField = new SortField("my_field", SortField.Type.FLOAT);
|
||||
final Sort sort = new Sort(sortField);
|
||||
final int numHits = 3;
|
||||
final int totalHitsThreshold = 3;
|
||||
|
||||
{ // simple sort
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(topDocs.scoreDocs.length, numHits);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
FieldDoc fieldDoc = (FieldDoc) topDocs.scoreDocs[i];
|
||||
assertEquals(1f * i, fieldDoc.fields[0]);
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
||||
|
@ -369,12 +367,12 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
int numHits = 0;
|
||||
do {
|
||||
for (int i = 0; i < numIndices; i++) {
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(readers[i], random().nextBoolean(), random().nextBoolean());
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, size, after, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
topDocs[i] = collector.topDocs();
|
||||
newSearcher(readers[i], random().nextBoolean(), random().nextBoolean(), false);
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, size, after, totalHitsThreshold);
|
||||
topDocs[i] = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
for (int docID = 0; docID < topDocs[i].scoreDocs.length; docID++) {
|
||||
topDocs[i].scoreDocs[docID].shardIndex = i;
|
||||
}
|
||||
|
@ -413,7 +411,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
|
||||
final int numHits = 10;
|
||||
final int totalHitsThreshold = 10;
|
||||
final int[] searchAfters = {3, 10, numDocs - 10};
|
||||
|
@ -422,10 +422,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
{
|
||||
final Sort sort = new Sort(FIELD_DOC);
|
||||
FieldDoc after = new FieldDoc(searchAfter, Float.NaN, new Integer[] {searchAfter});
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
int expNumHits =
|
||||
(searchAfter >= (numDocs - numHits)) ? (numDocs - searchAfter - 1) : numHits;
|
||||
assertEquals(expNumHits, topDocs.scoreDocs.length);
|
||||
|
@ -433,7 +432,7 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
int expectedDocID = searchAfter + 1 + i;
|
||||
assertEquals(expectedDocID, topDocs.scoreDocs[i].doc);
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
||||
|
@ -441,10 +440,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
{
|
||||
final Sort sort = new Sort(FIELD_DOC, FIELD_SCORE);
|
||||
FieldDoc after = new FieldDoc(searchAfter, Float.NaN, new Object[] {searchAfter, 1.0f});
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
int expNumHits =
|
||||
(searchAfter >= (numDocs - numHits)) ? (numDocs - searchAfter - 1) : numHits;
|
||||
assertEquals(expNumHits, topDocs.scoreDocs.length);
|
||||
|
@ -452,7 +450,7 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
int expectedDocID = searchAfter + 1 + i;
|
||||
assertEquals(expectedDocID, topDocs.scoreDocs[i].doc);
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, numDocs);
|
||||
}
|
||||
|
||||
|
@ -460,10 +458,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
{
|
||||
final Sort sort = new Sort(new SortField(null, SortField.Type.DOC, true));
|
||||
FieldDoc after = new FieldDoc(searchAfter, Float.NaN, new Integer[] {searchAfter});
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, after, totalHitsThreshold);
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
int expNumHits = (searchAfter < numHits) ? searchAfter : numHits;
|
||||
assertEquals(expNumHits, topDocs.scoreDocs.length);
|
||||
for (int i = 0; i < topDocs.scoreDocs.length; i++) {
|
||||
|
@ -533,6 +530,9 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher =
|
||||
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
|
||||
|
||||
final int numHits = 3;
|
||||
final int totalHitsThreshold = 3;
|
||||
|
@ -540,38 +540,34 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
|
||||
// sort by _doc should skip all non-competitive documents
|
||||
{
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
searcher.search(new MatchAllDocsQuery(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
|
||||
assertEquals(numHits, topDocs.scoreDocs.length);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
assertEquals(i, topDocs.scoreDocs[i].doc);
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, 10);
|
||||
}
|
||||
|
||||
// sort by _doc with a bool query should skip all non-competitive documents
|
||||
{
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
int lowerRange = 40;
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
bq.add(LongPoint.newRangeQuery("lf", lowerRange, Long.MAX_VALUE), BooleanClause.Occur.MUST);
|
||||
bq.add(new TermQuery(new Term("tf", "seg1")), BooleanClause.Occur.MUST);
|
||||
IndexSearcher searcher = newSearcher(reader, random().nextBoolean(), random().nextBoolean());
|
||||
searcher.search(bq.build(), collector);
|
||||
TopDocs topDocs = searcher.search(bq.build(), manager);
|
||||
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
assertEquals(numHits, topDocs.scoreDocs.length);
|
||||
for (int i = 0; i < numHits; i++) {
|
||||
Document d = searcher.doc(topDocs.scoreDocs[i].doc);
|
||||
assertEquals(Integer.toString(i + lowerRange), d.get("slf"));
|
||||
assertEquals("seg1", d.get("tf"));
|
||||
}
|
||||
assertTrue(collector.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value, 10);
|
||||
}
|
||||
|
||||
|
@ -608,13 +604,12 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
final Sort sort = new Sort(FIELD_DOC);
|
||||
|
||||
{
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, numHits, null, totalHitsThreshold);
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, null, totalHitsThreshold);
|
||||
BooleanQuery.Builder bq = new BooleanQuery.Builder();
|
||||
bq.add(LongPoint.newExactQuery("lf", 1), BooleanClause.Occur.MUST);
|
||||
bq.add(new TermQuery(new Term("id", "id3")), BooleanClause.Occur.MUST_NOT);
|
||||
searcher.search(bq.build(), collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
TopDocs topDocs = searcher.search(bq.build(), manager);
|
||||
assertEquals(2, topDocs.scoreDocs.length);
|
||||
}
|
||||
|
||||
|
@ -783,7 +778,8 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
}
|
||||
final IndexReader reader = DirectoryReader.open(writer);
|
||||
writer.close();
|
||||
IndexSearcher searcher = newSearcher(reader);
|
||||
// single threaded so totalhits is deterministic
|
||||
IndexSearcher searcher = newSearcher(reader, true, true, false);
|
||||
|
||||
SortedNumericSelector.Type type =
|
||||
RandomPicks.randomFrom(random(), SortedNumericSelector.Type.values());
|
||||
|
@ -807,16 +803,14 @@ public class TestSortOptimization extends LuceneTestCase {
|
|||
int batch = 1 + random().nextInt(100);
|
||||
int expectedHits = Math.min(numDocs - visitedHits, batch);
|
||||
|
||||
final TopFieldCollector collector =
|
||||
TopFieldCollector.create(sort, batch, (FieldDoc) after, totalHitsThreshold);
|
||||
searcher.search(query, collector);
|
||||
TopDocs topDocs = collector.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, batch, (FieldDoc) after, totalHitsThreshold);
|
||||
TopDocs topDocs = searcher.search(query, manager);
|
||||
ScoreDoc[] scoreDocs = topDocs.scoreDocs;
|
||||
|
||||
final TopFieldCollector collector2 =
|
||||
TopFieldCollector.create(sort2, batch, (FieldDoc) after, totalHitsThreshold);
|
||||
searcher.search(query, collector2);
|
||||
TopDocs topDocs2 = collector2.topDocs();
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager2 =
|
||||
TopFieldCollector.createSharedManager(sort2, batch, (FieldDoc) after, totalHitsThreshold);
|
||||
TopDocs topDocs2 = searcher.search(query, manager2);
|
||||
ScoreDoc[] scoreDocs2 = topDocs2.scoreDocs;
|
||||
|
||||
// assert that the resulting hits are the same
|
||||
|
|
|
@ -68,16 +68,16 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
super.tearDown();
|
||||
}
|
||||
|
||||
private TopFieldCollector doSearchWithThreshold(
|
||||
private static TopDocs doSearchWithThreshold(
|
||||
int numResults, int thresHold, Query q, Sort sort, IndexReader indexReader)
|
||||
throws IOException {
|
||||
IndexSearcher searcher = newSearcher(indexReader);
|
||||
TopFieldCollector tdc = TopFieldCollector.create(sort, numResults, thresHold);
|
||||
searcher.search(q, tdc);
|
||||
return tdc;
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
|
||||
TopFieldCollector.createSharedManager(sort, numResults, null, thresHold);
|
||||
return searcher.search(q, manager);
|
||||
}
|
||||
|
||||
private TopDocs doConcurrentSearchWithThreshold(
|
||||
private static TopDocs doConcurrentSearchWithThreshold(
|
||||
int numResults, int threshold, Query q, Sort sort, IndexReader indexReader)
|
||||
throws IOException {
|
||||
IndexSearcher searcher = newSearcher(indexReader, true, true, true);
|
||||
|
@ -100,11 +100,10 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort()};
|
||||
for (int i = 0; i < sort.length; i++) {
|
||||
Query q = new MatchAllDocsQuery();
|
||||
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, Integer.MAX_VALUE);
|
||||
|
||||
is.search(q, tdc);
|
||||
|
||||
ScoreDoc[] sd = tdc.topDocs().scoreDocs;
|
||||
ScoreDoc[] sd =
|
||||
is.search(q, TopFieldCollector.createSharedManager(sort[i], 10, null, Integer.MAX_VALUE))
|
||||
.scoreDocs;
|
||||
for (int j = 1; j < sd.length; j++) {
|
||||
assertTrue(sd[j].doc != sd[j - 1].doc);
|
||||
}
|
||||
|
@ -117,12 +116,10 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort()};
|
||||
for (int i = 0; i < sort.length; i++) {
|
||||
Query q = new MatchAllDocsQuery();
|
||||
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, Integer.MAX_VALUE);
|
||||
|
||||
is.search(q, tdc);
|
||||
|
||||
TopDocs td = tdc.topDocs();
|
||||
ScoreDoc[] sd = td.scoreDocs;
|
||||
ScoreDoc[] sd =
|
||||
is.search(q, TopFieldCollector.createSharedManager(sort[i], 10, null, Integer.MAX_VALUE))
|
||||
.scoreDocs;
|
||||
for (int j = 0; j < sd.length; j++) {
|
||||
assertTrue(Float.isNaN(sd[j].score));
|
||||
}
|
||||
|
@ -131,19 +128,20 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
|
||||
public void testSharedHitcountCollector() throws Exception {
|
||||
IndexSearcher concurrentSearcher = newSearcher(ir, true, true, true);
|
||||
IndexSearcher singleThreadedSearcher = newSearcher(ir, true, true, false);
|
||||
|
||||
// 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++) {
|
||||
Query q = new MatchAllDocsQuery();
|
||||
TopDocsCollector<Entry> tdc = TopFieldCollector.create(sort[i], 10, Integer.MAX_VALUE);
|
||||
|
||||
is.search(q, tdc);
|
||||
TopDocs td =
|
||||
singleThreadedSearcher.search(
|
||||
q, TopFieldCollector.createSharedManager(sort[i], 10, null, Integer.MAX_VALUE));
|
||||
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> tsdc =
|
||||
TopFieldCollector.createSharedManager(sort[i], 10, null, Integer.MAX_VALUE);
|
||||
|
||||
TopDocs td = tdc.topDocs();
|
||||
TopDocs td2 = concurrentSearcher.search(q, tsdc);
|
||||
ScoreDoc[] sd = td.scoreDocs;
|
||||
for (int j = 0; j < sd.length; j++) {
|
||||
|
@ -160,17 +158,16 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
Query q = new MatchAllDocsQuery();
|
||||
// check that setting trackTotalHits to false does not throw an NPE because
|
||||
// the index is not sorted
|
||||
TopDocsCollector<Entry> tdc;
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager;
|
||||
if (i % 2 == 0) {
|
||||
tdc = TopFieldCollector.create(sort, 10, 1);
|
||||
manager = TopFieldCollector.createSharedManager(sort, 10, null, 1);
|
||||
} else {
|
||||
FieldDoc fieldDoc = new FieldDoc(1, Float.NaN, new Object[] {1});
|
||||
tdc = TopFieldCollector.create(sort, 10, fieldDoc, 1);
|
||||
manager = TopFieldCollector.createSharedManager(sort, 10, fieldDoc, 1);
|
||||
}
|
||||
|
||||
is.search(q, tdc);
|
||||
TopDocs td = is.search(q, manager);
|
||||
|
||||
TopDocs td = tdc.topDocs();
|
||||
ScoreDoc[] sd = td.scoreDocs;
|
||||
for (int j = 0; j < sd.length; j++) {
|
||||
assertTrue(Float.isNaN(sd[j].score));
|
||||
|
@ -692,9 +689,8 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
};
|
||||
for (Query query : queries) {
|
||||
Sort sort = new Sort(new SortField[] {SortField.FIELD_SCORE, SortField.FIELD_DOC});
|
||||
TopFieldCollector fieldCollector = doSearchWithThreshold(5, 0, query, sort, indexReader);
|
||||
TopDocs tdc2 = doSearchWithThreshold(5, 0, query, sort, indexReader);
|
||||
TopDocs tdc = doConcurrentSearchWithThreshold(5, 0, query, sort, indexReader);
|
||||
TopDocs tdc2 = fieldCollector.topDocs();
|
||||
|
||||
assertTrue(tdc.totalHits.value > 0);
|
||||
assertTrue(tdc2.totalHits.value > 0);
|
||||
|
@ -719,20 +715,26 @@ public class TestTopFieldCollector extends LuceneTestCase {
|
|||
|
||||
try (IndexReader reader = DirectoryReader.open(w)) {
|
||||
IndexSearcher searcher = new IndexSearcher(reader);
|
||||
TopFieldCollector collector = TopFieldCollector.create(sort, 2, 10);
|
||||
searcher.search(new TermQuery(new Term("f", "foo")), collector);
|
||||
assertEquals(10, collector.totalHits);
|
||||
assertEquals(TotalHits.Relation.EQUAL_TO, collector.totalHitsRelation);
|
||||
TopFieldDocs topDocs =
|
||||
searcher.search(
|
||||
new TermQuery(new Term("f", "foo")),
|
||||
TopFieldCollector.createSharedManager(sort, 2, null, 10));
|
||||
assertEquals(10, topDocs.totalHits.value);
|
||||
assertEquals(TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation);
|
||||
|
||||
collector = TopFieldCollector.create(sort, 2, 2);
|
||||
searcher.search(new TermQuery(new Term("f", "foo")), collector);
|
||||
assertTrue(10 >= collector.totalHits);
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, collector.totalHitsRelation);
|
||||
topDocs =
|
||||
searcher.search(
|
||||
new TermQuery(new Term("f", "foo")),
|
||||
TopFieldCollector.createSharedManager(sort, 2, null, 2));
|
||||
assertTrue(10 >= topDocs.totalHits.value);
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation);
|
||||
|
||||
collector = TopFieldCollector.create(sort, 10, 2);
|
||||
searcher.search(new TermQuery(new Term("f", "foo")), collector);
|
||||
assertEquals(10, collector.totalHits);
|
||||
assertEquals(TotalHits.Relation.EQUAL_TO, collector.totalHitsRelation);
|
||||
topDocs =
|
||||
searcher.search(
|
||||
new TermQuery(new Term("f", "foo")),
|
||||
TopFieldCollector.createSharedManager(sort, 10, null, 2));
|
||||
assertEquals(10, topDocs.totalHits.value);
|
||||
assertEquals(TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -134,9 +134,10 @@ public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase {
|
|||
} else {
|
||||
after = null;
|
||||
}
|
||||
final TopFieldCollector collector1 =
|
||||
TopFieldCollector.create(sort, numHits, after, Integer.MAX_VALUE);
|
||||
final TopFieldCollector collector2 = TopFieldCollector.create(sort, numHits, after, 1);
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager1 =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, Integer.MAX_VALUE);
|
||||
CollectorManager<TopFieldCollector, TopFieldDocs> manager2 =
|
||||
TopFieldCollector.createSharedManager(sort, numHits, after, 1);
|
||||
|
||||
final Query query;
|
||||
if (random().nextBoolean()) {
|
||||
|
@ -144,17 +145,15 @@ public class TestTopFieldCollectorEarlyTermination extends LuceneTestCase {
|
|||
} else {
|
||||
query = new MatchAllDocsQuery();
|
||||
}
|
||||
searcher.search(query, collector1);
|
||||
searcher.search(query, collector2);
|
||||
TopDocs td1 = collector1.topDocs();
|
||||
TopDocs td2 = collector2.topDocs();
|
||||
TopDocs td1 = searcher.search(query, manager1);
|
||||
TopDocs td2 = searcher.search(query, manager2);
|
||||
|
||||
assertFalse(collector1.isEarlyTerminated());
|
||||
assertNotEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, td1.totalHits.relation);
|
||||
if (paging == false && maxSegmentSize > numHits && query instanceof MatchAllDocsQuery) {
|
||||
// Make sure that we sometimes early terminate
|
||||
assertTrue(collector2.isEarlyTerminated());
|
||||
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, td2.totalHits.relation);
|
||||
}
|
||||
if (collector2.isEarlyTerminated()) {
|
||||
if (td2.totalHits.relation == TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO) {
|
||||
assertTrue(td2.totalHits.value >= td1.scoreDocs.length);
|
||||
assertTrue(td2.totalHits.value <= reader.maxDoc());
|
||||
} else {
|
||||
|
|
Loading…
Reference in New Issue