lucene-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Yonik Seeley (JIRA)" <>
Subject [jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads
Date Fri, 20 Nov 2009 22:31:41 GMT


Yonik Seeley commented on LUCENE-2075:

bq. Also, the results for ConcurrentLRUCache are invalid (its hit rate is
way too high) - I think this is because its eviction process can take
a longish amount of time, which temporarily allows the map to hold way
too many entries, and means it's using up alot more transient RAM than
it should.

Yep - there's no hard limit.  It's not an issue in practice in Solr since doing the work to
generate a new entry to put in the cache is much more expensive than cache cleaning (i.e.
generation will never swamp cleaning).  Seems like a realistic benchmark would do some amount
of work on a cache miss?  Or perhaps putting it in lucene and doing real benchmarks?

bq. Another idea: I wonder whether a simple cache-line like cache would be sufficient. Ie,
we hash to a fixed slot and we evict whatever is

We need to balance the overhead of the cache with the hit ratio and the cost of a miss. for
the String intern cache, the cost of a miss is very low, hence lowering overhead but giving
up hit ratio is the right trade-off.  For this term cache, the cost of a miss seems relatively
high, and warrants increasing overhead to increase the hit ratio.

> Share the Term -> TermInfo cache across threads
> -----------------------------------------------
>                 Key: LUCENE-2075
>                 URL:
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Index
>            Reporter: Michael McCandless
>            Priority: Minor
>             Fix For: 3.1
>         Attachments:, LUCENE-2075.patch, LUCENE-2075.patch, LUCENE-2075.patch,
LUCENE-2075.patch, LUCENE-2075.patch, LUCENE-2075.patch
> Right now each thread creates its own (thread private) SimpleLRUCache,
> holding up to 1024 terms.
> This is rather wasteful, since if there are a high number of threads
> that come through Lucene, you're multiplying the RAM usage.  You're
> also cutting way back on likelihood of a cache hit (except the known
> multiple times we lookup a term within-query, which uses one thread).
> In NRT search we open new SegmentReaders (on tiny segments) often
> which each thread must then spend CPU/RAM creating & populating.
> Now that we are on 1.5 we can use java.util.concurrent.*, eg
> ConcurrentHashMap.  One simple approach could be a double-barrel LRU
> cache, using 2 maps (primary, secondary).  You check the cache by
> first checking primary; if that's a miss, you check secondary and if
> you get a hit you promote it to primary.  Once primary is full you
> clear secondary and swap them.
> Or... any other suggested approach?

This message is automatically generated by JIRA.
You can reply to this email to add a comment to the issue online.

To unsubscribe, e-mail:
For additional commands, e-mail:

View raw message