lucene-dev mailing list archives

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

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

Robert Muir commented on LUCENE-2075:
-------------------------------------

bq. So it sort of plays ping pong w/ the terms enum API

ping-pong, i like it.

bq. So I guess if there's a non-empty common suffix you should just always seek?

the suffix is just for quick comparison, not used at all in seeking.

in general, you can usually compute the next spot to go, even on a match.
if its a wildcard of "ab*" and the enum is at abc, its pretty stupid to compute abd and seek
to it, so I don't. (as long as there is match, just keep reading).

otherwise I am seeking the whole time, whenever a term doesn't match, I calculate the next
spot to go to. 

we can work it on that other issue if you want, i don't mean to clutter this one up... happy
to see you improve the *N case here :)

> 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, 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