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

LUCENE-10280: Store BKD blocks with continuous ids more efficiently #510

Merged
merged 12 commits into from Dec 7, 2021

Conversation

gf2121
Copy link
Contributor

@gf2121 gf2121 commented Dec 3, 2021

For scenes that index is sorted on the field, blocks with continuous ids may be a common situation. In this case we can handle this situation more efficiently. We just need to check

stritylysorted && (docIds[start+count-1] - docids[start] + 1) == count

to check if ids is continuous. If so, we can only write the first id of this block.

Copy link
Contributor

@jpountz jpountz left a comment

Choose a reason for hiding this comment

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

I'll wait before merging to give @iverase a chance to give his opinion on this change but it looks good to me.

out.writeVInt(docIds[start]);
return;
} else if (min2max <= (count << 4)) {
// Only trigger bitset optimization when max - min + 1 <= 16 * count in order to avoid
Copy link
Contributor

Choose a reason for hiding this comment

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

maybe assert that min2max > count?

Copy link
Contributor Author

@gf2121 gf2121 Dec 6, 2021

Choose a reason for hiding this comment

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

+1, fixed

Copy link
Contributor

@iverase iverase left a comment

Choose a reason for hiding this comment

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

Glad we didn't introduce another implementation of DocIdSetIterator and all there dance around it.

@gf2121
Copy link
Contributor Author

gf2121 commented Dec 6, 2021

Hi @jpountz ! Just to remind, maybe we can merge this now? :)

By the way, I found that there is a PR about using readLELongs in BKD apache/lucene-solr#1538. The discussion of this issue has stopped since last Year. This looks promising and I'd like to play with it but i wonder why it stopped, if there are some problems with this idea or if there has already been someone working on this ?

@iverase
Copy link
Contributor

iverase commented Dec 6, 2021

I will merge soon if Adrien does not beat me up.

I worked on the PR about using #readLELongs but never get a meaningful speed up that justify the added complexity. Maybe now that we have little endian codecs might make more sense. I am not planing to continue that work so please feel free to have a go.

@iverase iverase merged commit 8525356 into apache:main Dec 7, 2021
@gf2121
Copy link
Contributor Author

gf2121 commented Dec 8, 2021

@iverase Thanks for your explanation!

I worked on the PR about using #readLELongs but never get a meaningful speed up that justify the added complexity.

I find that we were trying to use #readLELongs to speed up 24/32 bit situation in the DocIdsWriter, which means the ids in the block are unsorted, typically happening in high cardinarlity fields. I think queries on high cardinality fields spend most of their time on visitDocValues but not readDocIds, so maybe this is the reason that we can not see a obvious gain on E2E took?

My current thoughts are about using readLELongs to speed up the sorted ids situation (means low or medium cardinality fields), whose bottleneck is reading docIds. For sorted arrays, we can compute the delta of the sorted ids and encode/decode them like what we do in StoredFieldsInts.

I raised an ISSUE based on this idea. The benchmark result i post in the issue looks promising. Would you like to help take a look when you have free time? Thanks!

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.

None yet

3 participants