You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by "Michael McCandless (JIRA)" <ji...@apache.org> on 2009/11/16 22:34:39 UTC

[jira] Created: (LUCENE-2075) Share the Term -> TermInfo cache across threads

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
            Priority: Minor
             Fix For: 3.1


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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Uwe Schindler updated LUCENE-2075:
----------------------------------

    Attachment: LUCENE-2075.patch

As PriorityQueue is generified since Lucene 3.0, I added missing generics. The class now compiles without unchecked warnings. I also removed lots of casts and parameterized the missing parts. Also added K type for inner map.

Nice work, even if I do not understand it completely :-)

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless updated LUCENE-2075:
---------------------------------------

    Attachment: LUCENE-2075.patch

Attached patch; all tests pass:

  * Switches the terms dict cache away from per-thread cache to shared
    (DoubleBarrelLRU) cache

  * Still uses the cache when seeking the term enum

However, I'm baffled: I re-ran the BenchWildcard test and saw no
measurable improvement in ????NNN query (yet, I confirmed it's now
storing into and then hitting on the cache), but I did see a gain in
the *N query (from ~4300 msec before to ~3500 msec) which I can't
explain because that query doens't use the cache at all (just the
linear scan).  I'm confused....

Robert maybe you can try this patch plus automaton patch and see if
you see this same odd behavior?


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781908#action_12781908 ] 

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

bq. I would keep it as it is, because we already minimized memory requirements, because before the cache was per-thread.

OK let's leave it at 1024, but with flex (which automaton query no longer needs the cache for), I think we should drop it and/or cutover to query-private scope.  I don't think sucking up 4 MB of RAM for this rather limited purpose is warranted.  I'll add a comment on LUCENE-2093.

> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yonik Seeley updated LUCENE-2075:
---------------------------------

    Attachment: LUCENE-2075.patch

Here's a simplified version of Solr's ConcurrentLRUCache.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Jason Rutherglen (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782071#action_12782071 ] 

Jason Rutherglen commented on LUCENE-2075:
------------------------------------------

{quote} And honestly I'm still tempted to do away with this
cache altogether and create a "query scope", private to each
query while it's running, where terms dict (and other places
that need to, over time) could store stuff. That'd give a
perfect within-query hit rate and wouldn't tie up any long term
RAM... {quote}

Sounds better than the caching approach which if I recall
correctly, Michael B. noted was kind of a hack (i.e. this isn't
really a cache, isn't it just a convenient way of recalling
immediately previously read data whose scope is really within
the query itself).

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781454#action_12781454 ] 

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

Uwe, why do you see so much variance in your times?  EG best for trie is 5-6 msec, but avg is ~190-205 msec.

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781383#action_12781383 ] 

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

bq. Robert maybe you can try this patch plus automaton patch and see if you see this same odd behavior?

confirmed, though on my machine, it is 4 second avg *N versus 6 second avg *N :)

I havent looked at the code, but fyi, even the smart mode is always "in term order" traversal, its just skipping over terms.
I think numeric range might do "out of order"? Uwe can confirm.
I don't know if this matters, either.


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781110#action_12781110 ] 

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

{quote}
bq. BTW the flex branch fixes this - TermsEnum.seek always checks the cache.

Can we fix this for trunk, too? But I think this issue talks about trunk.
{quote}

Right, this issue is about trunk (not flex API).  I think we could fix this (see my suggestions above), basically decoupling "useCache" from "seekTheEnum"... but we have to fix the terminfo cache to also store the term's ord.  I'll try out this approach...

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781045#action_12781045 ] 

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

Hi, I applied automaton patch and its benchmark (LUCENE-1606) against the flex branch, and kept with the old TermEnum api.

I tested two scenarios, an old index created with 3.0 (trunk) and a new index created with flex branch.
in both cases, its slower than trunk, but I assume this is due to flex branch not being optimized yet?... (last i saw it used new String() placeholder for utf conversion)

but i think it is fair to compare the flex branch with itself, with old idx versus new idx. I can only assume with a new idx it is using the caching.
these numbers are stable on HEAD and do not deviate much.
feel free to look at the benchmark code over there and suggest improvements if you think there is an issue with it.

||Pattern||Iter||AvgHits||AvgMS (old idx)||AvgMS (new idx)||
|N?N?N?N|10|1000.0|86.6|70.2|
|?NNNNNN|10|10.0|3.0|2.0|
|??NNNNN|10|100.0|12.5|7.2|
|???NNNN|10|1000.0|86.9|34.8|
|????NNN|10|10000.0|721.2|530.5|
|NN??NNN|10|100.0|8.3|4.0|
|NN?N*|10|10000.0|149.1|143.2|
|?NN*|10|100000.0|1061.4|836.7|
|*N|10|1000000.0|16329.7|11480.0|
|NNNNN??|10|100.0|2.7|2.2|



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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781627#action_12781627 ] 

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

bq. I am quite sure that also Robert's test is random (as he explained).

It's not random -- it's the specified pattern, parsed to
WildcardQuery, run 10 times, then take best or avg time.

{quote}
I fixed the test to only test few queries and repeat them quite often. For precStep=4 and long values, I got about 28 seeks per query, but there was no speed improvement. Maybe 28 seeks / query is too less for an effect. The number of terms seen per query was 70, so about 2.5 terms/seek which is typical for precStep=4 with this index value density (5 Mio random number in the range 2^-63..2^63). It is also important, that the random ranges hit many documents (in avg 1/3 of all docs), so most time in my opinion is used in collecting the results. Maybe I should try shorter and limited ranges.
{quote}

OK... it sounds like the differences may simply be in the noise for NRQ.

bq. If you deprecate SimpleLRUCache, you can also deprecate the MapCache abstract super class. But I wouldn't like to deprecate these classes, as I for myself use them in my own code for e.g. caching queries etc.

Hmm... I felt like because nothing in Lucene uses SimpleLRUCache
anymore, we should deprecate & remove it.  I don't think we should be
in the business of creating/exporting (as public classes) such
collections, unless we continue to use them.

I even wonder why we don't put these classes into oal.index, and make
them package private.  Ie, I think we make them public only to share
them across packages within lucene, not because we want/expect apps to
consume them.  The term "public" is heavily overloaded,
unfortunately.

Also, I already put a strong note to this effect in
DoubleBarrelLRUCache, ie we reserve future right to up and remove the
class.

bq. And even if you deprecate the Map, why remove the tests, they should stay alive until the class is removed?

Oh good point -- I'll resurrect & keep it.


> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781887#action_12781887 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

For testing we could add two AtomicIntegers to the cache that counts hits and requests to get a hit rate, only temporary to not affect performance.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781628#action_12781628 ] 

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

bq. You see the effect of the caching. The code ran 5000 rewrites with each query repeated 20 times.

OK so it sounds like NRQ in practice won't see a boost from this (it's too good already -- does too little seeking ;), though with this [contrived] test you were able to show the new seek cache is having a small positive effect.

> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yonik Seeley updated LUCENE-2075:
---------------------------------

    Attachment: LUCENE-2075.patch

New patch attached - while refreshing my memory on the exact algorithm, I noticed a bug :-)
Things won't work well after 2B accesses since Integer.MAX_VALUE is used instead of Long.MAX_VALUE.
Need to go fix Solr now too :-)

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Jason Rutherglen (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12778645#action_12778645 ] 

Jason Rutherglen commented on LUCENE-2075:
------------------------------------------

Solr used CHM as an LRU, however it turned out to be somewhat
less than truly LRU? I'd expect Google Collections to offer a
concurrent linked hash map however no dice?
http://code.google.com/p/google-collections/ 

Maybe there's a way to build a concurrent LRU using their CHM?

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781886#action_12781886 ] 

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

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

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Mark Miller (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779255#action_12779255 ] 

Mark Miller commented on LUCENE-2075:
-------------------------------------

When/If yonik finally pops up here, he will have some good info to add I think.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782085#action_12782085 ] 

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

bq. Sounds better than the caching approach

Right, the cache is a blunt (but effective) tool for simply sharing
information w/in a single query.  LUCENE-1195 was the original issue.

I'd rather make the private query scope (LUCENE-2093) and then remove
this cache, but, that's a bigger change.

Also, once flex lands, the MTQ's will require the cache alot less
(because the TermsEnum API can produce a docs() directly w/o going
back to the terms dict); my guess is we can drop the cache size to
something relatively tiny (32) and get most of the gains.


> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yonik Seeley updated LUCENE-2075:
---------------------------------

    Attachment: LUCENE-2075.patch

Here's a new version extending Cache<K,V>

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782141#action_12782141 ] 

Robert Muir edited comment on LUCENE-2075 at 11/24/09 8:21 PM:
---------------------------------------------------------------

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 regex of "ab[cd]" 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 :)

edit: remove the * for simplicity

      was (Author: rcmuir):
    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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780944#action_12780944 ] 

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


bq. a singe numeric range query will be doing many term seeks (one at the start of each enumeration). It doesn't look like these will currently utilize the cache - can someone refresh my memory on why this is?

You're right -- here's the code/comment:

{code}
  /** Returns an enumeration of terms starting at or after the named term. */
  public SegmentTermEnum terms(Term term) throws IOException {
    // don't use the cache in this call because we want to reposition the
    // enumeration
    get(term, false);
    return (SegmentTermEnum)getThreadResources().termEnum.clone();
  }
{code}

I think this is because "useCache" (the 2nd arg to get) is overloaded
-- if you look at get(), if useCache is true and you have a cache hit,
it doesn't do it's "normal" side-effect of repositioning the
thread-private TermEnum.  So you'd get incorrect results.

If get had a 2nd arg "repositionTermEnum", to decouple caching from
repositioning, then we could make use of the cache for NRQ (& soon
AutomatonTermEnum as well), though, this isn't so simple because the
cache entry (just a TermInfo) doesn't store the term's ord.  And we
don't want to add ord to TermInfo since, eg, this sucks up alot of
extra RAM storing the terms index.  Probably we should make a new
class that's used for caching, and not reuse TermInfo.

This was also done before NumericRangeQuery, ie, all MTQs before NRQ
did a single seek.

BTW the flex branch fixes this -- TermsEnum.seek always checks the
cache.


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780819#action_12780819 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

Aside: a singe numeric range query will be doing many term seeks (one at the start of each enumeration).  It doesn't look like these will currently utilize the cache - can someone refresh my memory on why this is?  We should keep the logic that prevents the cache while iterating over terms with a term enumerator, but it seems like using the cache for the initial seek would be nice.

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780946#action_12780946 ] 

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

{quote}
bq. Also, the results for ConcurrentLRUCache are invalid (its hit rate is way too high) - I think this is because its eviction process can take a longish amount of time, which temporarily allows the map to hold way too many entries, and means it's using up alot more transient RAM than it should.

Yep - there's no hard limit. It's not an issue in practice in Solr since doing the work to generate a new entry to put in the cache is much more expensive than cache cleaning (i.e. generation will never swamp cleaning).  Seems like a realistic benchmark would do some amount of work on a cache miss? Or perhaps putting it in lucene and doing real benchmarks?
{quote}

I agree the test is synthetic, so the blowup we're seeing is a worse
case sitatuion, but are you really sure this can never be hit in
practice?

EG as CPUs gain more and more cores... it becomes more and more
possible with time that the 1 thread that's trying to do the cleaning
will be swamped by the great many threads generating.  Then if the CPU
is over-saturated (too many threads running), that 1 thread doing the
cleaning only gets slices of CPU time vs all the other threads that
may be generating...

It makes me nervous using a collection that, in the "perfect storm",
suddenly consumes way too much RAM.  It's a leaky abstraction.

That said, I agree the test is obviously very synthetic.  It's not
like a real Lucene installation will be pushing 2M QPS through Lucene
any time soon...

But still I'm more comfortable w/ the simplicity of the double-barrel
approach.  In my tests its performance is in the same ballpark as
ConcurrentLRUCache; it's much simpler; and the .clear() calls appear
in practice to very quickly free up the entries.

{quote}
bq. Another idea: I wonder whether a simple cache-line like cache would be sufficient. Ie, we hash to a fixed slot and we evict whatever is there.

We need to balance the overhead of the cache with the hit ratio and the cost of a miss. for the String intern cache, the cost of a miss is very low, hence lowering overhead but giving up hit ratio is the right trade-off. For this term cache, the cost of a miss seems relatively high, and warrants increasing overhead to increase the hit ratio.
{quote}

OK I agree.

Yet another option... would be to create some sort of "thread-private
Query scope", ie, a store that's created & cleared per-Query where
Lucene can store things.  When a Term's info is retrieved, it'd be
stored here, and then that "query-private" cache is consulted whenever
that Term is looked up again within that query.  This would be the
"perfect cache" in that a single query would never see its terms
evicted due to other queries burning through the cache...

Though, net/net I suspect the overhead of creating/pulling from this
new cache would just be an overall search slowdown in practice.


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780795#action_12780795 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

bq. Also, the results for ConcurrentLRUCache are invalid (its hit rate is
way too high) - I think this is because its eviction process can take
a longish amount of time, which temporarily allows the map to hold way
too many entries, and means it's using up alot more transient RAM than
it should.

Yep - there's no hard limit.  It's not an issue in practice in Solr since doing the work to generate a new entry to put in the cache is much more expensive than cache cleaning (i.e. generation will never swamp cleaning).  Seems like a realistic benchmark would do some amount of work on a cache miss?  Or perhaps putting it in lucene and doing real benchmarks?

bq. Another idea: I wonder whether a simple cache-line like cache would be sufficient. Ie, we hash to a fixed slot and we evict whatever is
there.

We need to balance the overhead of the cache with the hit ratio and the cost of a miss. for the String intern cache, the cost of a miss is very low, hence lowering overhead but giving up hit ratio is the right trade-off.  For this term cache, the cost of a miss seems relatively high, and warrants increasing overhead to increase the hit ratio.


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781407#action_12781407 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

What about replacing the expensive division with a comparison with zero?

if (putCount.decrementAndGet()==0) {
  secondary.clear();
  swapCount.getAndIncrement();
  putCount.set(maxSize);
}

Also, I don't think there's a need for swapCount?  A simple "volatile boolean swapped" should do?


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779639#action_12779639 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Hi Yonik, thaks, that you used my class, but I found one type erasure problem in the PQueue, because thee Heap is erasured to Object[] by javac. The getValues() tries to cast this array -> ClassCastException. This is described here: http://safalra.com/programming/java/wrong-type-erasure/
The same happens in myInsertWithOverflow().

Will fix.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless updated LUCENE-2075:
---------------------------------------

    Attachment: LUCENE-2075.patch

Thanks Uwe!

I attached another one: made DBLRU final, tweaked javadocs, fixed spelling in the saturation comment, add you guys to the CHANGES entry.

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782039#action_12782039 ] 

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

bq. Woops - I was wrong here - Robert's test is random: on each iteration, it replaces any N's in the pattern w/ a random number 0-9.

Yeah, the terms are equally distributed 0000000-9999999 though, just a "fill"
The wildcard patterns themselves are filled with random numbers.

This is my basis for the new wildcard test btw, except maybe 1-10k, definitely want over 8192 :)
unless you have better ideas?

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782152#action_12782152 ] 

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

OK let's move this over to LUCENE-1606?  I'll respond there.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782046#action_12782046 ] 

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

bq. This is my basis for the new wildcard test btw, except maybe 1-10k, definitely want over 8192

Sounds great :)

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779514#action_12779514 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

The Solr one could be simplified a lot for Lucene... no need to keep some of the statistics and things like "isLive".

Testing via something like the double barrel approach will be tricky.  The behavior of ConcurrentLRUCache (i.e. the cost of puts) depends on the access pattern - in the best cases, a single linear scan would be all that's needed.  In the worst case, a subset of the  map needs to go into a priority queue.  It's all in markAndSweep... that's my monster - let me know if the comments don't make sense.

How many entries must be removed to be considered a success also obviously affects whether a single linear scan is enough.  If that's often the case, some other optimizations can be done such as not collecting the entries for further passes:
{code}
          // This entry *could* be in the bottom group.
          // Collect these entries to avoid another full pass... this is wasted
          // effort if enough entries are normally removed in this first pass.
          // An alternate impl could make a full second pass.
{code}

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781621#action_12781621 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

I changed my benchmark to better show the seek caching effect. For NRQ the overall improvement has no neglectible effect.

I chenged the rewrite mode of the NRQ to SCORING_BOOLEAN_QUEY and then just rewrote the queries to BQ and measured time. So no TermDocs/Collecting was in effect:

trunk: avg number of terms: 68.537; avg seeks=28.838; best time=1.022756 ms; worst time=17.036802 ms; avg=1.8388833272 ms
patch: avg number of terms: 68.537; avg seeks=28.838; best time=1.066616 ms; worst time=12.80917 ms; avg=1.6932529156 ms

You see the effect of the caching. The code ran 5000 rewrites with each query repeated 20 times.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781882#action_12781882 ] 

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

i am still triyng to figure out the use case.

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

isn't there a method i can use to force these to combine into one AutomatonQuery (I can use union, intersection, etc)?
I haven't done this, but we shouldnt create a private scoped-cache for something like this?

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781040#action_12781040 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

bq. BTW the flex branch fixes this - TermsEnum.seek always checks the cache.

Can we fix this for trunk, too? But I think *this* issue talks about trunk.

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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Uwe Schindler updated LUCENE-2075:
----------------------------------

    Attachment: LUCENE-2075.patch

Updated patch, adds missing @Overrides, we added in 3.0 and also makes the private PQ implement Iterable, the markAndSweep code is now synactical sugar :-)

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Mark Miller (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Mark Miller updated LUCENE-2075:
--------------------------------

    Attachment: ConcurrentLRUCache.java

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782117#action_12782117 ] 

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

bq. hould we additionally deprecate the SimpleMapCache, as it is an abstract super class no longer used?

Woops, sorry, you said that already and I forgot -- I'll do 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
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782069#action_12782069 ] 

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

OK as best I can tell, the reason why linear scan shows so much faster
with the new cache, is some kind of odd GC problem when you use
LinkedHashMap.... the DoubleBarrelLRUCache doesn't tickle GC in this
way.

If you turn on -verbose:gc when running the bench, you see this
healthy GC behavior during warmup (running wildcard "*"):

{code}
[GC 262656K->15659K(1006848K), 0.0357409 secs]
[GC 278315K->15563K(1006848K), 0.0351360 secs]
[GC 278219K->15595K(1006848K), 0.0150112 secs]
[GC 278251K->15563K(1006848K), 0.0054310 secs]
{code}
All minor collections, all fairly fast, all rather effective (~270 MB
down to ~15 MB).

But then when the test gets to the the *N query:

{code}
[GC 323520K->33088K(1022272K), 0.0377057 secs]
[GC 338432K->78536K(990592K), 0.1830592 secs]
[GC 344776K->118344K(1006336K), 0.1205320 secs]
[GC 384584K->158080K(987264K), 0.2340810 secs]
[GC 400640K->194264K(979136K), 0.2139520 secs]
[GC 436824K->230488K(989760K), 0.2017131 secs]
[GC 463192K->266501K(969152K), 0.1932188 secs]
[GC 499205K->301317K(989632K), 0.1918106 secs]
[GC 530437K->335541K(990080K), 0.1907594 secs]
[GC 564661K->369749K(990528K), 0.1905007 secs]
[GC 599445K->404117K(990208K), 0.1922657 secs]
[GC 633813K->438477K(991680K), 0.1994350 secs]
[GC 670157K->474250K(991040K), 0.2073795 secs]
[GC 705930K->508842K(992832K), 0.2061273 secs]
[GC 742570K->543770K(991936K), 0.1980306 secs]
[GC 777498K->578634K(994560K), 0.1975961 secs]
[GC 815818K->614010K(993664K), 0.2042480 secs]
[GC 851194K->649434K(996096K), 0.1940145 secs]
[GC 889754K->686551K(995264K), 0.1991030 secs]
[Full GC 686551K->18312K(995264K), 0.1838671 secs]
[GC 258632K->54088K(997120K), 0.0735258 secs]
[GC 296456K->90280K(996288K), 0.1382187 secs]
[GC 332648K->126512K(998592K), 0.1427443 secs]
[GC 371888K->163096K(997824K), 0.1472803 secs]
{code}

The minor collections are not nearly as effective -- way too many
objects are for some reason being marked as live (even though they are
not) and promoted to the older generation, thus making the minor
collection much more costly and also requiring major collection every
so often.

Now here's the really crazy thing: if I move the *N query up to be the
first query the benchmark runs, GC is healthy:

{code}
[GC 323868K->17216K(1027840K), 0.0060598 secs]
[GC 322496K->17128K(1006016K), 0.0062586 secs]
[GC 322408K->17160K(1027712K), 0.0008879 secs]
[GC 321672K->17192K(1027776K), 0.0003269 secs]
[GC 321704K->18669K(1028608K), 0.0012964 secs]
[GC 324205K->18741K(1027968K), 0.0104134 secs]
[GC 324277K->18613K(1029632K), 0.0083720 secs]
[GC 326261K->18677K(1029056K), 0.0003520 secs]
{code}

And the query runs about as fast as w/ the new cache.

So..... somehow, running the other queries sets object state up to
confuse GC later.  I'm pretty sure it's the linking that the
LinkedHashMap (in SimpleLRUCache) is doing, because if I forcefully
turn off all caching, GC acts healthy again, and that query runs as
fast as it does w/ the patch.

DoubleBarrelLRUCache doens't tickle GC in this way, so the *N query
runs fast with it.

Sheesh!!


> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781594#action_12781594 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

bq. I wonder if your test is getting any cache hits at all - if you do random ranges, and never repeat queries, then likely your hit rate is quite low?

I am quite sure that also Robert's test is random (as he explained).

I fixed the test to only test few queries and repeat them quite often. For precStep=4 and long values, I got about 28 seeks per query, but there was no speed improvement. Maybe 28 seeks / query is too less for an effect. The number of terms seen per query was 70, so about 2.5 terms/seek which is typical for precStep=4 with this index value density (5 Mio random number in the range 2^-63..2^63). It is also important, that the random ranges hit many documents (in avg 1/3 of all docs), so most time in my opinion is used in collecting the results. Maybe I should try shorter and limited ranges.

Robert: How many term enum seeks did your queries produce?

Currently I am indexing a 100 Mio docs, precStep=1, long values index (64 terms per doc). Let's see what happens here.

If you deprecate SimpleLRUCache, you can also deprecate the MapCache abstract super class. But I wouldn't like to deprecate these classes, as I for myself use them in my own code for e.g. caching queries etc. And even if you deprecate the Map, why remove the tests, they should stay alive until the class is removed?

Uwe

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782121#action_12782121 ] 

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

{quote}
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).
{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
this will return 0000NNN, and 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?


> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780824#action_12780824 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

The initial seek should really be optimized, this also affects the new AutomatonTermEnum for the future of RegEx queries, WildCardQueries and maybe FuzzyQueries with DFAs. With the automaton enum, depending of the DFA, there can be lot's of seeks (LUCENE-1606).

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781633#action_12781633 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

bq. For NRQ the overall improvement has no neglectible effect. 

Perhaps it's just the ratio of seeks to TermDocs.next() for the relatively large indexes you were testing against?

> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Uwe Schindler updated LUCENE-2075:
----------------------------------

    Attachment:     (was: LUCENE-2075.patch)

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781505#action_12781505 ] 

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

bq. Because of random ranges on the whole range. 

Ahh, OK.

I wonder if your test is getting any cache hits at all -- if you do random ranges, and never repeat queries, then likely your hit rate is quite low?

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


[jira] Assigned: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless reassigned LUCENE-2075:
------------------------------------------

    Assignee: Michael McCandless

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782035#action_12782035 ] 

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

{quote}
bq. I am quite sure that also Robert's test is random (as he explained).

It's not random - it's the specified pattern, parsed to
WildcardQuery, run 10 times, then take best or avg time.
{quote}

Woops -- I was wrong here -- Robert's test is random: on each iteration, it replaces any N's in the pattern w/ a random number 0-9.

Still baffled on why the linear scan shows gains w/ the cache... digging.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Paul Smith (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779641#action_12779641 ] 

Paul Smith commented on LUCENE-2075:
------------------------------------

bq. This cache impl should be able to support 1B operations per second for almost 300 years (i.e. the time it would take to overflow a long).

Hopefully Sun has released Java 7 by then. :)

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782094#action_12782094 ] 

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

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


> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Uwe Schindler updated LUCENE-2075:
----------------------------------

    Attachment: LUCENE-2075.patch

Patch that fixes the bug in javac with typed arrays (because of that it does not allow typed arrays without unchecked casts...).

I fixed the PQueue by returning a List<CacheEntry<K,V>> values() and also mad the private maxSize in the PriorityQueue protected. So it does not need to implement an own insertWithOverflow. As this class moves to Lucene Core, we should not make such bad hacks. 

We need a good testcase for the whole cache class. It was hard to me to find a good test that hits the PQueue at all (its only used in special cases). Hard stuff :(

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779631#action_12779631 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Sorry a small problem with cast. Will upload new patch, soon.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781667#action_12781667 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

bq. Perhaps it's just the ratio of seeks to TermDocs.next() for the relatively large indexes you were testing against?

Exactly thats the difference to AutomatonQuery: The AutomatonTermEnum does lot's of seeking in the TermEnum to not scan all terms. But each term normally has the same (small) docFreq. For NRQ, some terms have a very high docFreq (e.g. up to 100,000 for this large index in lowest precision), so most of the time in the query is enumerating docs.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781112#action_12781112 ] 

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


bq. in both cases, its slower than trunk, but I assume this is due to flex branch not being optimized yet?

The automaton benchmark looks great -- I'll dig into why the flex branch
is slower in both of these cases.

The first case tests old API on top of an old index, which I'm
surprised to see not matching trunk's performance.  The flex changes
are supposed to "optimize" that case by directly using the old (trunk)
code.

The second test tests old API emulated over a flex index, which I'm
also surprised to see is not faster than trunk -- there must be
something silly going on in the API emulation.

I'll dig...

When I tested MTQs (TermRangeQuery, WildcardQuery), using flex API on
flex index, they were reasonably faster, so I'll also try to get
automaton's FilteredTermEnum cutover to the flex API, and test 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
>            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
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781880#action_12781880 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

I would keep it as it is, because we already minimized memory requirements, because before the cache was per-thread.

> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless updated LUCENE-2075:
---------------------------------------

    Attachment: LUCENE-2075.patch

New patch, folding in Yonik's suggestions, adding a unit test (carried
over from TestSimpleLRUCache -- I'll "svn mv" when I commit it), and
deprecating SimpleLRUCache.


> 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


[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779354#action_12779354 ] 

Uwe Schindler edited comment on LUCENE-2075 at 11/18/09 9:39 AM:
-----------------------------------------------------------------

Should this ConcurrentLRUCache not better be fitted into the o.a.l.util.cache package?

About the Solr implementation: The generification has a "small" problem: get(), contains(), remove() and other by-key-querying methods should use Object as type for the key, not the generic K, because it is not bad to test with contains any java type (it would just return false). The sun generic howto explains that, also this one: [http://smallwig.blogspot.com/2007/12/why-does-setcontains-take-object-not-e.html]

Very funny video about that: [http://www.youtube.com/watch?v=wDN_EYUvUq0] (explaination starts at 4:35)

      was (Author: thetaphi):
    Should this ConcurrentLRUCache not better be fitted into the o.a.l.util.cache package?

About the Solr implementation: The generification has a "small" problem: get(), contains(), remove() and other by-key-querying methods should use Object as type for the key, not the generic V, because it is not bad to test with contains any java type (it would just return false). The sun generic howto explains that. Very funny video about that: [http://www.youtube.com/watch?v=wDN_EYUvUq0] (explaination starts at 4:35)
  
> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781410#action_12781410 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

I tested with an 5 mio doc index containing trie ints, but it seems that trie does not really profit from the seeking cache. With the default precStep of 4 no difference (max. 16 seeks per query), and with precStep of 1 (max. 64 seeks per query) it was even a little bit slower on average (???). The test compares also with FieldCacheRangeFilter which is always slower, also the field cache loading time did not really change (linear scan in term enum).

PrecisionStep: 4
trunk:
loading field cache time: 6367.667678 ms
avg number of terms: 68.1
TRIE:       best time=6.323709 ms; worst time=414.367469 ms; avg=201.18463369999998 ms; sum=32004735
FIELDCACHE: best time=64.770523 ms; worst time=265.487652 ms; avg=155.5479675 ms; sum=32004735

patch:
loading field cache time: 6295.055377 ms
avg number of terms: 68.1
TRIE:       best time=5.288102 ms; worst time=415.290771 ms; avg=195.72079685 ms; sum=32004735
FIELDCACHE: best time=65.511957 ms; worst time=202.482438 ms; avg=138.69083925 ms; sum=32004735

---

PrecisionStep: 1
trunk:
loading field cache time: 6416.105399 ms
avg number of terms: 19.85
TRIE:       best time=6.51228 ms; worst time=410.624255 ms; avg=192.33796475 ms; sum=32002505
FIELDCACHE: best time=65.349088 ms; worst time=211.308979 ms; avg=143.71657580000002 ms; sum=32002505

patch:
loading field cache time: 6809.792026 ms
avg number of terms: 19.85
TRIE:       best time=6.814832 ms; worst time=436.396525 ms; avg=205.6526038 ms; sum=32002505
FIELDCACHE: best time=64.939539 ms; worst time=277.474371 ms; avg=142.58939345 ms; sum=32002505

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781460#action_12781460 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Because of random ranges on the whole range. If you only request a very short range, it is faster (less seeks because maybe only highest precision affected for very short ranges) vs a full range query which may seek the maximum count. It is reproduceable, because the random seed was identical.

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781385#action_12781385 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

bq. think numeric range might do "out of order"? Uwe can confirm.

No its also in order. It starts with highest precision (first lower end, then upper end sub-range), which has a shift value of 0. This is smaller that lower prec terms with a bigger shift value that come later. And of course each sub-range is in ascending order because the terms are :-)

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781452#action_12781452 ] 

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

{quote}
What about replacing the expensive division with a comparison with zero?
{code}
if (putCount.decrementAndGet()==0) { secondary.clear(); swapCount.getAndIncrement(); putCount.set(maxSize); }
{code}
{quote}

Good idea!  I'll do that.

bq. Also, I don't think there's a need for swapCount? A simple "volatile boolean swapped" should do?

Excellent -- I'll do that too.


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779055#action_12779055 ] 

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

Since Solr already has already created a concurrent LRU, I think we simply reuse that?  Is there any reason not  to?

I don't think we need absolutely truly LRU for the terminfo cache.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781885#action_12781885 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

...not only AutomatonQueries can be combined, they can also be combined with other queries and then make use of the cache.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781869#action_12781869 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

{quote}
And honestly I'm still tempted to do away with this cache altogether
and create a "query scope", private to each query while it's running,
where terms dict (and other places that need to, over time) could
store stuff. That'd give a perfect within-query hit rate and wouldn't
tie up any long term RAM...
{quote}

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

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

> 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


[jira] Resolved: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless resolved LUCENE-2075.
----------------------------------------

    Resolution: Fixed

Thanks all!

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael Busch (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779638#action_12779638 ] 

Michael Busch commented on LUCENE-2075:
---------------------------------------

{quote}
Things won't work well after 2B accesses since Integer.MAX_VALUE is used
{quote}

>From ReentrantLock javadocs:
"This lock supports a maximum of 2147483648 recursive locks by the same thread."

I think you only use the lock for markAndSweep and everything else uses atomics, but ConcurrentHashMap uses ReentrantLocks internally for each segment. So overall, things wil probably run longer than 2B ops, but not sure how long.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779646#action_12779646 ] 

Uwe Schindler edited comment on LUCENE-2075 at 11/18/09 8:39 PM:
-----------------------------------------------------------------

Patch that fixes the bug in javac with typed arrays (because of that it does not allow generic array creation - the problem is that heap is a generic array in PQ, but implemented as Object[]).

I fixed the PQueue by returning a List<CacheEntry<K,V>> values() and also made the private maxSize in the PriorityQueue protected. So it does not need to implement an own insertWithOverflow. As this class moves to Lucene Core, we should not make such bad hacks. 

We need a good testcase for the whole cache class. It was hard to me to find a good test that hits the PQueue at all (its only used in special cases). Hard stuff :(

      was (Author: thetaphi):
    Patch that fixes the bug in javac with typed arrays (because of that it does not allow typed arrays without unchecked casts...).

I fixed the PQueue by returning a List<CacheEntry<K,V>> values() and also mad the private maxSize in the PriorityQueue protected. So it does not need to implement an own insertWithOverflow. As this class moves to Lucene Core, we should not make such bad hacks. 

We need a good testcase for the whole cache class. It was hard to me to find a good test that hits the PQueue at all (its only used in special cases). Hard stuff :(
  
> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782112#action_12782112 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Should we additionally deprecate the SimpleMapCache, as it is an abstract super class no longer used? The LinkedHashMap based cache simply extends this class and initializes it with an LinkedHashMap instance in the ctor.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ 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-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781863#action_12781863 ] 

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

Well, I just kept 1024 since that's what we currently do ;)

OK I just did a rough tally -- I think we're looking at ~100 bytes (on
32 bit JRE) per entry, including CHMs HashEntry, array in CHM,
TermInfoAndOrd, Term & its String text.

Not to mention DBLRU has 2X multiplier at peak, so 200 bytes.

So at 1024 we're looking at ~200KB peak used by this cache already,
per segment which is able to saturate that cache... so for a 20
segment index you're at ~4MB additional RAM consumed... so I don't
think we should increase this default.

Also, I don't think this cache is/should be attempting to achieve a
high hit rate *across* queries, only *within* a single query when that
query resolves the Term more than once.

I think caches that wrap more CPU, like Solr's query cache, are where
the app should aim for high hit rate.

Maybe we should even decrease the default size here -- what's
important is preventing in-fligh queries from evicting one another's
cache entries.

For NRQ, 1024 is apparently already plenty big for that (relatively
few seeks occur).

For automaton query, which does lots of seeking, once flex branch
lands there is no need for the cache (each lookup is done only once,
because the TermsEnum actualEnum is able to seek).  Before flex lands,
the cache is important, but only for automaton query I think.

And honestly I'm still tempted to do away with this cache altogether
and create a "query scope", private to each query while it's running,
where terms dict (and other places that need to, over time) could
store stuff.  That'd give a perfect within-query hit rate and wouldn't
tie up any long term RAM...


> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781372#action_12781372 ] 

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

I ended up subclassing TermInfo (had to remove its "final" modifier) to make a TermInfoAndOrd class that adds "int termOrd", and then fixed TermInfosReader.get to create that class and put/get into cache.

Now, in get we always consult the cache, and store into it for the non-sequential case, but now the 2nd boolean arg is "mustSeekEnum".  So if we have a cache hit, but must seek the enum, we fall through and do the existing scan/seek logic, but avoid the binary search through the terms index since we can use the ord from the cache hit.

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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless updated LUCENE-2075:
---------------------------------------

    Attachment: LUCENE-2075.patch

Also deprecates SimpleMapCache.

> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12780977#action_12780977 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

bq. I agree the test is synthetic, so the blowup we're seeing is a worse case sitatuion, but are you really sure this can never be hit in practice?

I'm personally comfortable that Solr isn't going to hit this for it's uses of the cache... it's simply the relative cost of generating a cache entry vs doing some cleaning.

bq. But still I'm more comfortable w/ the simplicity of the double-barrel approach. In my tests its performance is in the same ballpark as ConcurrentLRUCache;

But it wouldn't be the same performance in Lucene - a cache like LinkedHashMap would achieve a higher hit rate in real world scenarios.


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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781855#action_12781855 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Just one question: The cache is initialized with max 1024 entries. Why that number. If we share the cache between multiple threads, maybe we should raise the max size. Or make it configureable?

The entries in the cache are not very costly, why not use 8192 or 16384, MTQs would be happy with 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
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781386#action_12781386 ] 

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

Uwe, thanks. so both the enums behave in a very similar way.

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Mark Miller (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779253#action_12779253 ] 

Mark Miller commented on LUCENE-2075:
-------------------------------------

bq. Well, that's just hosted on code.google.com (ie it's not "Google's"), 

Ah - got that vibe, but it didn't really hit me.

bq. though they do state that they created a "Production Version"

Right - thats what I was thinking we might try. Though the whole, trying this from scratch to learn is a bit scary too ;) But hey, I'm not recommending, just perhaps trying it.

bq. I think FastLRUCache is probably best for Lucene, because it scales up well w/ high number of threads?

Indeed - though if we expect a low hit ratio, we might still compare it with regular old synchronized LinkedHashMap to be sure. In certain cases, puts become quite expensive I think.

bq. It looks like ConcurrentLRUCache (used by FastLRUCache, but the latter does other solr-specific things) is the right low-level one to use for Lucene?

Right.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781391#action_12781391 ] 

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

bq. To your patch: Looks good, I would only add @Overrides to the DoubleBarrelCache.

Ahh right will do -- not quite in Java 5 mode yet ;)

bq. What do we do with Yonik/mine's cache?

Solr's ConcurrentLRUCache makes me somewhat nervous, that it can blow up under high (admittedly, rather synthetic, by today's standards) load.

bq. Have you tried aut with NRQ, too?

I haven't; that'd be great if you could & report back.

{quote}
bq. Robert maybe you can try this patch plus automaton patch and see if you see this same odd behavior?

confirmed, though on my machine, it is 4 second avg *N versus 6 second avg *N 
{quote}

Weird -- I can't explain why this full scan is faster but the skipping scan is not.

bq. I havent looked at the code, but fyi, even the smart mode is always "in term order" traversal, its just skipping over terms.

Right -- but that skipping variant is now pulling from cache.  Let's see what NRQ results look like... though it does quite a bit less seeking than eg the ????NNN query.

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


[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781410#action_12781410 ] 

Uwe Schindler edited comment on LUCENE-2075 at 11/23/09 2:13 PM:
-----------------------------------------------------------------

I tested with an 5 mio doc index containing trie ints, but it seems that trie does not really profit from the seeking cache. With the default precStep of 4 no difference (max. 16 seeks per query), and with precStep of 1 (max. 64 seeks per query) it was even a little bit slower on average (???). The test compares also with FieldCacheRangeFilter which is always faster (because no deletes, optimized index), also the field cache loading time did not really change (linear scan in term enum).

PrecisionStep: 4
trunk:
loading field cache time: 6367.667678 ms
avg number of terms: 68.1
TRIE:       best time=6.323709 ms; worst time=414.367469 ms; avg=201.18463369999998 ms; sum=32004735
FIELDCACHE: best time=64.770523 ms; worst time=265.487652 ms; avg=155.5479675 ms; sum=32004735

patch:
loading field cache time: 6295.055377 ms
avg number of terms: 68.1
TRIE:       best time=5.288102 ms; worst time=415.290771 ms; avg=195.72079685 ms; sum=32004735
FIELDCACHE: best time=65.511957 ms; worst time=202.482438 ms; avg=138.69083925 ms; sum=32004735

---

PrecisionStep: 1
trunk:
loading field cache time: 6416.105399 ms
avg number of terms: 19.85
TRIE:       best time=6.51228 ms; worst time=410.624255 ms; avg=192.33796475 ms; sum=32002505
FIELDCACHE: best time=65.349088 ms; worst time=211.308979 ms; avg=143.71657580000002 ms; sum=32002505

patch:
loading field cache time: 6809.792026 ms
avg number of terms: 19.85
TRIE:       best time=6.814832 ms; worst time=436.396525 ms; avg=205.6526038 ms; sum=32002505
FIELDCACHE: best time=64.939539 ms; worst time=277.474371 ms; avg=142.58939345 ms; sum=32002505

      was (Author: thetaphi):
    I tested with an 5 mio doc index containing trie ints, but it seems that trie does not really profit from the seeking cache. With the default precStep of 4 no difference (max. 16 seeks per query), and with precStep of 1 (max. 64 seeks per query) it was even a little bit slower on average (???). The test compares also with FieldCacheRangeFilter which is always slower, also the field cache loading time did not really change (linear scan in term enum).

PrecisionStep: 4
trunk:
loading field cache time: 6367.667678 ms
avg number of terms: 68.1
TRIE:       best time=6.323709 ms; worst time=414.367469 ms; avg=201.18463369999998 ms; sum=32004735
FIELDCACHE: best time=64.770523 ms; worst time=265.487652 ms; avg=155.5479675 ms; sum=32004735

patch:
loading field cache time: 6295.055377 ms
avg number of terms: 68.1
TRIE:       best time=5.288102 ms; worst time=415.290771 ms; avg=195.72079685 ms; sum=32004735
FIELDCACHE: best time=65.511957 ms; worst time=202.482438 ms; avg=138.69083925 ms; sum=32004735

---

PrecisionStep: 1
trunk:
loading field cache time: 6416.105399 ms
avg number of terms: 19.85
TRIE:       best time=6.51228 ms; worst time=410.624255 ms; avg=192.33796475 ms; sum=32002505
FIELDCACHE: best time=65.349088 ms; worst time=211.308979 ms; avg=143.71657580000002 ms; sum=32002505

patch:
loading field cache time: 6809.792026 ms
avg number of terms: 19.85
TRIE:       best time=6.814832 ms; worst time=436.396525 ms; avg=205.6526038 ms; sum=32002505
FIELDCACHE: best time=64.939539 ms; worst time=277.474371 ms; avg=142.58939345 ms; sum=32002505
  
> 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
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781384#action_12781384 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Have you tried aut with NRQ, too? If not I will run a comparison with a integer index before/after this patch and measure query times. For lower precSteps it should get faster, as seeking the TermEnum is optimized.

To your patch: Looks good, I would only add @Overrides to the DoubleBarrelCache. What do we do with Yonik/mine's cache?

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779354#action_12779354 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Should this ConcurrentLRUCache not better be fitted into the o.a.l.util.cache package?

About the Solr implementation: The generification has a "small" problem: get(), contains(), remove() and other by-key-querying methods should use Object as type for the key, not the generic V, because it is not bad to test with contains any java type (it would just return false). The sun generic howto explains that. Very funny video about that: [http://www.youtube.com/watch?v=wDN_EYUvUq0] (explaination starts at 4:35)

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779437#action_12779437 ] 

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

I'll work out a simple perf test to compare the options...

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Earwin Burrfoot (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779512#action_12779512 ] 

Earwin Burrfoot commented on LUCENE-2075:
-----------------------------------------

> Well, that's just hosted on code.google.com (ie it's not "Google's"), and reading its description it sounds sort of experimental (though they do state that they created a "Production Version"). It made me a bit nervous... however, it does sound people use it in "production".

I run it in production for several months (starting from 'experimental' version) as a cache for Filters. No visible problems.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Mark Miller (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779071#action_12779071 ] 

Mark Miller commented on LUCENE-2075:
-------------------------------------

We should prob compare with google's (its apache 2 licensed, so why not)

Solr has two synchronized lru caches - LRUCache, which is basically just a synchronized LinkedHashMap, and FastLRUCache which I believe tries to minimize the cost of gets - however, unless you have a high hit ratio, it was tested as slower than LRUCache.

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>
> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Uwe Schindler updated LUCENE-2075:
----------------------------------

    Attachment: LUCENE-2075.patch

I updated the patch to add overrides. I also had to add one SupressWarnings, because the get() method does an unchecked cast (because it modifies the map, which is not in the contract of get(), but it's safe, because it only adds the key to the second map, if the first map already contains it, and therefore the key has correct type).

I will start now my tests with NRQ.

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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781878#action_12781878 ] 

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

OK I opened LUCENE-2093 to track the "query private scope" idea.

> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless updated LUCENE-2075:
---------------------------------------

    Attachment: LUCENE-2075.patch

First cut at a benchmark.  First, download
http://concurrentlinkedhashmap.googlecode.com/files/clhm-production.jar
and put into your lib subdir, then run "ant -lib
lib/clhm-production.jar compile-core", then run it something like
this:

{code}
java -server -Xmx1g -Xms1g -cp build/classes/java:lib/clhm-production.jar org.apache.lucene.util.cache.LRUBench 4 5.0 0.0 1024 1024
{code}

The args are:

  * numThreads

  * runSec

  * sharePct -- what %tg of the terms should be shared b/w the threads

  * cacheSize

  * termCountPerThread -- how many terms each thread will cycle through

The benchmark first sets up arrays of strings, per thread, based
termsCountPerThread & sharePct.  Then each thread steps through the
array, and for each entry, tries to get the string, and if it's not
present, puts it.  It records the hit & miss count, and prints summary
stats in the end, doing 3 rounds.

To mimic Lucene, each entry is tested twice in a row, ie, the 2nd time
we test the entry, it should be a hit.  Ie we expect a hit rate of 50%
if sharePct is 0.

Here's my output from the above command line, using Java 1.6.0_14 (64
bit) on OpenSolaris:

{code}
numThreads=4 runSec=5.0 sharePct=0.0 cacheSize=1024 termCountPerThread=1024

LRU cache size is 1024; each thread steps through 1024 strings; 0 of which are common

round 0
  sync(LinkedHashMap): Mops/sec=2.472 hitRate=50.734
  DoubleBarreLRU: Mops/sec=20.502 hitRate=50
  ConcurrentLRU: Mops/sec=17.936 hitRate=84.409
  ConcurrentLinkedHashMap: Mops/sec=1.248 hitRate=50.033

round 1
  sync(LinkedHashMap): Mops/sec=2.766 hitRate=50.031
  DoubleBarreLRU: Mops/sec=17.66 hitRate=50
  ConcurrentLRU: Mops/sec=17.82 hitRate=83.726
  ConcurrentLinkedHashMap: Mops/sec=1.266 hitRate=50.331

round 2
  sync(LinkedHashMap): Mops/sec=2.714 hitRate=50.168
  DoubleBarreLRU: Mops/sec=17.912 hitRate=50
  ConcurrentLRU: Mops/sec=17.866 hitRate=84.156
  ConcurrentLinkedHashMap: Mops/sec=1.26 hitRate=50.254
{code}

NOTE: I'm not sure about the correctness of DoubleBarrelLRU -- I just
quickly wrote it.

Also, the results for ConcurrentLRUCache are invalid (its hit rate is
way too high) -- I think this is because its eviction process can take
a longish amount of time, which temporarily allows the map to hold way
too many entries, and means it's using up alot more transient RAM than
it should.

In theory DoubleBarrelLRU should be vulnerable to the same issue, but
in practice it seems to affect it much less (I guess because
CHM.clear() must be very fast).

I'm not sure how to fix the benchmark to workaround that... maybe we
bring back the cleaning thread (from Solr's version), and give it a
high priority?

Another idea: I wonder whether a simple cache-line like cache would be
sufficient.  Ie, we hash to a fixed slot and we evict whatever is
there.


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


[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12782121#action_12782121 ] 

Robert Muir edited comment on LUCENE-2075 at 11/24/09 7:45 PM:
---------------------------------------------------------------

{quote}
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).
{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)

      was (Author: rcmuir):
    {quote}
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).
{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
this will return 0000NNN, and 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?

  
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779247#action_12779247 ] 

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

bq. We should prob compare with google's (its apache 2 licensed, so why not)

Well, that's just hosted on code.google.com (ie it's not "Google's"), and reading its description it sounds sort of experimental (though they do state that they created a "Production Version").  It made me a bit nervous... however, it does sound people use it in "production".

I think FastLRUCache is probably best for Lucene, because it scales up well w/ high number of threads?  My guess is it's slower cost for low hit rates is negligible to Lucene, but I'll run some perf tests.

It looks like ConcurrentLRUCache (used by FastLRUCache, but the latter does other solr-specific things) is the right low-level one to use for Lucene?

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>
> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Robert Muir (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12781916#action_12781916 ] 

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

Thanks mike, thats what I was missing
hitting the terms dict twice in the common case explains it to me :)


> 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Uwe Schindler (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779560#action_12779560 ] 

Uwe Schindler commented on LUCENE-2075:
---------------------------------------

Looks good! Can this cache subclass the abstract (Map)Cache; it is in the correct package but does not subclass Cache?

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Yonik Seeley (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12779640#action_12779640 ] 

Yonik Seeley commented on LUCENE-2075:
--------------------------------------

bq. "This lock supports a maximum of 2147483648 recursive locks by the same thread."

I read this as a maximum of recursive locks (which this class won't do at all)... not the total number of times one can successfully lock/unlock the lock.

This cache impl should be able to support 1B operations per second for almost 300 years (i.e. the time it would take to overflow a long).

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>         Attachments: ConcurrentLRUCache.java, 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


[jira] Commented: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Earwin Burrfoot (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12778675#action_12778675 ] 

Earwin Burrfoot commented on LUCENE-2075:
-----------------------------------------

There's no such thing in Google Collections. However, look at this - http://code.google.com/p/concurrentlinkedhashmap/

> 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
>            Priority: Minor
>             Fix For: 3.1
>
>
> 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


[jira] Updated: (LUCENE-2075) Share the Term -> TermInfo cache across threads

Posted by "Michael McCandless (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/LUCENE-2075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael McCandless updated LUCENE-2075:
---------------------------------------

    Attachment: LUCENE-2075.patch

New patch attached -- restores (deprecated) TestSimpleLRUCache.  I think this one is ready to commit?

> 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