LUCENE-10315: Speed up BKD leaf block ids codec by a 512 ints ForUtil#541
LUCENE-10315: Speed up BKD leaf block ids codec by a 512 ints ForUtil#541gf2121 merged 37 commits intoapache:mainfrom
Conversation
|
Nice. I wonder if we need to specialize for so many numbers of bits per value like we do for postings, or if we should only specialize for a few numbers of bits per value that are both useful and fast, e.g. 0, 4, 8, 16, 24 and 32. |
|
Thanks @jpountz ! I have two points to conifrm:
|
|
I was thinking of only supporting these numbers of bits per value indeed. For postings, numbers are always deltas, so we can generally expect them to be small. But for BKD trees it tends to be more an exception so I don't think we should spend too much effort on supporting so many bits per value and only focus on the ones that matter:
|
|
Thanks @jpountz for the great advice! I have implemented it and codes are simplified a lot. |
| for (int i = 0; i < 192; ++i) { | ||
| tmp[i] = longs[i] << 8; | ||
| } | ||
| for (int i = 0; i < 64; i++) { |
There was a problem hiding this comment.
This encoding logic is a bit different from the ForUtil we used in postings, as i want to make the remainder decoding can also trigger SIMD. Here is the JMH result:
Benchmark (bitsPerValue) (byteOrder) Mode Cnt Score Error Units
PackedIntsDecodeBenchmark.baseline 24 LE thrpt 5 7756530.597 ± 1654468.198 ops/s
PackedIntsDecodeBenchmark.candidate 24 LE thrpt 5 9681438.494 ± 2528482.525 ops/s
| return expandMask32((1L << bitsPerValue) - 1); | ||
| } | ||
|
|
||
| private static void expand16(long[] arr, final long base) { |
There was a problem hiding this comment.
This is a bit tricky but indeed helping the performance:
Benchmark (bitsPerValue) (byteOrder) Mode Cnt Score Error Units
PackedIntsDecodeBenchmark.plusAfterExpand 16 LE thrpt 20 7996083.715 ± 618197.203 ops/s
PackedIntsDecodeBenchmark.plusWhenExpand 16 LE thrpt 20 10681376.808 ± 542945.909 ops/s
There was a problem hiding this comment.
Thanks for creating these micro benchmarks.
jpountz
left a comment
There was a problem hiding this comment.
Thanks @gf2121.
If you have the time, I'd be curious to see how it would compare with a similar approach where we would use an int[] to hold the doc IDs instead of longs. Postings started with longs because it helped do "fake" SIMD operations, e.g. summing up a single long would sum up two ints under the hood. But as our BKD tree doesn't need to perform prefix sums, it doesn't need such tricks. We could add a new IndexInput#readInts similar to IndexInput#readLongs and see how the impl that uses longs compares to the one that uses ints?
| return expandMask32((1L << bitsPerValue) - 1); | ||
| } | ||
|
|
||
| private static void expand16(long[] arr, final long base) { |
There was a problem hiding this comment.
Thanks for creating these micro benchmarks.
lucene/core/src/java/org/apache/lucene/util/bkd/BKDForUtil.java
Outdated
Show resolved
Hide resolved
|
@iverase Thank you very much for helping debug so much!
I found out the bug based on this clue you provided, and fixed it in this commit. Also added some more tests around the Very sorry for my negligence, and thanks again for your help! |
iverase
left a comment
There was a problem hiding this comment.
I think it would be nice to add more test to readInts. Probably enough to add the readInts counterpart that are on BaseDirectoryTestCase and BaseChunkedDirectoryTestCase.
Otherwise just left a few comments but I think this a good change.
| curBufIndex = 0; | ||
| curLongBufferViews = null; | ||
| curFloatBufferViews = null; | ||
| curIntBufferViews = null; |
There was a problem hiding this comment.
curFloatBufferViews does not belong to this PR. I wonder if we should open a separate issue for this as it might lead to unknown bugs? what do you think @jpountz
There was a problem hiding this comment.
+1 to open a separate issue
| private final int[] tmp; | ||
|
|
||
| BKDForUtil(int maxPointsInLeaf) { | ||
| tmp = new int[maxPointsInLeaf / 4 * 3]; |
There was a problem hiding this comment.
I am a bit confused with this, can you add an explanation of why we are choosing that size. Probably it would be good to use parenthesis to clarify the order of execution,
|
|
||
| final class BKDForUtil { | ||
|
|
||
| static final int BLOCK_SIZE = 512; |
There was a problem hiding this comment.
I think this is not needed any more, should be remove it?
|
Hi @jpountz ! I wonder if you'd like to take another look at this PR? I'll merge and backport this if you agree. |
jpountz
left a comment
There was a problem hiding this comment.
LGTM
We should look into improving testing in a follow-up, there is now little testing for the legacy vint logic.
| curBufIndex = 0; | ||
| curLongBufferViews = null; | ||
| curFloatBufferViews = null; | ||
| curIntBufferViews = null; |
There was a problem hiding this comment.
+1 to open a separate issue
| } | ||
| } | ||
|
|
||
| public void readInts(int[] dst, int offset, int length) throws IOException { |
| } | ||
| for (int i = 0; i < quarterLen; i++) { | ||
| final int longIdx = off + i + quarterLen3; | ||
| tmp[i] |= (ints[longIdx] >>> 16) & 0xFF; |
There was a problem hiding this comment.
I think we don't even need toe mask since values use 24 bits?
| tmp[i] |= (ints[longIdx] >>> 16) & 0xFF; | |
| tmp[i] |= ints[longIdx] >>> 16; |
| out.writeVInt(doc - previous); | ||
| previous = doc; | ||
|
|
||
| if (Integer.toUnsignedLong(min2max) <= 0xFFFFL) { |
There was a problem hiding this comment.
since doc IDs are integers in [0, MAX_VALUE) we don't need to convert to an unsigned long (MAX_VALUE is not a legal doc ID)
| if (Integer.toUnsignedLong(min2max) <= 0xFFFFL) { | |
| if (min2max <= 0xFFFF) { |
| out.writeShort((short) (docIds[start + i] >>> 8)); | ||
| out.writeByte((byte) docIds[start + i]); | ||
| } | ||
| if (Integer.toUnsignedLong(max) <= 0xFFFFFFL) { |
There was a problem hiding this comment.
| if (Integer.toUnsignedLong(max) <= 0xFFFFFFL) { | |
| if (max <= 0xFFFFFF) { |
lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
Outdated
Show resolved
Hide resolved
lucene/core/src/java/org/apache/lucene/util/bkd/DocIdsWriter.java
Outdated
Show resolved
Hide resolved
Co-authored-by: Adrien Grand <jpountz@gmail.com>
Co-authored-by: Adrien Grand <jpountz@gmail.com>
… ForUtil (apache#541)" This reverts commit 8c67a38.
Elasticsearch (which based on lucene) can automatically infer types for users with its dynamic mapping feature. When users index some low cardinality fields, such as gender / age / status... they often use some numbers to represent the values, while ES will infer these fields as long, and ES uses BKD as the index of long fields. Just as #541 said, when the data volume grows, building the result set of low-cardinality fields will make the CPU usage and load very high even if we use a boolean query with filter clauses for low-cardinality fields. One reason is that it uses a ReentrantLock to limit accessing LRUQueryCache. QPS and costs of their queries are often high, which often causes trying locking failures when obtaining the cache, resulting in low concurrency in accessing the cache. So I replace the ReentrantLock with a ReentrantReadWriteLock. I only use the read lock when I need to get the cache for a query,
Elasticsearch (which based on lucene) can automatically infer types for users with its dynamic mapping feature. When users index some low cardinality fields, such as gender / age / status... they often use some numbers to represent the values, while ES will infer these fields as long, and ES uses BKD as the index of long fields. Just as #541 said, when the data volume grows, building the result set of low-cardinality fields will make the CPU usage and load very high even if we use a boolean query with filter clauses for low-cardinality fields. One reason is that it uses a ReentrantLock to limit accessing LRUQueryCache. QPS and costs of their queries are often high, which often causes trying locking failures when obtaining the cache, resulting in low concurrency in accessing the cache. So I replace the ReentrantLock with a ReentrantReadWriteLock. I only use the read lock when I need to get the cache for a query,
Elasticsearch (which based on lucene) can automatically infers types for users with its dynamic mapping feature. When users index some low cardinality fields, such as gender / age / status... they often use some numbers to represent the values, while ES will infer these fields as long, and ES uses BKD as the index of long fields. When the data volume grows, building the result set of low-cardinality fields will make the CPU usage and load very high.
This is a flame graph we obtained from the production environment:
flame.svg
It can be seen that almost all CPU is used in addAll. When we reindex long to keyword, the cluster load and search latency are greatly reduced ( We spent weeks of time to reindex all indices... ). I know that ES recommended to use keyword for term/terms query and long for range query in the document, but there are always some users who didn't realize this and keep their habit of using sql database, or dynamic mapping automatically selects the type for them. All in all, users won't realize that there would be such a big difference in performance between long and keyword fields in low cardinality fields. So from my point of view it will make sense if we can make BKD works better for the low/medium cardinality fields.
As far as i can see, for low cardinality fields, there are two advantages of keyword over long:
This approach is targeting to solve the first point. The basic idea is trying to use a 512 ints ForUtil for BKD ids codec. I benchmarked this optimization by mocking some random LongPoint and querying them with PointInSetQuery.
Benchmark Result
The indices size is reduced for low cardinality fields and flat for high cardinality fields.