Remove `supportsConcurrency` flag from top-docs collector managers. (#13977)

Our collector managers have a `supportsConcurrency` flag to optimize the case
when they are used in a single thread. This PR proposes to remove this flag now
that the optimization doesn't do much as a result of #13943.
This commit is contained in:
Adrien Grand 2024-11-05 15:36:40 +01:00
parent 7f50732cc3
commit 48026bbc25
23 changed files with 91 additions and 137 deletions

View File

@ -112,8 +112,7 @@ public abstract class ReadTask extends PerfTask {
// pulling the Weight ourselves:
int totalHitsThreshold = withTotalHits() ? Integer.MAX_VALUE : 1;
TopFieldCollectorManager collectorManager =
new TopFieldCollectorManager(
sort, numHits, null, totalHitsThreshold, searcher.getSlices().length > 1);
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold);
hits = searcher.search(q, collectorManager);
} else {
hits = searcher.search(q, numHits);

View File

@ -564,10 +564,8 @@ public class IndexSearcher {
}
final int cappedNumHits = Math.min(numHits, limit);
final boolean supportsConcurrency = getSlices().length > 1;
CollectorManager<TopScoreDocCollector, TopDocs> manager =
new TopScoreDocCollectorManager(
cappedNumHits, after, TOTAL_HITS_THRESHOLD, supportsConcurrency);
new TopScoreDocCollectorManager(cappedNumHits, after, TOTAL_HITS_THRESHOLD);
return search(query, manager);
}
@ -699,12 +697,9 @@ public class IndexSearcher {
}
final int cappedNumHits = Math.min(numHits, limit);
final Sort rewrittenSort = sort.rewrite(this);
final LeafSlice[] leafSlices = getSlices();
final boolean supportsConcurrency = leafSlices.length > 1;
final CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(
rewrittenSort, cappedNumHits, after, TOTAL_HITS_THRESHOLD, supportsConcurrency);
new TopFieldCollectorManager(rewrittenSort, cappedNumHits, after, TOTAL_HITS_THRESHOLD);
TopFieldDocs topDocs = search(query, manager);
if (doDocScores) {

View File

@ -46,9 +46,7 @@ public class SortRescorer extends Rescorer {
List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();
TopFieldCollector collector =
new TopFieldCollectorManager(
sort, topN, null, Integer.MAX_VALUE, searcher.getSlices().length > 1)
.newCollector();
new TopFieldCollectorManager(sort, topN, null, Integer.MAX_VALUE).newCollector();
// Now merge sort docIDs from hits, with reader's leaves:
int hitUpto = 0;

View File

@ -35,8 +35,6 @@ public class TopFieldCollectorManager implements CollectorManager<TopFieldCollec
private final int totalHitsThreshold;
private final MaxScoreAccumulator minScoreAcc;
private final List<TopFieldCollector> collectors;
private final boolean supportsConcurrency;
private boolean collectorCreated;
/**
* Creates a new {@link TopFieldCollectorManager} from the given arguments.
@ -53,9 +51,32 @@ public class TopFieldCollectorManager implements CollectorManager<TopFieldCollec
* count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
* count accurate, but this will also make query processing slower.
* @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
* @deprecated Use {@link #TopFieldCollectorManager(Sort, int, FieldDoc, int)}, the
* supportsConcurrency parameter is now a no-op.
*/
@Deprecated
public TopFieldCollectorManager(
Sort sort, int numHits, FieldDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
this(sort, numHits, after, totalHitsThreshold);
}
/**
* Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
* internal states.
*
* <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
* <code>numHits</code>.
*
* @param sort the sort criteria (SortFields).
* @param numHits the number of results to collect.
* @param after the previous doc after which matching docs will be collected.
* @param totalHitsThreshold the number of docs to count accurately. If the query matches more
* than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
* hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
* count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
* count accurate, but this will also make query processing slower.
*/
public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
if (totalHitsThreshold < 0) {
throw new IllegalArgumentException(
"totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
@ -88,35 +109,11 @@ public class TopFieldCollectorManager implements CollectorManager<TopFieldCollec
this.sort = sort;
this.numHits = numHits;
this.after = after;
this.supportsConcurrency = supportsConcurrency;
this.totalHitsThreshold = totalHitsThreshold;
this.minScoreAcc =
supportsConcurrency && totalHitsThreshold != Integer.MAX_VALUE
? new MaxScoreAccumulator()
: null;
this.minScoreAcc = totalHitsThreshold != Integer.MAX_VALUE ? new MaxScoreAccumulator() : null;
this.collectors = new ArrayList<>();
}
/**
* Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
* internal states.
*
* <p><b>NOTE</b>: The instances returned by this method pre-allocate a full array of length
* <code>numHits</code>.
*
* @param sort the sort criteria (SortFields).
* @param numHits the number of results to collect.
* @param after the previous doc after which matching docs will be collected.
* @param totalHitsThreshold the number of docs to count accurately. If the query matches more
* than {@code totalHitsThreshold} hits then its hit count will be a lower bound. On the other
* hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
* count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
* count accurate, but this will also make query processing slower.
*/
public TopFieldCollectorManager(Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
this(sort, numHits, after, totalHitsThreshold, true);
}
/**
* Creates a new {@link TopFieldCollectorManager} from the given arguments, with thread-safe
* internal states.
@ -138,13 +135,6 @@ public class TopFieldCollectorManager implements CollectorManager<TopFieldCollec
@Override
public TopFieldCollector newCollector() {
if (collectorCreated && supportsConcurrency == false) {
throw new IllegalStateException(
"This TopFieldCollectorManager was created without concurrency (supportsConcurrency=false), but multiple collectors are being created");
} else {
collectorCreated = true;
}
FieldValueHitQueue<FieldValueHitQueue.Entry> queue =
FieldValueHitQueue.create(sort.getSort(), numHits);

View File

@ -31,8 +31,6 @@ public class TopScoreDocCollectorManager
private final ScoreDoc after;
private final int totalHitsThreshold;
private final MaxScoreAccumulator minScoreAcc;
private final boolean supportsConcurrency;
private boolean collectorCreated;
/**
* Creates a new {@link TopScoreDocCollectorManager} given the number of hits to collect and the
@ -54,28 +52,13 @@ public class TopScoreDocCollectorManager
* hand if the query matches less than or exactly {@code totalHitsThreshold} hits then the hit
* count of the result will be accurate. {@link Integer#MAX_VALUE} may be used to make the hit
* count accurate, but this will also make query processing slower.
* @param supportsConcurrency to use thread-safe and slower internal states for count tracking.
* @deprecated Use {@link #TopScoreDocCollectorManager(int, ScoreDoc, int)}, the
* supportsConcurrency parameter is now a no-op.
*/
@Deprecated
public TopScoreDocCollectorManager(
int numHits, ScoreDoc after, int totalHitsThreshold, boolean supportsConcurrency) {
if (totalHitsThreshold < 0) {
throw new IllegalArgumentException(
"totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
}
if (numHits <= 0) {
throw new IllegalArgumentException(
"numHits must be > 0; please use TotalHitCountCollectorManager if you just need the total hit count");
}
this.numHits = numHits;
this.after = after;
this.supportsConcurrency = supportsConcurrency;
this.totalHitsThreshold = Math.max(totalHitsThreshold, numHits);
this.minScoreAcc =
supportsConcurrency && totalHitsThreshold != Integer.MAX_VALUE
? new MaxScoreAccumulator()
: null;
this(numHits, after, totalHitsThreshold);
}
/**
@ -100,7 +83,20 @@ public class TopScoreDocCollectorManager
* count accurate, but this will also make query processing slower.
*/
public TopScoreDocCollectorManager(int numHits, ScoreDoc after, int totalHitsThreshold) {
this(numHits, after, totalHitsThreshold, true);
if (totalHitsThreshold < 0) {
throw new IllegalArgumentException(
"totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
}
if (numHits <= 0) {
throw new IllegalArgumentException(
"numHits must be > 0; please use TotalHitCountCollectorManager if you just need the total hit count");
}
this.numHits = numHits;
this.after = after;
this.totalHitsThreshold = Math.max(totalHitsThreshold, numHits);
this.minScoreAcc = totalHitsThreshold != Integer.MAX_VALUE ? new MaxScoreAccumulator() : null;
}
/**
@ -129,13 +125,6 @@ public class TopScoreDocCollectorManager
@Override
public TopScoreDocCollector newCollector() {
if (collectorCreated && supportsConcurrency == false) {
throw new IllegalStateException(
"This TopScoreDocCollectorManager was created without concurrency (supportsConcurrency=false), but multiple collectors are being created");
} else {
collectorCreated = true;
}
if (after == null) {
return new TopScoreDocCollector.SimpleTopScoreDocCollector(
numHits, totalHitsThreshold, minScoreAcc);

View File

@ -67,7 +67,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
assertEquals(IndexWriter.MAX_DOCS, ir.numDocs());
IndexSearcher searcher = new IndexSearcher(ir);
TopScoreDocCollectorManager collectorManager =
new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, true);
new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE);
TopDocs hits = searcher.search(new TermQuery(new Term("field", "text")), collectorManager);
assertEquals(IndexWriter.MAX_DOCS, hits.totalHits.value());

View File

@ -155,7 +155,7 @@ public class TestBooleanQueryVisitSubscorers extends LuceneTestCase {
private final Set<Scorer> tqsSet = new HashSet<>();
MyCollector() {
super(new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector());
super(new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE).newCollector());
}
@Override

View File

@ -98,7 +98,7 @@ public class TestElevationComparator extends LuceneTestCase {
TopDocs topDocs =
searcher.search(
newq.build(), new TopFieldCollectorManager(sort, 50, null, Integer.MAX_VALUE, true));
newq.build(), new TopFieldCollectorManager(sort, 50, null, Integer.MAX_VALUE));
int nDocsReturned = topDocs.scoreDocs.length;
assertEquals(4, nDocsReturned);

View File

@ -81,8 +81,7 @@ public class TestNeedsScores extends LuceneTestCase {
assertEquals(5, searcher.count(constantScore));
TopDocs hits =
searcher.search(
constantScore, new TopScoreDocCollectorManager(5, null, Integer.MAX_VALUE, true));
searcher.search(constantScore, new TopScoreDocCollectorManager(5, null, Integer.MAX_VALUE));
assertEquals(5, hits.totalHits.value());
// Queries that support dynamic pruning like top-score or top-doc queries that do not compute

View File

@ -110,8 +110,7 @@ public class TestPositiveScoresOnlyCollector extends LuceneTestCase {
writer.close();
Scorer s = new SimpleScorer();
TopDocsCollector<ScoreDoc> tdc =
new TopScoreDocCollectorManager(scores.length, null, Integer.MAX_VALUE, false)
.newCollector();
new TopScoreDocCollectorManager(scores.length, null, Integer.MAX_VALUE).newCollector();
Collector c = new PositiveScoresOnlyCollector(tdc);
LeafCollector ac = c.getLeafCollector(ir.leaves().get(0));
ac.setScorer(s);

View File

@ -234,10 +234,10 @@ public class TestSearchAfter extends LuceneTestCase {
allManager = new TopScoreDocCollectorManager(maxDoc, null, Integer.MAX_VALUE);
doScores = false;
} else if (sort == Sort.RELEVANCE) {
allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE, true);
allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE);
doScores = true;
} else {
allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE, true);
allManager = new TopFieldCollectorManager(sort, maxDoc, null, Integer.MAX_VALUE);
doScores = random().nextBoolean();
}
all = searcher.search(query, allManager);
@ -268,15 +268,13 @@ public class TestSearchAfter extends LuceneTestCase {
if (VERBOSE) {
System.out.println(" iter lastBottom=" + lastBottom);
}
pagedManager =
new TopScoreDocCollectorManager(pageSize, lastBottom, Integer.MAX_VALUE, true);
pagedManager = new TopScoreDocCollectorManager(pageSize, lastBottom, Integer.MAX_VALUE);
} else {
if (VERBOSE) {
System.out.println(" iter lastBottom=" + lastBottom);
}
pagedManager =
new TopFieldCollectorManager(
sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE, true);
new TopFieldCollectorManager(sort, pageSize, (FieldDoc) lastBottom, Integer.MAX_VALUE);
}
paged = searcher.search(query, pagedManager);
if (doScores) {

View File

@ -237,7 +237,7 @@ public class TestSortOptimization extends LuceneTestCase {
sortField2.setMissingValue(0L); // set a competitive missing value
final Sort sort = new Sort(sortField1, sortField2);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold);
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
assertEquals(topDocs.scoreDocs.length, numHits);
assertEquals(
@ -264,7 +264,7 @@ public class TestSortOptimization extends LuceneTestCase {
sortField.setMissingValue(Long.MAX_VALUE); // set a competitive missing value
final Sort sort = new Sort(sortField);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold);
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
assertEquals(topDocs.scoreDocs.length, numHits);
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value(), numDocs);
@ -279,7 +279,7 @@ public class TestSortOptimization extends LuceneTestCase {
sortField.setMissingValue(Long.MAX_VALUE); // set a competitive missing value
final Sort sort = new Sort(sortField);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold);
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
assertEquals(topDocs.scoreDocs.length, numHits);
assertNonCompetitiveHitsAreSkipped(topDocs.totalHits.value(), numDocs);
@ -347,7 +347,7 @@ public class TestSortOptimization extends LuceneTestCase {
sortField.setMissingValue(0L); // missing value is not competitive
final Sort sort = new Sort(sortField);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold);
topDocs1 = searcher.search(new MatchAllDocsQuery(), manager);
assertNonCompetitiveHitsAreSkipped(topDocs1.totalHits.value(), numDocs);
}
@ -358,7 +358,7 @@ public class TestSortOptimization extends LuceneTestCase {
final Sort sort = new Sort(sortField);
sortField.setOptimizeSortWithPoints(false);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold);
topDocs2 = searcher.search(new MatchAllDocsQuery(), manager);
// assert that the resulting hits are the same
assertEquals(topDocs1.scoreDocs.length, topDocs2.scoreDocs.length);
@ -381,7 +381,7 @@ public class TestSortOptimization extends LuceneTestCase {
sortField2.setMissingValue(0L); // missing value is not competitive
final Sort multiSorts = new Sort(new SortField[] {sortField1, sortField2});
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(multiSorts, numHits, null, totalHitsThreshold, true);
new TopFieldCollectorManager(multiSorts, numHits, null, totalHitsThreshold);
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
// can't optimization with NumericDocValues when there are multiple comparators
assertEquals(topDocs.totalHits.value(), numDocs);
@ -959,7 +959,7 @@ public class TestSortOptimization extends LuceneTestCase {
// test search
int numHits = 1 + random().nextInt(100);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(new Sort(sortField), numHits, null, numHits, true);
new TopFieldCollectorManager(new Sort(sortField), numHits, null, numHits);
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
for (int i = 0; i < topDocs.scoreDocs.length; i++) {
long expectedSeqNo = seqNos.get(i);
@ -1010,12 +1010,12 @@ public class TestSortOptimization extends LuceneTestCase {
int expectedHits = Math.min(numDocs - visitedHits, batch);
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, batch, (FieldDoc) after, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, batch, (FieldDoc) after, totalHitsThreshold);
TopDocs topDocs = searcher.search(query, manager);
ScoreDoc[] scoreDocs = topDocs.scoreDocs;
CollectorManager<TopFieldCollector, TopFieldDocs> manager2 =
new TopFieldCollectorManager(sort2, batch, (FieldDoc) after, totalHitsThreshold, true);
new TopFieldCollectorManager(sort2, batch, (FieldDoc) after, totalHitsThreshold);
TopDocs topDocs2 = searcher.search(query, manager2);
ScoreDoc[] scoreDocs2 = topDocs2.scoreDocs;
@ -1210,7 +1210,7 @@ public class TestSortOptimization extends LuceneTestCase {
final int totalHitsThreshold = 5;
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, true);
new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold);
IndexSearcher searcher =
newSearcher(reader, random().nextBoolean(), random().nextBoolean(), false);
TopDocs topDocs = searcher.search(new MatchAllDocsQuery(), manager);
@ -1240,7 +1240,7 @@ public class TestSortOptimization extends LuceneTestCase {
IndexSearcher searcher = newSearcher(reader, true, true, false);
Query query = new MatchAllDocsQuery();
CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new TopFieldCollectorManager(sort, n, after, n, true);
new TopFieldCollectorManager(sort, n, after, n);
TopDocs topDocs = searcher.search(query, manager);
IndexSearcher unoptimizedSearcher =
newSearcher(new NoIndexDirectoryReader(reader), true, true, false);

View File

@ -160,8 +160,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
int numResults, int thresHold, Query q, IndexReader indexReader) throws IOException {
IndexSearcher searcher = newSearcher(indexReader, true, true, false);
TopScoreDocCollectorManager collectorManager =
new TopScoreDocCollectorManager(
numResults, null, thresHold, searcher.getSlices().length > 1);
new TopScoreDocCollectorManager(numResults, null, thresHold);
return searcher.search(q, collectorManager);
}
@ -169,8 +168,7 @@ public class TestTopDocsCollector extends LuceneTestCase {
int numResults, int threshold, Query q, IndexReader indexReader) throws IOException {
IndexSearcher searcher = newSearcher(indexReader, true, true, true);
TopScoreDocCollectorManager collectorManager =
new TopScoreDocCollectorManager(
numResults, null, threshold, searcher.getSlices().length > 1);
new TopScoreDocCollectorManager(numResults, null, threshold);
return searcher.search(q, collectorManager);
}

View File

@ -56,7 +56,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
public TopDocs search(Weight weight, int topN) throws IOException {
TopScoreDocCollector collector =
new TopScoreDocCollectorManager(topN, null, Integer.MAX_VALUE, false).newCollector();
new TopScoreDocCollectorManager(topN, null, Integer.MAX_VALUE).newCollector();
searchLeaf(ctx, 0, DocIdSetIterator.NO_MORE_DOCS, weight, collector);
return collector.topDocs();
}
@ -348,8 +348,7 @@ public class TestTopDocsMerge extends LuceneTestCase {
subHits = subSearcher.search(w, numHits);
} else {
final TopFieldCollector c =
new TopFieldCollectorManager(sort, numHits, null, Integer.MAX_VALUE, false)
.newCollector();
new TopFieldCollectorManager(sort, numHits, null, Integer.MAX_VALUE).newCollector();
subSearcher.search(w, c);
subHits = c.topDocs(0, numHits);
}

View File

@ -76,8 +76,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
throws IOException {
IndexSearcher searcher = newSearcher(indexReader);
TopFieldCollectorManager manager =
new TopFieldCollectorManager(
sort, numResults, null, thresHold, searcher.getSlices().length > 1);
new TopFieldCollectorManager(sort, numResults, null, thresHold);
return searcher.search(q, manager);
}
@ -87,8 +86,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
IndexSearcher searcher = newSearcher(indexReader, true, true, true);
TopFieldCollectorManager collectorManager =
new TopFieldCollectorManager(
sort, numResults, null, threshold, searcher.getSlices().length > 1);
new TopFieldCollectorManager(sort, numResults, null, threshold);
TopDocs topDoc = searcher.search(q, collectorManager);
@ -122,7 +120,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
for (int i = 0; i < sort.length; i++) {
Query q = new MatchAllDocsQuery();
TopFieldCollectorManager tdc =
new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE, false);
new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE);
TopDocs td = is.search(q, tdc);
ScoreDoc[] sd = td.scoreDocs;
for (int j = 0; j < sd.length; j++) {
@ -384,7 +382,7 @@ public class TestTopFieldCollector extends LuceneTestCase {
Sort[] sort = new Sort[] {new Sort(SortField.FIELD_DOC), new Sort()};
for (int i = 0; i < sort.length; i++) {
TopDocsCollector<Entry> tdc =
new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE, false).newCollector();
new TopFieldCollectorManager(sort[i], 10, null, Integer.MAX_VALUE).newCollector();
TopDocs td = tdc.topDocs();
assertEquals(0, td.totalHits.value());
}
@ -716,18 +714,17 @@ public class TestTopFieldCollector extends LuceneTestCase {
try (IndexReader reader = DirectoryReader.open(w)) {
IndexSearcher searcher = new IndexSearcher(reader);
TopFieldCollectorManager collectorManager =
new TopFieldCollectorManager(sort, 2, null, 10, true);
TopFieldCollectorManager collectorManager = new TopFieldCollectorManager(sort, 2, null, 10);
TopDocs topDocs = searcher.search(new TermQuery(new Term("f", "foo")), collectorManager);
assertEquals(10, topDocs.totalHits.value());
assertEquals(TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation());
collectorManager = new TopFieldCollectorManager(sort, 2, null, 2, true);
collectorManager = new TopFieldCollectorManager(sort, 2, null, 2);
topDocs = searcher.search(new TermQuery(new Term("f", "foo")), collectorManager);
assertTrue(10 >= topDocs.totalHits.value());
assertEquals(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO, topDocs.totalHits.relation());
collectorManager = new TopFieldCollectorManager(sort, 10, null, 2, true);
collectorManager = new TopFieldCollectorManager(sort, 10, null, 2);
topDocs = searcher.search(new TermQuery(new Term("f", "foo")), collectorManager);
assertEquals(10, topDocs.totalHits.value());
assertEquals(TotalHits.Relation.EQUAL_TO, topDocs.totalHits.relation());

View File

@ -197,9 +197,8 @@ public class DrillSideways {
limit = 1; // the collector does not alow numHits = 0
}
final int fTopN = Math.min(topN, limit);
final boolean supportsConcurrency = searcher.getSlices().length > 1;
final TopFieldCollectorManager collectorManager =
new TopFieldCollectorManager(sort, fTopN, after, Integer.MAX_VALUE, supportsConcurrency);
new TopFieldCollectorManager(sort, fTopN, after, Integer.MAX_VALUE);
final ConcurrentDrillSidewaysResult<TopFieldDocs> r = search(query, collectorManager);
TopFieldDocs topDocs = r.collectorResult;
@ -230,9 +229,8 @@ public class DrillSideways {
limit = 1; // the collector does not alow numHits = 0
}
final int fTopN = Math.min(topN, limit);
final boolean supportsConcurrency = searcher.getSlices().length > 1;
final TopScoreDocCollectorManager collectorManager =
new TopScoreDocCollectorManager(fTopN, after, Integer.MAX_VALUE, supportsConcurrency);
new TopScoreDocCollectorManager(fTopN, after, Integer.MAX_VALUE);
final ConcurrentDrillSidewaysResult<TopDocs> r = search(query, collectorManager);
return new DrillSidewaysResult(
r.facets,

View File

@ -285,11 +285,11 @@ public class FacetsCollectorManager implements CollectorManager<FacetsCollector,
throw new IllegalArgumentException("after must be a FieldDoc; got " + after);
}
TopFieldCollectorManager topFieldCollectorManager =
new TopFieldCollectorManager(sort, n, (FieldDoc) after, Integer.MAX_VALUE, true);
new TopFieldCollectorManager(sort, n, (FieldDoc) after, Integer.MAX_VALUE);
multiCollectorManager = new MultiCollectorManager(topFieldCollectorManager, fcm);
} else {
TopScoreDocCollectorManager topScoreDocCollectorManager =
new TopScoreDocCollectorManager(n, after, Integer.MAX_VALUE, true);
new TopScoreDocCollectorManager(n, after, Integer.MAX_VALUE);
multiCollectorManager = new MultiCollectorManager(topScoreDocCollectorManager, fcm);
}
Object[] result = searcher.search(q, multiCollectorManager);

View File

@ -295,13 +295,12 @@ public class BlockGroupingCollector extends SimpleCollector {
"cannot sort by relevance within group: needsScores=false");
}
collector =
new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE, false)
new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE)
.newCollector();
} else {
// Sort by fields
collector =
new TopFieldCollectorManager(
withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE, false)
new TopFieldCollectorManager(withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE)
.newCollector(); // TODO: disable exact counts?
}

View File

@ -129,7 +129,7 @@ public class TopGroupsCollector<T> extends SecondPassGroupingCollector<T> {
() ->
new TopDocsAndMaxScoreCollector(
true,
new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE, false)
new TopScoreDocCollectorManager(maxDocsPerGroup, null, Integer.MAX_VALUE)
.newCollector(),
null);
} else {
@ -137,7 +137,7 @@ public class TopGroupsCollector<T> extends SecondPassGroupingCollector<T> {
() -> {
TopFieldCollector topDocsCollector =
new TopFieldCollectorManager(
withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE, false)
withinGroupSort, maxDocsPerGroup, null, Integer.MAX_VALUE)
.newCollector(); // TODO: disable exact counts?
MaxScoreCollector maxScoreCollector = getMaxScores ? new MaxScoreCollector() : null;
return new TopDocsAndMaxScoreCollector(false, topDocsCollector, maxScoreCollector);

View File

@ -468,7 +468,7 @@ public class TestJoinUtil extends LuceneTestCase {
final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
final TopScoreDocCollector topScoreDocCollector =
new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector();
new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE).newCollector();
indexSearcher.search(
joinQuery, MultiCollector.wrap(new BitSetCollector(actualResult), topScoreDocCollector));
assertBitSet(expectedResult, actualResult, indexSearcher);
@ -1546,7 +1546,7 @@ public class TestJoinUtil extends LuceneTestCase {
// be also testing TopDocsCollector...
final BitSet actualResult = new FixedBitSet(indexSearcher.getIndexReader().maxDoc());
final TopScoreDocCollector topScoreDocCollector =
new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE, false).newCollector();
new TopScoreDocCollectorManager(10, null, Integer.MAX_VALUE).newCollector();
indexSearcher.search(
joinQuery,
MultiCollector.wrap(new BitSetCollector(actualResult), topScoreDocCollector));

View File

@ -315,8 +315,7 @@ public final class SearchImpl extends LukeModel implements Search {
} else {
int hitsThreshold = exactHitsCount ? Integer.MAX_VALUE : DEFAULT_TOTAL_HITS_THRESHOLD;
TopScoreDocCollectorManager collectorManager =
new TopScoreDocCollectorManager(
pageSize, after, hitsThreshold, searcher.getSlices().length > 1);
new TopScoreDocCollectorManager(pageSize, after, hitsThreshold);
topDocs = searcher.search(query, collectorManager);
}

View File

@ -739,8 +739,7 @@ public class AnalyzingInfixSuggester extends Lookup implements Closeable {
searcherMgrReadLock.unlock();
}
try {
TopFieldCollectorManager c =
new TopFieldCollectorManager(SORT, num, null, 1, searcher.getSlices().length > 1);
TopFieldCollectorManager c = new TopFieldCollectorManager(SORT, num, null, 1);
// System.out.println("got searcher=" + searcher);
TopFieldDocs hits = searcher.search(finalQuery, c);

View File

@ -705,12 +705,10 @@ public class CheckHits {
private static void doCheckTopScores(Query query, IndexSearcher searcher, int numHits)
throws IOException {
boolean supportsConcurrency = searcher.getSlices().length > 1;
TopScoreDocCollectorManager complete =
new TopScoreDocCollectorManager(
numHits, null, Integer.MAX_VALUE, supportsConcurrency); // COMPLETE
new TopScoreDocCollectorManager(numHits, null, Integer.MAX_VALUE); // COMPLETE
TopScoreDocCollectorManager topScores =
new TopScoreDocCollectorManager(numHits, null, 1, supportsConcurrency); // TOP_SCORES
new TopScoreDocCollectorManager(numHits, null, 1); // TOP_SCORES
TopDocs completeTopDocs = searcher.search(query, complete);
TopDocs topScoresTopDocs = searcher.search(query, topScores);
checkEqual(query, completeTopDocs.scoreDocs, topScoresTopDocs.scoreDocs);