Skip to content

Commit

Permalink
LUCENE-10002: Deprecate IndexSearch#search(Query, Collector) in favor…
Browse files Browse the repository at this point in the history
… of IndexSearcher#search(Query, CollectorManager) - TopFieldCollectorManager & TopScoreDocCollectorManager (#240)
  • Loading branch information
zacharymorn authored Nov 28, 2023
1 parent 17bb733 commit 38ca8d3
Show file tree
Hide file tree
Showing 36 changed files with 748 additions and 547 deletions.
6 changes: 6 additions & 0 deletions lucene/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,10 @@ API Changes

* GITHUB#12599: Add RandomAccessInput#readBytes method to the RandomAccessInput interface. (Ignacio Vera)

* GITHUB#11041: Deprecate IndexSearch#search(Query, Collector) in favor of
IndexSearcher#search(Query, CollectorManager) for TopFieldCollectorManager
and TopScoreDocCollectorManager. (Zach Chen, Adrien Grand, Michael McCandless, Greg Miller, Luca Cavanna)

New Features
---------------------

Expand Down Expand Up @@ -172,6 +176,8 @@ API Changes
* GITHUB#12799: Make TaskExecutor constructor public and use TaskExecutor for concurrent
HNSW graph build. (Shubham Chaudhary)

*

New Features
---------------------

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.TopFieldCollector;
import org.apache.lucene.search.TopScoreDocCollector;
import org.apache.lucene.search.TopFieldCollectorManager;
import org.apache.lucene.search.TopScoreDocCollectorManager;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;

Expand Down Expand Up @@ -110,15 +110,17 @@ public int doLogic() throws Exception {
// the IndexSearcher search methods that take
// Weight public again, we can go back to
// pulling the Weight ourselves:
TopFieldCollector collector =
TopFieldCollector.create(sort, numHits, withTotalHits() ? Integer.MAX_VALUE : 1);
searcher.search(q, collector);
hits = collector.topDocs();
int totalHitsThreshold = withTotalHits() ? Integer.MAX_VALUE : 1;
TopFieldCollectorManager collectorManager =
new TopFieldCollectorManager(
sort, numHits, null, totalHitsThreshold, searcher.getSlices().length > 1);
hits = searcher.search(q, collectorManager);
} else {
hits = searcher.search(q, numHits);
}
} else {
Collector collector = createCollector();

searcher.search(q, collector);
// hits = collector.topDocs();
}
Expand Down Expand Up @@ -183,7 +185,8 @@ protected int withTopDocs(IndexSearcher searcher, Query q, TopDocs hits) throws
}

protected Collector createCollector() throws Exception {
return TopScoreDocCollector.create(numHits(), withTotalHits() ? Integer.MAX_VALUE : 1);
return new TopScoreDocCollectorManager(numHits(), withTotalHits() ? Integer.MAX_VALUE : 1)
.newCollector();
}

protected Document retrieveDoc(StoredFields storedFields, int id) throws IOException {
Expand Down
73 changes: 13 additions & 60 deletions lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
Expand Down Expand Up @@ -62,9 +61,9 @@
* match lots of documents, counting the number of hits may take much longer than computing the top
* hits so this trade-off allows to get some minimal information about the hit count without slowing
* down search too much. The {@link TopDocs#scoreDocs} array is always accurate however. If this
* behavior doesn't suit your needs, you should create collectors manually with either {@link
* TopScoreDocCollector#create} or {@link TopFieldCollector#create} and call {@link #search(Query,
* Collector)}.
* behavior doesn't suit your needs, you should create collectorManagers manually with either {@link
* TopScoreDocCollectorManager} or {@link TopFieldCollectorManager} and call {@link #search(Query,
* CollectorManager)}.
*
* <p><a id="thread-safety"></a>
*
Expand Down Expand Up @@ -455,35 +454,10 @@ public TopDocs searchAfter(ScoreDoc after, Query query, int numHits) throws IOEx
}

final int cappedNumHits = Math.min(numHits, limit);

final LeafSlice[] leafSlices = getSlices();
final CollectorManager<TopScoreDocCollector, TopDocs> manager =
new CollectorManager<TopScoreDocCollector, TopDocs>() {

private final HitsThresholdChecker hitsThresholdChecker =
leafSlices.length <= 1
? HitsThresholdChecker.create(Math.max(TOTAL_HITS_THRESHOLD, numHits))
: HitsThresholdChecker.createShared(Math.max(TOTAL_HITS_THRESHOLD, numHits));

private final MaxScoreAccumulator minScoreAcc =
leafSlices.length <= 1 ? null : new MaxScoreAccumulator();

@Override
public TopScoreDocCollector newCollector() throws IOException {
return TopScoreDocCollector.create(
cappedNumHits, after, hitsThresholdChecker, minScoreAcc);
}

@Override
public TopDocs reduce(Collection<TopScoreDocCollector> collectors) throws IOException {
final TopDocs[] topDocs = new TopDocs[collectors.size()];
int i = 0;
for (TopScoreDocCollector collector : collectors) {
topDocs[i++] = collector.topDocs();
}
return TopDocs.merge(0, cappedNumHits, topDocs);
}
};
final boolean supportsConcurrency = getSlices().length > 1;
CollectorManager<TopScoreDocCollector, TopDocs> manager =
new TopScoreDocCollectorManager(
cappedNumHits, after, TOTAL_HITS_THRESHOLD, supportsConcurrency);

return search(query, manager);
}
Expand All @@ -510,7 +484,10 @@ public TopDocs search(Query query, int n) throws IOException {
*
* @throws TooManyClauses If a query would exceed {@link IndexSearcher#getMaxClauseCount()}
* clauses.
* @deprecated This method is being deprecated in favor of {@link IndexSearcher#search(Query,
* CollectorManager)} due to its support for concurrency in IndexSearcher
*/
@Deprecated
public void search(Query query, Collector results) throws IOException {
query = rewrite(query, results.scoreMode().needsScores());
search(leafContexts, createWeight(query, results.scoreMode(), 1), results);
Expand Down Expand Up @@ -602,34 +579,10 @@ private TopFieldDocs searchAfter(
final Sort rewrittenSort = sort.rewrite(this);
final LeafSlice[] leafSlices = getSlices();

final boolean supportsConcurrency = leafSlices.length > 1;
final CollectorManager<TopFieldCollector, TopFieldDocs> manager =
new CollectorManager<>() {

private final HitsThresholdChecker hitsThresholdChecker =
leafSlices.length <= 1
? HitsThresholdChecker.create(Math.max(TOTAL_HITS_THRESHOLD, numHits))
: HitsThresholdChecker.createShared(Math.max(TOTAL_HITS_THRESHOLD, numHits));

private final MaxScoreAccumulator minScoreAcc =
leafSlices.length <= 1 ? null : new MaxScoreAccumulator();

@Override
public TopFieldCollector newCollector() throws IOException {
// TODO: don't pay the price for accurate hit counts by default
return TopFieldCollector.create(
rewrittenSort, cappedNumHits, after, hitsThresholdChecker, minScoreAcc);
}

@Override
public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
final TopFieldDocs[] topDocs = new TopFieldDocs[collectors.size()];
int i = 0;
for (TopFieldCollector collector : collectors) {
topDocs[i++] = collector.topDocs();
}
return TopDocs.merge(rewrittenSort, 0, cappedNumHits, topDocs);
}
};
new TopFieldCollectorManager(
rewrittenSort, cappedNumHits, after, TOTAL_HITS_THRESHOLD, supportsConcurrency);

TopFieldDocs topDocs = search(query, manager);
if (doDocScores) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,10 @@ public TopDocs rescore(IndexSearcher searcher, TopDocs firstPassTopDocs, int top

List<LeafReaderContext> leaves = searcher.getIndexReader().leaves();

TopFieldCollector collector = TopFieldCollector.create(sort, topN, Integer.MAX_VALUE);
TopFieldCollector collector =
new TopFieldCollectorManager(
sort, topN, null, Integer.MAX_VALUE, searcher.getSlices().length > 1)
.newCollector();

// Now merge sort docIDs from hits, with reader's leaves:
int hitUpto = 0;
Expand Down
3 changes: 1 addition & 2 deletions lucene/core/src/java/org/apache/lucene/search/TopDocs.java
Original file line number Diff line number Diff line change
Expand Up @@ -232,8 +232,7 @@ public static TopDocs merge(
/**
* Returns a new TopFieldDocs, containing topN results across the provided TopFieldDocs, sorting
* by the specified {@link Sort}. Each of the TopDocs must have been sorted by the same Sort, and
* sort field values must have been filled (ie, <code>fillFields=true</code> must be passed to
* {@link TopFieldCollector#create}).
* sort field values must have been filled.
*
* @see #merge(Sort, int, int, TopFieldDocs[])
* @lucene.experimental
Expand Down
108 changes: 17 additions & 91 deletions lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.Comparator;
import java.util.List;
import java.util.Objects;
Expand Down Expand Up @@ -174,7 +173,7 @@ private static boolean canEarlyTerminateOnPrefix(Sort searchSort, Sort indexSort
* Implements a TopFieldCollector over one SortField criteria, with tracking
* document scores and maxScore.
*/
private static class SimpleFieldCollector extends TopFieldCollector {
static class SimpleFieldCollector extends TopFieldCollector {
final Sort sort;
final FieldValueHitQueue<Entry> queue;

Expand Down Expand Up @@ -225,7 +224,7 @@ public void collect(int doc) throws IOException {
/*
* Implements a TopFieldCollector when after != null.
*/
private static final class PagingFieldCollector extends TopFieldCollector {
static final class PagingFieldCollector extends TopFieldCollector {

final Sort sort;
int collectedHits;
Expand Down Expand Up @@ -405,9 +404,13 @@ protected void updateMinCompetitiveScore(Scorable scorer) throws IOException {
* 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.
* @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
* @deprecated This method is deprecated in favor of the constructor of {@link
* TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
*/
@Deprecated
public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThreshold) {
return create(sort, numHits, null, totalHitsThreshold);
return new TopFieldCollectorManager(sort, numHits, null, totalHitsThreshold, false)
.newCollector();
}

/**
Expand All @@ -429,106 +432,29 @@ public static TopFieldCollector create(Sort sort, int numHits, int totalHitsThre
* field is indexed both with doc values and points. In this case, there is an assumption that
* the same data is stored in these points and doc values.
* @return a {@link TopFieldCollector} instance which will sort the results by the sort criteria.
* @deprecated This method is deprecated in favor of the constructor of {@link
* TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
*/
@Deprecated
public static TopFieldCollector create(
Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {
if (totalHitsThreshold < 0) {
throw new IllegalArgumentException(
"totalHitsThreshold must be >= 0, got " + totalHitsThreshold);
}

return create(
sort,
numHits,
after,
HitsThresholdChecker.create(Math.max(totalHitsThreshold, numHits)),
null /* bottomValueChecker */);
}

/**
* Same as above with additional parameters to allow passing in the threshold checker and the max
* score accumulator.
*/
static TopFieldCollector create(
Sort sort,
int numHits,
FieldDoc after,
HitsThresholdChecker hitsThresholdChecker,
MaxScoreAccumulator minScoreAcc) {

if (sort.getSort().length == 0) {
throw new IllegalArgumentException("Sort must contain at least one field");
}

if (numHits <= 0) {
throw new IllegalArgumentException(
"numHits must be > 0; please use TotalHitCountCollector if you just need the total hit count");
}

if (hitsThresholdChecker == null) {
throw new IllegalArgumentException("hitsThresholdChecker should not be null");
}

FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.getSort(), numHits);

if (after == null) {
// inform a comparator that sort is based on this single field
// to enable some optimizations for skipping over non-competitive documents
// We can't set single sort when the `after` parameter is non-null as it's
// an implicit sort over the document id.
if (queue.comparators.length == 1) {
queue.comparators[0].setSingleSort();
}
return new SimpleFieldCollector(sort, queue, numHits, hitsThresholdChecker, minScoreAcc);
} else {
if (after.fields == null) {
throw new IllegalArgumentException(
"after.fields wasn't set; you must pass fillFields=true for the previous search");
}

if (after.fields.length != sort.getSort().length) {
throw new IllegalArgumentException(
"after.fields has "
+ after.fields.length
+ " values but sort has "
+ sort.getSort().length);
}

return new PagingFieldCollector(
sort, queue, after, numHits, hitsThresholdChecker, minScoreAcc);
}
return new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, false)
.newCollector();
}

/**
* Create a CollectorManager which uses a shared hit counter to maintain number of hits and a
* shared {@link MaxScoreAccumulator} to propagate the minimum score accross segments if the
* primary sort is by relevancy.
*
* @deprecated This method is deprecated in favor of the constructor of {@link
* TopFieldCollectorManager} due to its support for concurrency in IndexSearcher
*/
@Deprecated
public static CollectorManager<TopFieldCollector, TopFieldDocs> createSharedManager(
Sort sort, int numHits, FieldDoc after, int totalHitsThreshold) {

int totalHitsMax = Math.max(totalHitsThreshold, numHits);
return new CollectorManager<>() {
private final HitsThresholdChecker hitsThresholdChecker =
HitsThresholdChecker.createShared(totalHitsMax);
private final MaxScoreAccumulator minScoreAcc =
totalHitsMax == Integer.MAX_VALUE ? null : new MaxScoreAccumulator();

@Override
public TopFieldCollector newCollector() throws IOException {
return create(sort, numHits, after, hitsThresholdChecker, minScoreAcc);
}

@Override
public TopFieldDocs reduce(Collection<TopFieldCollector> collectors) throws IOException {
final TopFieldDocs[] topDocs = new TopFieldDocs[collectors.size()];
int i = 0;
for (TopFieldCollector collector : collectors) {
topDocs[i++] = collector.topDocs();
}
return TopDocs.merge(sort, 0, numHits, topDocs);
}
};
return new TopFieldCollectorManager(sort, numHits, after, totalHitsThreshold, true);
}

/**
Expand Down
Loading

0 comments on commit 38ca8d3

Please sign in to comment.