-
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
Introduce new encoding of BPV 21 for DocIdsWriter used in BKD Tree #13521
base: main
Are you sure you want to change the base?
Conversation
The high-level change makes sense to me. We've used this trick to encode 3 21-bit integers in a 64-bit long in the past, it makes sense to me that it's helping here too. I also like that it would apply to all segments that have less than 2M docs, which should cover a large number of segments. In terms of code organization, can you write your benchmarks as a JMH benchmark and put it under lucene/benchmark-jmh? |
This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the dev@lucene.apache.org list. Thank you for your contribution! |
While trying out 2 different ways ( one a subset of other ) I found that x86 - EC2 instance type
aarch64 - EC2 instance type
JDK
@jpountz IMO We should use Let me know your thoughts on the same. |
The approach is pretty neat. I'm wondering if That is, unrolling the loop to process 3 longs per iteration is faster than processing 1 long per iteration. What about 2 longs per iteration? What about 4 longs per iteration? Since I've been playing around with the incubating vector API recently, I'm going to try downloading your microbenchmark and adding a vectorized implementation. (I have access to an M1 Mac that should be able to process 2 longs at a time, plus an Intel Xeon whose AVX-512 operations should probably be able to do 8 longs.) |
I tried modifying the loop to process 4 longs per iteration and noticed no difference on my Xeon host, which is unsurprising since there was no difference between 1 and 3. I also tried the following SIMD implementation of
Unfortunately, it performs noticeably worse than the other implementations:
Maybe I'm doing it wrong 🤷 |
Okay -- I was able to speed up the SIMD implementation a fair bit. Honestly, my main stupid mistake was that I hadn't declared I removed the array allocations in each call, as well as the scalar operations within the vector loop.
It's still slower than the scalar implementation, but it's a lot closer:
|
…mpare performance
…mpare performance
This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the dev@lucene.apache.org list. Thank you for your contribution! |
Background
Lucene uses 3 different ways of storing the docIds in KDD file of a BKD Tree based index if the docIds in a leaf block are not sorted :
BPV_24 uses less number of bitwise/arithmetic operations compared to BPV_16. Even if we represent the docIds using any other number of bits like 21/22/23/25/26/27/28/29/30/31, the number of bitwise operations used for encoding and decoding will increase. Let's compare 2 encoding schemes as an example,
Note
|
represents one packed long and,
separates the individual docIds in the packed longs.Only decoding is shown in the example, but the same applies for encoding as well.
For BPV_24
Packs 8 docIds in 3 longs.So, 512 docIds in ( 3/8 * 512 ) = 192 longs
It uses 18 bitwise operations to decode 8 docIds. So, 512 docIds require ( 18/8 * 512 ) = 1152 bitwise operations.
The bitwise operators used for decoding 8 docIds can also be visualised as follows :
M
is mask using AND,S
is bitwise left/right shift andO
is logical OR to join 2 partial halves present in different longs.For BPV_20
Packs 16 docIds in 5 longs. So, 512 docIds require ( 5/16 *512 ) = 160 longs
However, it will use 38 bitwise operations to decode 16 docIds. So, 512 docIds require ( 38/16 * 512 ) = 1216 bitwise ops.
I have analysed the same for other BPV like
21/22/23/25/26/27/28/29/30/31
and in all cases the number of bitwise operations for encoding and decoding is higher than BPV_24.Solution
While analysing for BPV_21, I observed that if we just pack 3 docIds in a long, then number of bitwise operations in encoding and decoding can be reduced to be less than BPV_24. The extra bit can be kept at leftmost position ( MSB ) as
0
to reduce the number of operations.Decoding
In this case, it requires 12 bitwise operations to decode 9 docIds. So, 512 docIds will require ( 12/9 * 512 ) ~ 683 bitwise ops.
It will store 9 docIds in 3 packed longs. So, 512 docIds require ( 3/9 *512 ) ~ 171 longs. This will reduce the number of longs required for such leaves compared to BPV_24 by 21 (192-171)
Micro Benchmark
Since, introducing BPV_21 will compete with BPV_24, I wrote a micro benchmark to compare the encoding and decoding speeds of both these variations.
Bit21With2StepsAddEncoder
andBit21With3StepsAddEncoder
both perform encode/decode using the proposed BPV_21 format.Bit24Encoder
is the exact replica of BPV_24 used in Lucene today.Java version Used
Input to the benchmark :
Sample run command
The data used in the benchmark is
lucene/core/src/java/org/apache/lucene/util/bkd/docIds/data/finalfile.txt
which contains all docId sequences that can be represented in 21 bits ( Max is <= 0x001FFFFFL aka 20,97,151 ). This has been extracted from first 10 million docs of NYC Taxi data by only indexing the fieldfare_amount
as a double point.There are 6509 docId sequences in the input file and 6493 of them contain 512 docIds each. There are a total of 33,28,103 docIds in those 6509 sequences.
Input scale factor
multiplies the number of docIds sequences by the given factor to increase the load for the benchmark.The script below was executed 5 times and the numbers are the average of those runs.
10
is the number of iterations for the encoder and1000
is the input scale factor.Write latencies numbers exhibited 2 patterns, one when EBS Write latency peaked and other during normal EBS latency. Both the variations are captured below.
There was also size reduction of around 200 MB in kdd file when indexing entire NYC Taxi data with this change.
Next steps
Need inputs from the maintainers and contributors on this new BPV format and other benchmarks that need to be executed ( probably luceneutil ? ) to justify that this change doesn't cause any like regressions as seen in SIMD Optimisation.
After feedback, I will fix build failures, add UTs and remove the
org.apache.lucene.util.bkd.docIds
package for this to be in a state to be merged.