[ 
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-unsubscr...@lucene.apache.org
For additional commands, e-mail: java-dev-h...@lucene.apache.org

Reply via email to