lucene-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Uwe Schindler" <...@thetaphi.de>
Subject RE: svn commit: r884895 - in /lucene/java/trunk: ./ src/java/org/apache/lucene/index/ src/java/org/apache/lucene/util/cache/ src/test/org/apache/lucene/util/cache/
Date Fri, 27 Nov 2009 21:16:58 GMT
You can pass the printed random seed to the newRandom call in the failing
testcase.

I would do it now, but I wanted to go to bed, so I can look into it
tomorrow.

-----
Uwe Schindler
H.-H.-Meier-Allee 63, D-28213 Bremen
http://www.thetaphi.de
eMail: uwe@thetaphi.de

> -----Original Message-----
> From: Michael McCandless [mailto:lucene@mikemccandless.com]
> Sent: Friday, November 27, 2009 10:05 PM
> To: java-dev@lucene.apache.org
> Subject: Re: svn commit: r884895 - in /lucene/java/trunk: ./
> src/java/org/apache/lucene/index/ src/java/org/apache/lucene/util/cache/
> src/test/org/apache/lucene/util/cache/
> 
> That is a spooky failure -- it's an assertion I added.  Hmm I can also
> get it to happen if I run the tests enough.  I'll try fixing the seed!
> 
> Mike
> 
> On Fri, Nov 27, 2009 at 3:57 PM, Robert Muir <rcmuir@gmail.com> wrote:
> > I got a suspicious test error today, I think related to the change.
> >
> > I cant reproduce it by running ant test again so I think the same random
> > seed might need to be used?
> >
> >     [junit] Testsuite: org.apache.lucene.search.TestNumericRangeQuery32
> >     [junit] Tests run: 26, Failures: 0, Errors: 1, Time elapsed: 4.527
> sec
> >     [junit]
> >     [junit] ------------- Standard Output ---------------
> >     [junit] Found 60 distinct terms in range for field 'field8'
> (constant
> > score filter rewrite).
> >     [junit] Found 60 distinct terms in range for field 'field8'
> (constant
> > score boolean rewrite).
> >     [junit] Found 60 distinct terms in range for field 'field8'
> (filter).
> >     [junit] Found 15 distinct terms in range for field 'field4'
> (constant
> > score filter rewrite).
> >     [junit] Found 15 distinct terms in range for field 'field4'
> (constant
> > score boolean rewrite).
> >     [junit] Found 15 distinct terms in range for field 'field4'
> (filter).
> >     [junit] Found 9 distinct terms in range for field 'field2' (constant
> > score filter rewrite).
> >     [junit] Found 9 distinct terms in range for field 'field2' (constant
> > score boolean rewrite).
> >     [junit] Found 9 distinct terms in range for field 'field2' (filter).
> >     [junit] Found 56 distinct terms in left open range for field
> 'field8'.
> >     [junit] Found 11 distinct terms in left open range for field
> 'field4'.
> >     [junit] Found 8 distinct terms in left open range for field
> 'field2'.
> >     [junit] Found 213 distinct terms in right open range for field
> 'field8'.
> >     [junit] Found 33 distinct terms in right open range for field
> 'field4'.
> >     [junit] Found 14 distinct terms in right open range for field
> 'field2'.
> >     [junit] Average number of terms during random search on 'field8':
> >     [junit]  Trie query: 180.76
> >     [junit]  Classical query: 2641.88
> >     [junit] Average number of terms during random search on 'field4':
> >     [junit]  Trie query: 38.4
> >     [junit]  Classical query: 3368.94
> >     [junit] Average number of terms during random search on 'field2':
> >     [junit]  Trie query: 17.98
> >     [junit]  Classical query: 3627.38
> >     [junit] NOTE: random seed of testcase 'testRangeSplit_4bit' was:
> > -3500708189935903943
> >     [junit] TermEnum on 'field4' for range [-22769,19977453] contained
> 15
> > terms.
> >     [junit] TermEnum on 'field4' for range [19977453,-22769] contained 0
> > terms.
> >     [junit] TermEnum on 'field4' for range [66627232,133254464]
> contained 0
> > terms.
> >     [junit] ------------- ---------------- ---------------
> >     [junit] Testcase:
> > testRangeSplit_4bit(org.apache.lucene.search.TestNumericRangeQuery32):
> > Caused an ERROR
> >     [junit] null
> >     [junit] java.lang.AssertionError
> >     [junit]     at
> > org.apache.lucene.index.TermInfosReader.get(TermInfosReader.java:255)
> >     [junit]     at
> > org.apache.lucene.index.TermInfosReader.terms(TermInfosReader.java:317)
> >     [junit]     at
> > org.apache.lucene.index.SegmentReader.terms(SegmentReader.java:869)
> >     [junit]     at
> >
> org.apache.lucene.index.DirectoryReader$MultiTermEnum.<init>(DirectoryRead
> er.java:996)
> >     [junit]     at
> > org.apache.lucene.index.DirectoryReader.terms(DirectoryReader.java:636)
> >     [junit]     at
> >
> org.apache.lucene.search.NumericRangeQuery$NumericRangeTermEnum.next(Numer
> icRangeQuery.java:535)
> >     [junit]     at
> >
> org.apache.lucene.search.NumericRangeQuery$NumericRangeTermEnum.<init>(Num
> ericRangeQuery.java:476)
> >     [junit]     at
> >
> org.apache.lucene.search.NumericRangeQuery.getEnum(NumericRangeQuery.java:
> 306)
> >     [junit]     at
> >
> org.apache.lucene.search.MultiTermQuery$ConstantScoreAutoRewrite.rewrite(M
> ultiTermQuery.java:230)
> >     [junit]     at
> > org.apache.lucene.search.MultiTermQuery.rewrite(MultiTermQuery.java:371)
> >     [junit]     at
> > org.apache.lucene.search.IndexSearcher.rewrite(IndexSearcher.java:267)
> >     [junit]     at org.apache.lucene.search.Query.weight(Query.java:100)
> >     [junit]     at
> > org.apache.lucene.search.Searcher.createWeight(Searcher.java:147)
> >     [junit]     at
> > org.apache.lucene.search.Searcher.search(Searcher.java:98)
> >     [junit]     at
> > org.apache.lucene.search.Searcher.search(Searcher.java:108)
> >     [junit]     at
> >
> org.apache.lucene.search.TestNumericRangeQuery32.testRangeSplit(TestNumeri
> cRangeQuery32.java:312)
> >     [junit]     at
> >
> org.apache.lucene.search.TestNumericRangeQuery32.testRangeSplit_4bit(TestN
> umericRangeQuery32.java:330)
> >     [junit]     at
> > org.apache.lucene.util.LuceneTestCase.runBare(LuceneTestCase.java:208)
> >     [junit]
> >     [junit]
> >     [junit] Test org.apache.lucene.search.TestNumericRangeQuery32 FAILED
> >
> >
> > On Fri, Nov 27, 2009 at 10:32 AM, <mikemccand@apache.org> wrote:
> >>
> >> Author: mikemccand
> >> Date: Fri Nov 27 15:32:57 2009
> >> New Revision: 884895
> >>
> >> URL: http://svn.apache.org/viewvc?rev=884895&view=rev
> >> Log:
> >> LUCENE-2075: share the terms dict cache across threads
> >>
> >> Added:
> >>
> >>
>  lucene/java/trunk/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCa
> che.java
> >>   (with props)
> >>
> >>
>  lucene/java/trunk/src/test/org/apache/lucene/util/cache/BaseTestLRU.java
> >> (with props)
> >>
> >>
>  lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestDoubleBarrelL
> RUCache.java
> >>   (with props)
> >> Modified:
> >>    lucene/java/trunk/CHANGES.txt
> >>    lucene/java/trunk/src/java/org/apache/lucene/index/TermInfo.java
> >>
>  lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
> >>
> >>
>  lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.ja
> va
> >>
> >>
>  lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.ja
> va
> >>
> >>
>  lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestSimpleLRUCach
> e.java
> >>
> >> Modified: lucene/java/trunk/CHANGES.txt
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/CHANGES.txt?rev=884895&r1=8
> 84894&r2=884895&view=diff
> >>
> >>
> ==========================================================================
> ====
> >> --- lucene/java/trunk/CHANGES.txt (original)
> >> +++ lucene/java/trunk/CHANGES.txt Fri Nov 27 15:32:57 2009
> >> @@ -30,6 +30,13 @@
> >>  * LUCENE-2086: When resolving deleted terms, do so in term sort order
> >>   for better performance. (Bogdan Ghidireac via Mike McCandless)
> >>
> >> +* LUCENE-2075: Terms dict cache is now shared across threads instead
> >> +  of being stored separately in thread local storage.  Also fixed
> >> +  terms dict so that the cache is used when seeking the thread local
> >> +  term enum, which will be important for MultiTermQuery impls that do
> >> +  lots of seeking (Mike McCandless, Uwe Schindler, Robert Muir, Yonik
> >> +  Seeley)
> >> +
> >>  Build
> >>
> >>  ======================= Release 3.0.0 2009-11-25
> =======================
> >>
> >> Modified:
> lucene/java/trunk/src/java/org/apache/lucene/index/TermInfo.java
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/
> index/TermInfo.java?rev=884895&r1=884894&r2=884895&view=diff
> >>
> >>
> ==========================================================================
> ====
> >> --- lucene/java/trunk/src/java/org/apache/lucene/index/TermInfo.java
> >> (original)
> >> +++ lucene/java/trunk/src/java/org/apache/lucene/index/TermInfo.java
> Fri
> >> Nov 27 15:32:57 2009
> >> @@ -19,7 +19,7 @@
> >>
> >>  /** A TermInfo is the record of information stored for a term.*/
> >>
> >> -final class TermInfo {
> >> +class TermInfo {
> >>   /** The number of documents which contain the term. */
> >>   int docFreq = 0;
> >>
> >> @@ -42,6 +42,28 @@
> >>     skipOffset = ti.skipOffset;
> >>   }
> >>
> >> +  public boolean equals(Object obj) {
> >> +    if (obj instanceof TermInfo) {
> >> +      TermInfo other = (TermInfo) obj;
> >> +      return other.docFreq == docFreq &&
> >> +        other.freqPointer == freqPointer &&
> >> +        other.proxPointer == proxPointer &&
> >> +        other.skipOffset == skipOffset;
> >> +    } else {
> >> +      return false;
> >> +    }
> >> +  }
> >> +
> >> +  public int hashCode() {
> >> +    final int PRIME = 17;
> >> +    int result = 1;
> >> +    result = PRIME * result + docFreq;
> >> +    result = (int) (PRIME * result + freqPointer);
> >> +    result = (int) (PRIME * result + proxPointer);
> >> +    result = (int) (PRIME * result + skipOffset);
> >> +    return result;
> >> +  }
> >> +
> >>   final void set(int docFreq,
> >>                  long freqPointer, long proxPointer, int skipOffset)
{
> >>     this.docFreq = docFreq;
> >>
> >> Modified:
> >> lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/
> index/TermInfosReader.java?rev=884895&r1=884894&r2=884895&view=diff
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >> lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
> >> (original)
> >> +++
> >> lucene/java/trunk/src/java/org/apache/lucene/index/TermInfosReader.java
> Fri
> >> Nov 27 15:32:57 2009
> >> @@ -21,7 +21,7 @@
> >>
> >>  import org.apache.lucene.store.Directory;
> >>  import org.apache.lucene.util.cache.Cache;
> >> -import org.apache.lucene.util.cache.SimpleLRUCache;
> >> +import org.apache.lucene.util.cache.DoubleBarrelLRUCache;
> >>  import org.apache.lucene.util.CloseableThreadLocal;
> >>
> >>  /** This stores a monotonically increasing set of <Term, TermInfo>
> pairs
> >> in a
> >> @@ -44,15 +44,23 @@
> >>   private final int totalIndexInterval;
> >>
> >>   private final static int DEFAULT_CACHE_SIZE = 1024;
> >> +
> >> +  // Just adds term's ord to TermInfo
> >> +  private final static class TermInfoAndOrd extends TermInfo {
> >> +    final int termOrd;
> >> +    public TermInfoAndOrd(TermInfo ti, int termOrd) {
> >> +      super(ti);
> >> +      this.termOrd = termOrd;
> >> +    }
> >> +  }
> >> +
> >> +  private final Cache<Term,TermInfoAndOrd> termsCache = new
> >> DoubleBarrelLRUCache<Term,TermInfoAndOrd>(DEFAULT_CACHE_SIZE);
> >>
> >>   /**
> >>    * Per-thread resources managed by ThreadLocal
> >>    */
> >>   private static final class ThreadResources {
> >>     SegmentTermEnum termEnum;
> >> -
> >> -    // Used for caching the least recently looked-up Terms
> >> -    Cache<Term,TermInfo> termInfoCache;
> >>   }
> >>
> >>   TermInfosReader(Directory dir, String seg, FieldInfos fis, int
> >> readBufferSize, int indexDivisor)
> >> @@ -130,6 +138,7 @@
> >>     if (origEnum != null)
> >>       origEnum.close();
> >>     threadResources.close();
> >> +    termsCache.close();
> >>   }
> >>
> >>   /** Returns the number of term/value pairs in the set. */
> >> @@ -142,8 +151,6 @@
> >>     if (resources == null) {
> >>       resources = new ThreadResources();
> >>       resources.termEnum = terms();
> >> -      // Cache does not have to be thread-safe, it is only used by one
> >> thread at the same time
> >> -      resources.termInfoCache = new
> >> SimpleLRUCache<Term,TermInfo>(DEFAULT_CACHE_SIZE);
> >>       threadResources.set(resources);
> >>     }
> >>     return resources;
> >> @@ -176,26 +183,20 @@
> >>
> >>   /** Returns the TermInfo for a Term in the set, or null. */
> >>   TermInfo get(Term term) throws IOException {
> >> -    return get(term, true);
> >> +    return get(term, false);
> >>   }
> >>
> >>   /** Returns the TermInfo for a Term in the set, or null. */
> >> -  private TermInfo get(Term term, boolean useCache) throws IOException
> {
> >> +  private TermInfo get(Term term, boolean mustSeekEnum) throws
> >> IOException {
> >>     if (size == 0) return null;
> >>
> >>     ensureIndexIsRead();
> >>
> >> -    TermInfo ti;
> >> +    TermInfoAndOrd tiOrd = termsCache.get(term);
> >>     ThreadResources resources = getThreadResources();
> >> -    Cache<Term,TermInfo> cache = null;
> >>
> >> -    if (useCache) {
> >> -      cache = resources.termInfoCache;
> >> -      // check the cache first if the term was recently looked up
> >> -      ti = cache.get(term);
> >> -      if (ti != null) {
> >> -        return ti;
> >> -      }
> >> +    if (!mustSeekEnum && tiOrd != null) {
> >> +      return tiOrd;
> >>     }
> >>
> >>     // optimize sequential access: first try scanning cached enum w/o
> >> seeking
> >> @@ -208,16 +209,23 @@
> >>     || term.compareTo(indexTerms[enumOffset]) < 0) {
> >>        // no need to seek
> >>
> >> +        final TermInfo ti;
> >> +
> >>         int numScans = enumerator.scanTo(term);
> >>         if (enumerator.term() != null &&
> term.compareTo(enumerator.term())
> >> == 0) {
> >>           ti = enumerator.termInfo();
> >> -          if (cache != null && numScans > 1) {
> >> +          if (numScans > 1) {
> >>             // we only  want to put this TermInfo into the cache if
> >>             // scanEnum skipped more than one dictionary entry.
> >>             // This prevents RangeQueries or WildcardQueries to
> >>             // wipe out the cache when they iterate over a large
> numbers
> >>             // of terms in order
> >> -            cache.put(term, ti);
> >> +            if (tiOrd == null) {
> >> +              termsCache.put(term, new TermInfoAndOrd(ti, (int)
> >> enumerator.position));
> >> +            } else {
> >> +              assert ti.equals(tiOrd);
> >> +              assert (int) enumerator.position == tiOrd.termOrd;
> >> +            }
> >>           }
> >>         } else {
> >>           ti = null;
> >> @@ -228,12 +236,24 @@
> >>     }
> >>
> >>     // random-access: must seek
> >> -    seekEnum(enumerator, getIndexOffset(term));
> >> +    final int indexPos;
> >> +    if (tiOrd != null) {
> >> +      indexPos = tiOrd.termOrd / totalIndexInterval;
> >> +    } else {
> >> +      // Must do binary search:
> >> +      indexPos = getIndexOffset(term);
> >> +    }
> >> +
> >> +    seekEnum(enumerator, indexPos);
> >>     enumerator.scanTo(term);
> >> +    final TermInfo ti;
> >>     if (enumerator.term() != null && term.compareTo(enumerator.term())
> ==
> >> 0) {
> >>       ti = enumerator.termInfo();
> >> -      if (cache != null) {
> >> -        cache.put(term, ti);
> >> +      if (tiOrd == null) {
> >> +        termsCache.put(term, new TermInfoAndOrd(ti, (int)
> >> enumerator.position));
> >> +      } else {
> >> +        assert ti.equals(tiOrd);
> >> +        assert (int) enumerator.position == tiOrd.termOrd;
> >>       }
> >>     } else {
> >>       ti = null;
> >> @@ -294,9 +314,7 @@
> >>
> >>   /** 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);
> >> +    get(term, true);
> >>     return (SegmentTermEnum)getThreadResources().termEnum.clone();
> >>   }
> >>  }
> >>
> >> Added:
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCac
> he.java
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/
> util/cache/DoubleBarrelLRUCache.java?rev=884895&view=auto
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCac
> he.java
> >> (added)
> >> +++
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCac
> he.java
> >> Fri Nov 27 15:32:57 2009
> >> @@ -0,0 +1,128 @@
> >> +package org.apache.lucene.util.cache;
> >> +
> >> +/**
> >> + * Licensed to the Apache Software Foundation (ASF) under one or more
> >> + * contributor license agreements.  See the NOTICE file distributed
> with
> >> + * this work for additional information regarding copyright ownership.
> >> + * The ASF licenses this file to You under the Apache License, Version
> >> 2.0
> >> + * (the "License"); you may not use this file except in compliance
> with
> >> + * the License.  You may obtain a copy of the License at
> >> + *
> >> + *     http://www.apache.org/licenses/LICENSE-2.0
> >> + *
> >> + * Unless required by applicable law or agreed to in writing, software
> >> + * distributed under the License is distributed on an "AS IS" BASIS,
> >> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> + * See the License for the specific language governing permissions and
> >> + * limitations under the License.
> >> + */
> >> +
> >> +import java.util.concurrent.ConcurrentHashMap;
> >> +import java.util.concurrent.atomic.AtomicInteger;
> >> +import java.util.Map;
> >> +
> >> +/**
> >> + * Simple concurrent LRU cache, using a "double barrel"
> >> + * approach where two ConcurrentHashMaps record entries.
> >> + *
> >> + * <p>At any given time, one hash is primary and the other
> >> + * is secondary.  {@link #get} first checks primary, and if
> >> + * that's a miss, checks secondary.  If secondary has the
> >> + * entry, it's promoted to primary.  Once primary is full,
> >> + * the secondary is cleared and the two are swapped.</p>
> >> + *
> >> + * <p>This is not as space efficient as other possible
> >> + * concurrent approaches (see LUCENE-2075): to achieve
> >> + * perfect LRU(N) it requires 2*N storage.  But, this
> >> + * approach is relatively simple and seems in practice to
> >> + * not grow unbounded in size when under hideously high
> >> + * load.</p>
> >> + *
> >> + * <p>NOTE: this class is meant only to be used internally
> >> + * by Lucene; it's only public so it can be shared across
> >> + * packages.  This means the API is freely subject to
> >> + * change, and, the class could be removed entirely, in any
> >> + * Lucene release.  Use directly at your own risk!
> >> + */
> >> +
> >> +final public class DoubleBarrelLRUCache<K,V> extends Cache<K,V>
{
> >> +  private final Map<K,V> cache1;
> >> +  private final Map<K,V> cache2;
> >> +  private final AtomicInteger countdown;
> >> +  private volatile boolean swapped;
> >> +  private final int maxSize;
> >> +
> >> +  public DoubleBarrelLRUCache(int maxSize) {
> >> +    this.maxSize = maxSize;
> >> +    countdown = new AtomicInteger(maxSize);
> >> +    cache1 = new ConcurrentHashMap<K,V>();
> >> +    cache2 = new ConcurrentHashMap<K,V>();
> >> +  }
> >> +
> >> +  @Override
> >> +  public boolean containsKey(Object k) {
> >> +    return false;
> >> +  }
> >> +
> >> +  @Override
> >> +  public void close() {
> >> +  }
> >> +
> >> +  @Override @SuppressWarnings("unchecked")
> >> +  public V get(Object key) {
> >> +    final Map<K,V> primary;
> >> +    final Map<K,V> secondary;
> >> +    if (swapped) {
> >> +      primary = cache2;
> >> +      secondary = cache1;
> >> +    } else {
> >> +      primary = cache1;
> >> +      secondary = cache2;
> >> +    }
> >> +
> >> +    // Try primary frist
> >> +    V result = primary.get(key);
> >> +    if (result == null) {
> >> +      // Not found -- try secondary
> >> +      result = secondary.get(key);
> >> +      if (result != null) {
> >> +        // Promote to primary
> >> +        put((K) key, result);
> >> +      }
> >> +    }
> >> +    return result;
> >> +  }
> >> +
> >> +  @Override
> >> +  public void put(K key, V value) {
> >> +    final Map<K,V> primary;
> >> +    final Map<K,V> secondary;
> >> +    if (swapped) {
> >> +      primary = cache2;
> >> +      secondary = cache1;
> >> +    } else {
> >> +      primary = cache1;
> >> +      secondary = cache2;
> >> +    }
> >> +    primary.put(key, value);
> >> +
> >> +    if (countdown.decrementAndGet() == 0) {
> >> +      // Time to swap
> >> +
> >> +      // NOTE: there is saturation risk here, that the
> >> +      // thread that's doing the clear() takes too long to
> >> +      // do so, while other threads continue to add to
> >> +      // primary, but in practice this seems not to be an
> >> +      // issue (see LUCENE-2075 for benchmark & details)
> >> +
> >> +      // First, clear secondary
> >> +      secondary.clear();
> >> +
> >> +      // Second, swap
> >> +      swapped = !swapped;
> >> +
> >> +      // Third, reset countdown
> >> +      countdown.set(maxSize);
> >> +    }
> >> +  }
> >> +}
> >>
> >> Propchange:
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCac
> he.java
> >>
> >> -----------------------------------------------------------------------
> -------
> >>    svn:eol-style = native
> >>
> >> Modified:
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.jav
> a
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/
> util/cache/SimpleLRUCache.java?rev=884895&r1=884894&r2=884895&view=diff
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.jav
> a
> >> (original)
> >> +++
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleLRUCache.jav
> a
> >> Fri Nov 27 15:32:57 2009
> >> @@ -24,7 +24,9 @@
> >>  * Simple LRU cache implementation that uses a LinkedHashMap.
> >>  * This cache is not synchronized, use {@link
> >> Cache#synchronizedCache(Cache)}
> >>  * if needed.
> >> - *
> >> + *
> >> + * @deprecated Lucene's internal use of this class has now
> >> + * switched to {@link DoubleBarrelLRUCache}.
> >>  */
> >>  public class SimpleLRUCache<K,V> extends SimpleMapCache<K,V> {
> >>   private final static float LOADFACTOR = 0.75f;
> >>
> >> Modified:
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.jav
> a
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/java/org/apache/lucene/
> util/cache/SimpleMapCache.java?rev=884895&r1=884894&r2=884895&view=diff
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.jav
> a
> >> (original)
> >> +++
> >>
> lucene/java/trunk/src/java/org/apache/lucene/util/cache/SimpleMapCache.jav
> a
> >> Fri Nov 27 15:32:57 2009
> >> @@ -25,6 +25,9 @@
> >>  * Simple cache implementation that uses a HashMap to store (key,
> value)
> >> pairs.
> >>  * This cache is not synchronized, use {@link
> >> Cache#synchronizedCache(Cache)}
> >>  * if needed.
> >> + *
> >> + * @deprecated Lucene's internal use of this class has now
> >> + * switched to {@link DoubleBarrelLRUCache}.
> >>  */
> >>  public class SimpleMapCache<K,V> extends Cache<K,V> {
> >>   protected Map<K,V> map;
> >>
> >> Added:
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/BaseTestLRU.java
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/
> util/cache/BaseTestLRU.java?rev=884895&view=auto
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/BaseTestLRU.java
> >> (added)
> >> +++
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/BaseTestLRU.java
> Fri
> >> Nov 27 15:32:57 2009
> >> @@ -0,0 +1,60 @@
> >> +package org.apache.lucene.util.cache;
> >> +
> >> +/**
> >> +* Licensed to the Apache Software Foundation (ASF) under one or more
> >> +* contributor license agreements.  See the NOTICE file distributed
> with
> >> +* this work for additional information regarding copyright ownership.
> >> +* The ASF licenses this file to You under the Apache License, Version
> 2.0
> >> +* (the "License"); you may not use this file except in compliance with
> >> +* the License.  You may obtain a copy of the License at
> >> +*
> >> +*     http://www.apache.org/licenses/LICENSE-2.0
> >> +*
> >> +* Unless required by applicable law or agreed to in writing, software
> >> +* distributed under the License is distributed on an "AS IS" BASIS,
> >> +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> +* See the License for the specific language governing permissions and
> >> +* limitations under the License.
> >> +*/
> >> +
> >> +import org.apache.lucene.util.LuceneTestCase;
> >> +
> >> +public class BaseTestLRU extends LuceneTestCase {
> >> +
> >> +  protected void testCache(Cache cache, int n) throws Exception {
> >> +    Object dummy = new Object();
> >> +
> >> +    for (int i = 0; i < n; i++) {
> >> +      cache.put(Integer.valueOf(i), dummy);
> >> +    }
> >> +
> >> +    // access every 2nd item in cache
> >> +    for (int i = 0; i < n; i+=2) {
> >> +      assertNotNull(cache.get(Integer.valueOf(i)));
> >> +    }
> >> +
> >> +    // add n/2 elements to cache, the ones that weren't
> >> +    // touched in the previous loop should now be thrown away
> >> +    for (int i = n; i < n + (n / 2); i++) {
> >> +      cache.put(Integer.valueOf(i), dummy);
> >> +    }
> >> +
> >> +    // access every 4th item in cache
> >> +    for (int i = 0; i < n; i+=4) {
> >> +      assertNotNull(cache.get(Integer.valueOf(i)));
> >> +    }
> >> +
> >> +    // add 3/4n elements to cache, the ones that weren't
> >> +    // touched in the previous loops should now be thrown away
> >> +    for (int i = n; i < n + (n * 3 / 4); i++) {
> >> +      cache.put(Integer.valueOf(i), dummy);
> >> +    }
> >> +
> >> +    // access every 4th item in cache
> >> +    for (int i = 0; i < n; i+=4) {
> >> +      assertNotNull(cache.get(Integer.valueOf(i)));
> >> +    }
> >> +
> >> +  }
> >> +
> >> +}
> >>
> >> Propchange:
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/BaseTestLRU.java
> >>
> >> -----------------------------------------------------------------------
> -------
> >>    svn:eol-style = native
> >>
> >> Added:
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLR
> UCache.java
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/
> util/cache/TestDoubleBarrelLRUCache.java?rev=884895&view=auto
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLR
> UCache.java
> >> (added)
> >> +++
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLR
> UCache.java
> >> Fri Nov 27 15:32:57 2009
> >> @@ -0,0 +1,103 @@
> >> +package org.apache.lucene.util.cache;
> >> +
> >> +/**
> >> +* Licensed to the Apache Software Foundation (ASF) under one or more
> >> +* contributor license agreements.  See the NOTICE file distributed
> with
> >> +* this work for additional information regarding copyright ownership.
> >> +* The ASF licenses this file to You under the Apache License, Version
> 2.0
> >> +* (the "License"); you may not use this file except in compliance with
> >> +* the License.  You may obtain a copy of the License at
> >> +*
> >> +*     http://www.apache.org/licenses/LICENSE-2.0
> >> +*
> >> +* Unless required by applicable law or agreed to in writing, software
> >> +* distributed under the License is distributed on an "AS IS" BASIS,
> >> +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> >> implied.
> >> +* See the License for the specific language governing permissions and
> >> +* limitations under the License.
> >> +*/
> >> +
> >> +public class TestDoubleBarrelLRUCache extends BaseTestLRU {
> >> +
> >> +  public void testLRUCache() throws Exception {
> >> +    final int n = 100;
> >> +    testCache(new DoubleBarrelLRUCache(n), n);
> >> +  }
> >> +
> >> +  private class CacheThread extends Thread {
> >> +    private final Object[] objs;
> >> +    private final Cache<Object,Object> c;
> >> +    private final long endTime;
> >> +    volatile boolean failed;
> >> +
> >> +    public CacheThread(Cache<Object,Object> c,
> >> +                     Object[] objs, long endTime) {
> >> +      this.c = c;
> >> +      this.objs = objs;
> >> +      this.endTime = endTime;
> >> +    }
> >> +
> >> +    public void run() {
> >> +      try {
> >> +        long count = 0;
> >> +        long miss = 0;
> >> +        long hit = 0;
> >> +        final int limit = objs.length;
> >> +
> >> +        while(true) {
> >> +          final Object obj = objs[(int) ((count/2) % limit)];
> >> +          Object v = c.get(obj);
> >> +          if (v == null) {
> >> +            c.put(obj, obj);
> >> +            miss++;
> >> +          } else {
> >> +            assert obj == v;
> >> +            hit++;
> >> +          }
> >> +          if ((++count % 10000) == 0) {
> >> +            if (System.currentTimeMillis() >= endTime)  {
> >> +              break;
> >> +            }
> >> +          }
> >> +        }
> >> +
> >> +        addResults(miss, hit);
> >> +      } catch (Throwable t) {
> >> +        failed = true;
> >> +        throw new RuntimeException(t);
> >> +      }
> >> +    }
> >> +  }
> >> +
> >> +  long totMiss, totHit;
> >> +  void addResults(long miss, long hit) {
> >> +    totMiss += miss;
> >> +    totHit += hit;
> >> +  }
> >> +
> >> +  public void testThreadCorrectness() throws Exception {
> >> +    final int NUM_THREADS = 4;
> >> +    final int CACHE_SIZE = 512;
> >> +    final int OBJ_COUNT = 3*CACHE_SIZE;
> >> +
> >> +    Cache<Object,Object> c = new
> >> DoubleBarrelLRUCache<Object,Object>(1024);
> >> +
> >> +    Object[] objs = new Object[OBJ_COUNT];
> >> +    for(int i=0;i<OBJ_COUNT;i++) {
> >> +      objs[i] = new Object();
> >> +    }
> >> +
> >> +    final CacheThread[] threads = new CacheThread[NUM_THREADS];
> >> +    final long endTime = System.currentTimeMillis()+((long) 1000);
> >> +    for(int i=0;i<NUM_THREADS;i++) {
> >> +      threads[i] = new CacheThread(c, objs, endTime);
> >> +      threads[i].start();
> >> +    }
> >> +    for(int i=0;i<NUM_THREADS;i++) {
> >> +      threads[i].join();
> >> +      assert !threads[i].failed;
> >> +    }
> >> +    //System.out.println("hits=" + totHit + " misses=" + totMiss);
> >> +  }
> >> +
> >> +}
> >>
> >> Propchange:
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLR
> UCache.java
> >>
> >> -----------------------------------------------------------------------
> -------
> >>    svn:eol-style = native
> >>
> >> Modified:
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestSimpleLRUCache
> .java
> >> URL:
> >>
> http://svn.apache.org/viewvc/lucene/java/trunk/src/test/org/apache/lucene/
> util/cache/TestSimpleLRUCache.java?rev=884895&r1=884894&r2=884895&view=dif
> f
> >>
> >>
> ==========================================================================
> ====
> >> ---
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestSimpleLRUCache
> .java
> >> (original)
> >> +++
> >>
> lucene/java/trunk/src/test/org/apache/lucene/util/cache/TestSimpleLRUCache
> .java
> >> Fri Nov 27 15:32:57 2009
> >> @@ -17,47 +17,10 @@
> >>  * limitations under the License.
> >>  */
> >>
> >> -import org.apache.lucene.util.LuceneTestCase;
> >> -
> >> -public class TestSimpleLRUCache extends LuceneTestCase {
> >> -
> >> +/** @deprecated */
> >> +public class TestSimpleLRUCache extends BaseTestLRU {
> >>   public void testLRUCache() throws Exception {
> >>     final int n = 100;
> >> -    Object dummy = new Object();
> >> -
> >> -    Cache cache = new SimpleLRUCache(n);
> >> -
> >> -    for (int i = 0; i < n; i++) {
> >> -      cache.put(Integer.valueOf(i), dummy);
> >> -    }
> >> -
> >> -    // access every 2nd item in cache
> >> -    for (int i = 0; i < n; i+=2) {
> >> -      assertNotNull(cache.get(Integer.valueOf(i)));
> >> -    }
> >> -
> >> -    // add n/2 elements to cache, the ones that weren't
> >> -    // touched in the previous loop should now be thrown away
> >> -    for (int i = n; i < n + (n / 2); i++) {
> >> -      cache.put(Integer.valueOf(i), dummy);
> >> -    }
> >> -
> >> -    // access every 4th item in cache
> >> -    for (int i = 0; i < n; i+=4) {
> >> -      assertNotNull(cache.get(Integer.valueOf(i)));
> >> -    }
> >> -
> >> -    // add 3/4n elements to cache, the ones that weren't
> >> -    // touched in the previous loops should now be thrown away
> >> -    for (int i = n; i < n + (n * 3 / 4); i++) {
> >> -      cache.put(Integer.valueOf(i), dummy);
> >> -    }
> >> -
> >> -    // access every 4th item in cache
> >> -    for (int i = 0; i < n; i+=4) {
> >> -      assertNotNull(cache.get(Integer.valueOf(i)));
> >> -    }
> >> -
> >> +    testCache(new SimpleLRUCache(n), n);
> >>   }
> >> -
> >>  }
> >>
> >>
> >
> >
> >
> > --
> > Robert Muir
> > rcmuir@gmail.com
> >
> 
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-dev-help@lucene.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Mime
View raw message