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

Refactor QueryCache to improve concurrency and performance #14222

Open
sgup432 opened this issue Feb 12, 2025 · 10 comments
Open

Refactor QueryCache to improve concurrency and performance #14222

sgup432 opened this issue Feb 12, 2025 · 10 comments

Comments

@sgup432
Copy link
Contributor

sgup432 commented Feb 12, 2025

Description

Given the significant role of LRUQueryCache in Lucene, I see opportunities to enhance its performance. Although there have been many discussions on this topic like here, they haven’t led to anywhere.

Currently, QueryCache uses a Map<CacheKey, LeafCache>, where:

  • CacheKey is tied to a specific segment.
  • LeafCache is per-segment and maintains a Map<Query, CacheAndCount>, where CacheAndCount is essentially a docID set.

Current Issue:

We use a global read/write lock to put items into the cache. This allows only one writer at a time holding other writers or other readers.

Even if we were to introduce multiple read/write locks(via multiple partitions), it won't help with CacheKey being our key. As if there are only 5 segments but 10,000 unique queries, we end up with just 5 unique cache keys. This means a large number of requests compete for the same write lock, significantly degrading performance by either waiting or not using cache (by skipping it if lock not obtained).

One simple Solution:

  • Introduce a Composite Key

    • Instead of CacheKey alone, use (CacheKey, Query) as the key.
    • The new structure would be Map<CompositeKey, CacheCount>
  • Partition the Cache for Better Concurrency

    • Create multiple LRU partitions (e.g., 8, 16, or 64 , configurable).
    • Each partition has its own read/write lock to reduce contention.
    • Assign incoming requests to specific partition using:
      hash(CacheKey, Query) % partition_count (as an example)
  • I also see we use uniqueQueries/mostRecentlyUsedQueries here, to maintain a LRU list for the queries. I think we can remove it and instead maintain a separate LRU list per partition, using CompositeKey.

  • For Eviction & Cleanup, we can collect stale CacheKey entries in a Set<CacheKey> when segment is closed due to merge. A background thread later iterates over cache entries and clears them, similar to how RequestCache in OpenSearch works.

Let me know what you think. I think it might work pretty well(and simple enough) but I am open to other ideas.

@sgup432
Copy link
Contributor Author

sgup432 commented Feb 12, 2025

I almost have a prototype ready for above solution, I will add benchmarking results here comparing performance.

@stefanvodita
Copy link
Contributor

Sounds interesting, keen to see if we measure a performance improvement!

@sgup432
Copy link
Contributor Author

sgup432 commented Feb 21, 2025

I got busy with other stuff but got sometime to run initial benchmark for this.

I essentially micro-benchmarked putIfAbsent() and get()methods in isolation for QueryCache for simplicity. Here is the benchmark code. It basically creates sample queries(10000) and cacheHelpers(assuming 16 lucene segments).

I created a LRUQueryCacheV2, with things recommended above. It creates 16(for this test) QueryCacheSegments with each having its own in-memory map to store composite key and value. Composite key is nothing but a combination of (CacheKey, Query), and it uses its hashcode() to determine which partition it will end up going. Rest its pretty similar to existing LRUQueryCache.

Some parts of eviction logic is yet to be fully written for V2, like clearing entires when a lucene segment is merged etc. Also ran existing UT for QueryCache on top of LRUQueryCacheV2 for high level correctness, some 16 are passing and 10 failing(basically due to incomplete eviction logic).

Coming to results:

Here v1 refers to existing QueryCache and v2 refers to my version of QueryCache.
Benchmarks can be run using: java --module-path lucene/benchmark-jmh/build/benchmarks --module org.apache.lucene.benchmark.jmh QueryCacheBenchmark

Performance Comparison: v1 vs. v2

Benchmark Version Throughput (ops/s) Error (ops/s) Performance Gain (v2 vs. v1)
Concurrent Get & Put (Mixed Load)
concurrentGetAndPuts v1 1,857,864 ±57,408 3.02x
concurrentGetAndPuts_v2 v2 5,614,289 ±96,352
Get Performance (Read-Only in Mixed load)
concurrentGetAndPuts_get v1 814,891 ±75,165 5.27x
concurrentGetAndPuts_getV2 v2 4,298,377 ±114,633
Put Performance (Write-Only in Mixed Load)
concurrentGetAndPuts_put v1 1,042,973 ±49,868 1.26x
concurrentGetAndPuts_putV2 v2 1,315,912 ±32,133
Concurrent Puts (Write-Only Load)
concurrent_puts_v1 v1 1,387,740 ±35,309 2.83x
concurrent_puts_v2 v2 3,933,324 ±58,046

Raw results:

Benchmark                                                                Mode  Cnt        Score        Error  Units
QueryCacheBenchmark.concurrentGetAndPuts                                thrpt   25  1857864.371 ±  57408.178  ops/s
QueryCacheBenchmark.concurrentGetAndPuts:concurrentGetAndPuts_get       thrpt   25   814891.042 ±  75165.491  ops/s
QueryCacheBenchmark.concurrentGetAndPuts:concurrentGetAndPuts_put       thrpt   25  1042973.329 ±  49868.486  ops/s
QueryCacheBenchmark.concurrentGetAndPuts_v2                             thrpt   25  5614289.356 ±  96352.346  ops/s
QueryCacheBenchmark.concurrentGetAndPuts_v2:concurrentGetAndPuts_getV2  thrpt   25  4298377.070 ± 114633.945  ops/s
QueryCacheBenchmark.concurrentGetAndPuts_v2:concurrentGetAndPuts_putV2  thrpt   25  1315912.286 ±  32133.146  ops/s
QueryCacheBenchmark.concurrent_puts_v1                                  thrpt   25  1387740.110 ±  35309.681  ops/s
QueryCacheBenchmark.concurrent_puts_v2                                  thrpt   25  3933324.449 ±  58046.222  ops/s

I only assumed 16 lucene segments for this test which is less for a OpenSearch node with multiple indices. With more, we will see more improvements. Also eviction wrt segment merges will be handled on a separate thread for v2 which is unaccounted for, but even with that, it should be highly performant.

@sgup432
Copy link
Contributor Author

sgup432 commented Feb 21, 2025

I re-ran test with 1mb cache, and assuming 50 lucene segments
Numbers are even better!

Performance Comparison: v1 vs v2

Benchmark Version Throughput (ops/s) Error (± ops/s) Performance Gain (v2 vs. v1)
Concurrent Get & Put (Mixed Load) v1 1,281,312.313 32,524.805 4.44x
v2 5,686,201.010 91,006.718
Get Performance (Read-Only in Mixed load) v1 538,790.219 82,806.104 8.16x
v2 4,399,376.709 102,910.001
Put Performance (Write-Only in Mixed Load) v1 742,522.095 80,860.990 1.73x
v2 1,286,824.300 28,852.757
Concurrent Puts (Write-Only Load) v1 919,128.018 67,900.245 4.24x
v2 3,895,563.219 56,761.564

@msokolov
Copy link
Contributor

This shows a nice improvement on the microbenchmark! But in a typical workload we expect to be spending most of our time executing queries rather than caching them, which will reduce the amount of time spent acquiring locks, and the contention, therefore the gains could become negligible -- or not. Is it possible in the benchmarking framework to simulate a more realistic workload?

@sgup432
Copy link
Contributor Author

sgup432 commented Feb 24, 2025

But in a typical workload we expect to be spending most of our time executing queries rather than caching them, which will reduce the amount of time spent acquiring locks, and the contention, therefore the gains could become negligible

It depends on the workload. Due to the query cache's caching policy, not all queries will be cached. However, with a workload that heavily utilizes the query cache, we can still expect overall benefits—maybe not as pronounced as the microbenchmark results above. These results suggest that the cache does not have to be a bottleneck itself.

Is it possible in the benchmarking framework to simulate a more realistic workload?

Yes. I am trying that as well. Do you have any recommendations, such as any existing benchmarks I could use for this use case?

@stefanvodita
Copy link
Contributor

Do you have any recommendations, such as any existing benchmarks I could use for this use case?

Have you looked at luceneutil?

@sgup432
Copy link
Contributor Author

sgup432 commented Feb 26, 2025

Have you looked at luceneutil?

I did. But I was not sure whether it works with the cache enabled? As with most benchmarks, caches are usually disabled to ensure accurate results.

@stefanvodita
Copy link
Contributor

I had a quick look and there are a lot of lines like setQueryCache(null); // don't bench the cache. I guess luceneutil made the same decision you were describing.

@sgup432
Copy link
Contributor Author

sgup432 commented Feb 26, 2025

I had a quick look and there are a lot of lines like setQueryCache(null); // don't bench the cache. I guess luceneutil made the same decision you were describing.

Ah thanks for this! I will try uncommenting that line and then running the benchmarks.

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

No branches or pull requests

3 participants