Skip to content
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

Merged
merged 8 commits into from
Jun 30, 2023

Conversation

jpountz
Copy link
Contributor

@jpountz jpountz commented Jun 21, 2023

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

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
@jpountz jpountz added this to the 9.8.0 milestone Jun 21, 2023
@jpountz jpountz requested a review from msokolov June 21, 2023 11:20
Copy link
Contributor

@zhaih zhaih left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall LGTM, thanks!

@@ -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();
Copy link
Contributor

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?

Copy link
Contributor Author

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?

Copy link
Contributor

@iverase iverase Jun 28, 2023

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 continue statement), I wonder if we should call it even in that case? we are not building a leaf collector in that case, sorry.

Copy link
Contributor

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?

Copy link
Contributor

@msokolov msokolov left a 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();
Copy link
Contributor

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
Copy link
Contributor

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();
Copy link
Contributor

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...

Copy link
Contributor Author

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;
Copy link
Contributor

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

Copy link
Contributor Author

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;
Copy link
Contributor

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

Copy link
Contributor Author

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.

@jpountz jpountz merged commit 8811f31 into apache:main Jun 30, 2023
@jpountz jpountz deleted the post-collection-hook branch June 30, 2023 13:19
jpountz added a commit to jpountz/lucene that referenced this pull request Jun 30, 2023
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.
Comment on lines +71 to 72
LeafCollector leafCollector = collector.getLeafCollector(context);
try {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Comparing this to the

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();
code I wonder if the 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();

Copy link
Contributor Author

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.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

#12609 opened

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Add a postCollect hook to LeafCollector
5 participants