Login | Register For Free | Help
Search for: (Advanced)

Mailing List Archive: Lucene: Java-Dev

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

 

 

Lucene java-dev RSS feed   Index | Next | Previous | View Threaded


jira at apache

Nov 18, 2009, 1:40 AM

Post #1 of 5 (263 views)
Permalink
[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads

[ 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 [at] lucene
For additional commands, e-mail: java-dev-help [at] lucene


jira at apache

Nov 18, 2009, 12:40 PM

Post #2 of 5 (230 views)
Permalink
[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads [In reply to]

[ 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 [at] lucene
For additional commands, e-mail: java-dev-help [at] lucene


jira at apache

Nov 23, 2009, 6:13 AM

Post #3 of 5 (200 views)
Permalink
[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads [In reply to]

[ 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 [at] lucene
For additional commands, e-mail: java-dev-help [at] lucene


jira at apache

Nov 24, 2009, 11:47 AM

Post #4 of 5 (181 views)
Permalink
[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads [In reply to]

[ 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 [at] lucene
For additional commands, e-mail: java-dev-help [at] lucene


jira at apache

Nov 24, 2009, 12:23 PM

Post #5 of 5 (180 views)
Permalink
[jira] Issue Comment Edited: (LUCENE-2075) Share the Term -> TermInfo cache across threads [In reply to]

[ 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 [at] lucene
For additional commands, e-mail: java-dev-help [at] lucene

Lucene java-dev RSS feed   Index | Next | Previous | View Threaded
 
 


Interested in having your list archived? Contact Gossamer Threads
 
  Web Applications & Managed Hosting Powered by Gossamer Threads Inc.