[ 
https://issues.apache.org/jira/browse/LUCENE-9038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16971036#comment-16971036
 ] 

Ben Manes commented on LUCENE-9038:
-----------------------------------

I added a per segment ring buffers for recording the query hits. When over half 
full then {{tryLock}} is used to optimistically drain back onto the LRU. When 
an entry is added via {{putIfAbsent}}, the buffers are drained prior to the 
eviction. The maps were changed to use {{ConcurrentHashMap}} so that reads can 
be lock-free when successful. This should improve read throughput, but is no 
longer a strict LRU like your tests expect. In the face of concurrency that 
shouldn't be expected, so the stats + lru ordering tests are intentionally 
broken.

This design is similar to Guava's cache which had forked the legacy CHM to add 
per-segment features. When I had proposed how to add LRU, I had stubbed the 
code using {{ConcurrentLinkedQueue}} read buffers. Unfortunately my intent to 
replace that with ring buffers never garnered interest, even though it improved 
performance 25x. This still suffers from hotspots as hot entries are in the 
same segment, but similarly its least invasive fix given existing code.

I also added a simple striped locking scheme for computing the query results. 
This should allow only a single thread to compute the results for a given query 
and avoid stampede effects. A subsequent thread would block waiting for the 
first to finish caching the results.

How does this look?

[^cache.patch] 

> Evaluate Caffeine for LruQueryCache
> -----------------------------------
>
>                 Key: LUCENE-9038
>                 URL: https://issues.apache.org/jira/browse/LUCENE-9038
>             Project: Lucene - Core
>          Issue Type: Improvement
>            Reporter: Ben Manes
>            Priority: Major
>         Attachments: CaffeineQueryCache.java, cache.patch
>
>
> [LRUQueryCache|https://github.com/apache/lucene-solr/blob/master/lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java]
>  appears to play a central role in Lucene's performance. There are many 
> issues discussing its performance, such as LUCENE-7235, LUCENE-7237, 
> LUCENE-8027, LUCENE-8213, and LUCENE-9002. It appears that the cache's 
> overhead can be just as much of a benefit as a liability, causing various 
> workarounds and complexity.
> When reviewing the discussions and code, the following issues are concerning:
> # The cache is guarded by a single lock for all reads and writes.
> # All computations are performed outside of the any locking to avoid 
> penalizing other callers. This  doesn't handle the cache stampedes meaning 
> that multiple threads may cache miss, compute the value, and try to store it. 
> That redundant work becomes expensive under load and can be mitigated with ~ 
> per-key locks.
> # The cache queries the entry to see if it's even worth caching. At first 
> glance one assumes that is so that inexpensive entries don't bang on the lock 
> or thrash the LRU. However, this is also used to indicate data dependencies 
> for uncachable items (per JIRA), which perhaps shouldn't be invoking the 
> cache.
> # The cache lookup is skipped if the global lock is held and the value is 
> computed, but not stored. This means a busy lock reduces performance across 
> all usages and the cache's effectiveness degrades. This is not counted in the 
> miss rate, giving a false impression.
> # An attempt was made to perform computations asynchronously, due to their 
> heavy cost on tail latencies. That work was reverted due to test failures and 
> is being worked on.
> # An [in-progress change|https://github.com/apache/lucene-solr/pull/940] 
> tries to avoid LRU thrashing due to large, infrequently used items being 
> cached.
> # The cache is tightly intertwined with business logic, making it hard to 
> tease apart core algorithms and data structures from the usage scenarios.
> It seems that more and more items skip being cached because of concurrency 
> and hit rate performance, causing special case fixes based on knowledge of 
> the external code flows. Since the developers are experts on search, not 
> caching, it seems justified to evaluate if an off-the-shelf library would be 
> more helpful in terms of developer time, code complexity, and performance. 
> Solr has already introduced [Caffeine|https://github.com/ben-manes/caffeine] 
> in SOLR-8241 and SOLR-13817.
> The proposal is to replace the internals {{LruQueryCache}} so that external 
> usages are not affected in terms of the API. However, like in {{SolrCache}}, 
> a difference is that Caffeine only bounds by either the number of entries or 
> an accumulated size (e.g. bytes), but not both constraints. This likely is an 
> acceptable divergence in how the configuration is honored.
> cc [~ab], [~dsmiley]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org

Reply via email to