Index: CHANGES.txt =================================================================== --- CHANGES.txt (revision 883337) +++ CHANGES.txt (working copy) @@ -20,6 +20,12 @@ * 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, Yonik Seeley) + Build ======================= Release 3.0.0 2009-11-25 ======================= Index: src/test/org/apache/lucene/util/cache/TestSimpleLRUCache.java =================================================================== --- src/test/org/apache/lucene/util/cache/TestSimpleLRUCache.java (revision 883337) +++ src/test/org/apache/lucene/util/cache/TestSimpleLRUCache.java (working copy) @@ -1,63 +0,0 @@ -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 TestSimpleLRUCache extends LuceneTestCase { - - 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))); - } - - } - -} Index: src/test/org/apache/lucene/util/cache/TestDoubleBarrelLRUCache.java =================================================================== --- src/test/org/apache/lucene/util/cache/TestDoubleBarrelLRUCache.java (revision 0) +++ src/test/org/apache/lucene/util/cache/TestDoubleBarrelLRUCache.java (revision 0) @@ -0,0 +1,138 @@ +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 TestDoubleBarrelLRUCache extends LuceneTestCase { + + public void testLRUCache() throws Exception { + final int n = 100; + Object dummy = new Object(); + + Cache cache = new DoubleBarrelLRUCache(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))); + } + } + + private class CacheThread extends Thread { + private final Object[] objs; + private final Cache c; + private final long endTime; + volatile boolean failed; + + public CacheThread(Cache 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 c = new DoubleBarrelLRUCache(1024); + + Object[] objs = new Object[OBJ_COUNT]; + for(int i=0;iAt 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.

+ * + *

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.

+ * + *

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 extends Cache { + private final Map cache1; + private final Map 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(); + cache2 = new ConcurrentHashMap(); + } + + @Override + public boolean containsKey(Object k) { + return false; + } + + @Override + public void close() { + } + + @Override @SuppressWarnings("unchecked") + public V get(Object key) { + final Map primary; + final Map 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 primary; + final Map 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; + + // Reset countdown + countdown.set(maxSize); + } + } +} Property changes on: src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCache.java ___________________________________________________________________ Added: svn:eol-style + native