-
Notifications
You must be signed in to change notification settings - Fork 1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add a post-collection hook to LeafCollector. #12380
Conversation
This adds `LeafCollector#finish` as a per-segment post-collection hook. While it was already possible to do this sort of things on top of the collector API before, a downside is that the last leaf would need to be post-collected in the current thread instead of using the executor, which is a missed opportunity for making queries concurrent. Closes apache#12375
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Overall LGTM, thanks!
lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysScorer.java
Outdated
Show resolved
Hide resolved
@@ -749,6 +749,9 @@ protected void search(List<LeafReaderContext> leaves, Weight weight, Collector c | |||
partialResult = true; | |||
} | |||
} | |||
// Note: this is called if collection ran successfully, including the above special cases of | |||
// CollectionTerminatedException and TimeExceededException, but no other exception. | |||
leafCollector.finish(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder whether it worths passing in the exceptions if any in case of early termination, but I can't think of a concrete example of how it might be useful right now (maybe user want a faster finish step in case of early terminated by time?), maybe we can add it later if there's a real need?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I can't think of a use-case either. Another argument could be that CollectionTerminatedException is only one way to skip hits, LeafCollector#competitiveIterator
and Scorer#setMinCompetitiveScore
are other ones, why would we give more information to finish() for one way of skipping and not for other ones?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One thing I notice in the case there is no doc of interest, it won't be called (see we are not building a leaf collector in that case, sorry.continue
statement), I wonder if we should call it even in that case?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This could be an opportunity for capturing statistics about how often time-limitation is applied?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
looks good thanks, just one or two little questions
@@ -749,6 +749,9 @@ protected void search(List<LeafReaderContext> leaves, Weight weight, Collector c | |||
partialResult = true; | |||
} | |||
} | |||
// Note: this is called if collection ran successfully, including the above special cases of | |||
// CollectionTerminatedException and TimeExceededException, but no other exception. | |||
leafCollector.finish(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This could be an opportunity for capturing statistics about how often time-limitation is applied?
// NOTE: this also clears the priorityQueue: | ||
for (SuggestScoreDoc hit : priorityQueue.getResults()) { | ||
pendingResults.add(hit); | ||
} | ||
|
||
// Deduplicate all hits: we already dedup'd efficiently within each segment by |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
any particular reason to change the order of operations here?
|
||
// Deduplicate all hits: we already dedup'd efficiently within each segment by | ||
// truncating the FST top paths search, but across segments there may still be dups: | ||
seenSurfaceForms.clear(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
oh I see we did it both ways. Probably makes no difference? Still superstitiously I would always want to clear/delete things last just in case...
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 I liked moving the clear
last better
@@ -49,7 +50,9 @@ public LeafCollector getLeafCollector(LeafReaderContext context) throws IOExcept | |||
assert context.docBase >= previousLeafMaxDoc; | |||
previousLeafMaxDoc = context.docBase + context.reader().maxDoc(); | |||
|
|||
assert hasFinishedCollectingPreviousLeaf; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
it's a pity we can't assert that we finished the final leaf
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your comment, I wanted to look into that and then forgot. It should be doable via AssertingIndexSearcher
.
|
||
@Override | ||
public void finish() throws IOException { | ||
assert finishCalled == false; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Did we previously disallow re-use of LeafCollectors? If not, this could break someone
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yes, it's disallowed by design since LeafCollector#collect
must collect doc IDs in doc ID order.
This adds `LeafCollector#finish` as a per-segment post-collection hook. While it was already possible to do this sort of things on top of the collector API before, a downside is that the last leaf would need to be post-collected in the current thread instead of using the executor, which is a missed opportunity for making queries concurrent.
LeafCollector leafCollector = collector.getLeafCollector(context); | ||
try { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Comparing this to the
lucene/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
Lines 710 to 740 in 6d764c3
final LeafCollector leafCollector; | |
try { | |
leafCollector = collector.getLeafCollector(ctx); | |
} catch ( | |
@SuppressWarnings("unused") | |
CollectionTerminatedException e) { | |
// there is no doc of interest in this reader context | |
// continue with the following leaf | |
continue; | |
} | |
BulkScorer scorer = weight.bulkScorer(ctx); | |
if (scorer != null) { | |
if (queryTimeout != null) { | |
scorer = new TimeLimitingBulkScorer(scorer, queryTimeout); | |
} | |
try { | |
scorer.score(leafCollector, ctx.reader().getLiveDocs()); | |
} catch ( | |
@SuppressWarnings("unused") | |
CollectionTerminatedException e) { | |
// collection was terminated prematurely | |
// continue with the following leaf | |
} catch ( | |
@SuppressWarnings("unused") | |
TimeLimitingBulkScorer.TimeExceededException e) { | |
partialResult = true; | |
} | |
} | |
// Note: this is called if collection ran successfully, including the above special cases of | |
// CollectionTerminatedException and TimeExceededException, but no other exception. | |
leafCollector.finish(); |
getLeafCollector
call should move inside the try
block here too?
final LeafCollector leafCollector;
try {
leafCollector = collector.getLeafCollector(context);
...
} catch (CollectionTerminatedException e) {
...
}
if (leafCollector != null) leafCollector.finish();
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Trying to remember what was on my mind at the time of the change, I think I wanted to keep the logic simple, since unlike IndexSearcher which may run any Collector, here it may only be a TopSuggestDocsCollector
, which never throws a CollectionTerminatedException
. I'm ok with moving the getLeafCollector
call under the try
block though, if you open a PR I'll be happy to approve it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
#12609 opened
This adds
LeafCollector#finish
as a per-segment post-collection hook. While it was already possible to do this sort of things on top of the collector API before, a downside is that the last leaf would need to be post-collected in the current thread instead of using the executor, which is a missed opportunity for making queries concurrent.Closes #12375