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

Michael McCandless commented on LUCENE-2075:
--------------------------------------------

bq. Uwe i just wonder if the cache would in practice get used much.

This cache (mapping Term -> TermInfo) does get used alot for "normal"
atomic queries we first hit the terms dict to get the docFreq (to
compute idf), then later hit it again with the exact same term, to
get the TermDocs enum.

So, for these queries our hit rate is 50%, but, it's rather overkill
to be using a shared cache for this (query-private scope is much
cleaner).  EG a large automaton query running concurrently with other
queries could evict entries before they read the term the 2nd time.

Existing MTQs (except NRQ) which seek once and then scan to completion
don't hit the cache (though, I think they do double-load each term,
which is wasteful; likely this is part of the perf gains for flex).

NRQ doens't do enough seeking wrt iterating/collecting the docs for
the cache to make that much a difference.

The upcoming automaton query should benefit.... however in testing we
saw only the full linear scan benefit, which I'm still needing to get
to the bottom of.


> Share the Term -> TermInfo cache across threads
> -----------------------------------------------
>
>                 Key: LUCENE-2075
>                 URL: https://issues.apache.org/jira/browse/LUCENE-2075
>             Project: Lucene - Java
>          Issue Type: Improvement
>          Components: Index
>            Reporter: Michael McCandless
>            Assignee: Michael McCandless
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, LUCENE-2075.patch, 
> LUCENE-2075.patch, LUCENE-2075.patch, LUCENE-2075.patch, 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: java-dev-unsubscr...@lucene.apache.org
For additional commands, e-mail: java-dev-h...@lucene.apache.org

Reply via email to