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 19:33:39 GMT

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

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

{quote}
Mike, I think you also might be seeing strangeness with that wildcard test due to the fact
that most Terms automaton seeks to do not actually exist.
instead its simply 'the next possible subsequence', according to the DFA, and it relies on
sort order of TermEnum to do the rest...
{quote}

Hang on -- the weirdness I was seeing was for the *N query, which does
full linear scan...  as best I can tell, the weird GC problems with
LinkedHashMap completely explain that weirdness (and boy was
it weird!).

But it sounds like you're talking about the seek-intensive ????NNN
query?  In that case it's only 1 in 10 seek'd terms that don't exist
(though it is a rather contrived test).

I guess if we created a much more sparse index, then re-ran that
query, we'd see many more seeks to non-existent terms.

But I think in general seek to non-existent term is harmless, because,
since it did not exist, it's not like you (or the app) will turnaround
and ask for that term's docFreq, the TermDocs, etc.  Ie, we don't
cache that 'I could not find term XXX', but I think we don't need to.


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