lucene-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Michael McCandless (JIRA)" <j...@apache.org>
Subject [jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads
Date Tue, 24 Nov 2009 11:44:40 GMT

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

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

bq. With Query Scope you mean a whole query, so not only a MTQ? If you combine multiple AutomatonQueries
in a BooleanQuery it could also profit from the cache (as it is currently).

Right, I think the top level query would open up the scope... and free it once it's done running.

bq. I think until Flex, we should commit this and use the cache. When Flex is out, we may
think of doing this different.

OK let's go with the shared cache for now, and revisit once flex lands.  I'll open a new issue...

But should we drop cache to maybe 512?  Typing up 4MB RAM (with cache size 1024) for a "normal"
index is kinda alot...

> 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-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Mime
View raw message