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 20:15:39 GMT

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

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

{quote}
I guess I worded this wrong. you are right only 1 in 10 seek'ed terms.
But it will read a lot of useless terms too. This is because it does not try to seek until
there is a mismatch.

first it will seek to \u0000\u0000\u0000\u0000NNN
edit: this will return 00000000 which fails, then it will seek to 0000NNN
this will be a match

since this was a match, next it will read sequentially the next term, which will not match,
so it must seek again.
now it must backtrack and will try 0001NNN, match, it will do the sequential thing again.
perhaps this optimization of 'don't seek unless you encounter a mismatch' is not helping the
caching?
(sorry i cant step thru this thing in my mind easily)
{quote}

So it sort of plays ping pong w/ the terms enum API, until it finds an
intersection.  (This is very similar to how filters are applied
currently).

In this case, I agree it should not bother with the "first=true" case
-- it never matches in this particular test -- it should simply seek
to the next one.  Inside the term enum API, that seek will fallback to
a scan, anyway, if it's "close" (within the same index block).

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

We should test performance of that.


> 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