Index: lucene/src/test/org/apache/lucene/util/cache/TestSimpleLRUCache.java =================================================================== --- lucene/src/test/org/apache/lucene/util/cache/TestSimpleLRUCache.java (revision 942059) +++ lucene/src/test/org/apache/lucene/util/cache/TestSimpleLRUCache.java (working copy) @@ -1,27 +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. -*/ - -/** @deprecated */ -@Deprecated -public class TestSimpleLRUCache extends BaseTestLRU { - public void testLRUCache() throws Exception { - final int n = 100; - testCache(new SimpleLRUCache(n), n); - } -} Index: lucene/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLRUCache.java =================================================================== --- lucene/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLRUCache.java (revision 942059) +++ lucene/src/test/org/apache/lucene/util/cache/TestDoubleBarrelLRUCache.java (working copy) @@ -17,8 +17,45 @@ * limitations under the License. */ -public class TestDoubleBarrelLRUCache extends BaseTestLRU { +import org.apache.lucene.util.LuceneTestCase; +public class TestDoubleBarrelLRUCache extends LuceneTestCase { + + private void testCache(DoubleBarrelLRUCache 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))); + } + } + public void testLRUCache() throws Exception { final int n = 100; testCache(new DoubleBarrelLRUCache(n), n); @@ -26,11 +63,11 @@ private class CacheThread extends Thread { private final Object[] objs; - private final Cache c; + private final DoubleBarrelLRUCache c; private final long endTime; volatile boolean failed; - public CacheThread(Cache c, + public CacheThread(DoubleBarrelLRUCache c, Object[] objs, long endTime) { this.c = c; this.objs = objs; @@ -81,7 +118,7 @@ final int CACHE_SIZE = 512; final int OBJ_COUNT = 3*CACHE_SIZE; - Cache c = new DoubleBarrelLRUCache(1024); + DoubleBarrelLRUCache c = new DoubleBarrelLRUCache(1024); Object[] objs = new Object[OBJ_COUNT]; for(int i=0;i 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))); - } - - } - -} Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (revision 942059) +++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardTermsDictReader.java (working copy) @@ -37,7 +37,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.cache.Cache; import org.apache.lucene.util.cache.DoubleBarrelLRUCache; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CodecUtil; @@ -68,7 +67,7 @@ private final Comparator termComp; // Caches the most recently looked-up field + terms: - private final Cache termsCache; + private final DoubleBarrelLRUCache termsCache; // Reads the terms index private StandardTermsIndexReader indexReader; Index: lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java =================================================================== --- lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (revision 942059) +++ lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (working copy) @@ -26,7 +26,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.CloseableThreadLocal; import org.apache.lucene.util.cache.DoubleBarrelLRUCache; -import org.apache.lucene.util.cache.Cache; /** This stores a monotonically increasing set of pairs in a * Directory. Pairs are accessed either by Term or by ordinal position the @@ -62,7 +61,7 @@ } } - private final Cache termsCache = new DoubleBarrelLRUCache(DEFAULT_CACHE_SIZE); + private final DoubleBarrelLRUCache termsCache = new DoubleBarrelLRUCache(DEFAULT_CACHE_SIZE); /** * Per-thread resources managed by ThreadLocal @@ -146,7 +145,6 @@ if (origEnum != null) origEnum.close(); threadResources.close(); - termsCache.close(); } /** Returns the number of term/value pairs in the set. */ Index: lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java =================================================================== --- lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java (revision 942059) +++ lucene/src/java/org/apache/lucene/util/DoubleBarrelLRUCache.java (working copy) @@ -41,7 +41,7 @@ * @lucene.internal */ -final public class DoubleBarrelLRUCache extends Cache { +final public class DoubleBarrelLRUCache { private final Map cache1; private final Map cache2; private final AtomicInteger countdown; @@ -55,17 +55,7 @@ cache2 = new ConcurrentHashMap(); } - @Override - public boolean containsKey(Object k) { - return false; - } - - @Override - public void close() { - } - - @Override @SuppressWarnings("unchecked") - public V get(Object key) { + public V get(K key) { final Map primary; final Map secondary; if (swapped) { @@ -83,13 +73,12 @@ result = secondary.get(key); if (result != null) { // Promote to primary - put((K) key, result); + put(key, result); } } return result; } - @Override public void put(K key, V value) { final Map primary; final Map secondary; Index: lucene/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java =================================================================== --- lucene/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java (revision 942059) +++ lucene/src/java/org/apache/lucene/util/cache/SimpleLRUCache.java (working copy) @@ -1,49 +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 java.util.LinkedHashMap; -import java.util.Map; - -/** - * 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}. - * - * @lucene.internal - */ -@Deprecated -public class SimpleLRUCache extends SimpleMapCache { - private final static float LOADFACTOR = 0.75f; - - /** - * Creates a last-recently-used cache with the specified size. - */ - public SimpleLRUCache(final int cacheSize) { - super(new LinkedHashMap((int) Math.ceil(cacheSize / LOADFACTOR) + 1, LOADFACTOR, true) { - @Override - protected boolean removeEldestEntry(Map.Entry eldest) { - return size() > cacheSize; - } - }); - } - -} Index: lucene/src/java/org/apache/lucene/util/cache/Cache.java =================================================================== --- lucene/src/java/org/apache/lucene/util/cache/Cache.java (revision 942059) +++ lucene/src/java/org/apache/lucene/util/cache/Cache.java (working copy) @@ -1,113 +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 java.io.Closeable; - -/** - * Base class for cache implementations. - * - * @lucene.internal - */ -public abstract class Cache implements Closeable { - - /** - * Simple Cache wrapper that synchronizes all - * calls that access the cache. - */ - static class SynchronizedCache extends Cache { - private Object mutex; - private Cache cache; - - SynchronizedCache(Cache cache) { - this.cache = cache; - this.mutex = this; - } - - SynchronizedCache(Cache cache, Object mutex) { - this.cache = cache; - this.mutex = mutex; - } - - @Override - public void put(K key, V value) { - synchronized(mutex) {cache.put(key, value);} - } - - @Override - public V get(Object key) { - synchronized(mutex) {return cache.get(key);} - } - - @Override - public boolean containsKey(Object key) { - synchronized(mutex) {return cache.containsKey(key);} - } - - @Override - public void close() { - synchronized(mutex) {cache.close();} - } - - @Override - Cache getSynchronizedCache() { - return this; - } - } - - /** - * Returns a thread-safe cache backed by the specified cache. - * In order to guarantee thread-safety, all access to the backed cache must - * be accomplished through the returned cache. - */ - public static Cache synchronizedCache(Cache cache) { - return cache.getSynchronizedCache(); - } - - /** - * Called by {@link #synchronizedCache(Cache)}. This method - * returns a {@link SynchronizedCache} instance that wraps - * this instance by default and can be overridden to return - * e. g. subclasses of {@link SynchronizedCache} or this - * in case this cache is already synchronized. - */ - Cache getSynchronizedCache() { - return new SynchronizedCache(this); - } - - /** - * Puts a (key, value)-pair into the cache. - */ - public abstract void put(K key, V value); - - /** - * Returns the value for the given key. - */ - public abstract V get(Object key); - - /** - * Returns whether the given key is in this cache. - */ - public abstract boolean containsKey(Object key); - - /** - * Closes the cache. - */ - public abstract void close(); - -} Index: lucene/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCache.java =================================================================== --- lucene/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCache.java (revision 942059) +++ lucene/src/java/org/apache/lucene/util/cache/DoubleBarrelLRUCache.java (working copy) @@ -1,124 +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 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. - * - *

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.

- * - *

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.

- * - * @lucene.internal - */ - -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; - - // Third, reset countdown - countdown.set(maxSize); - } - } -} Index: lucene/src/java/org/apache/lucene/util/cache/SimpleMapCache.java =================================================================== --- lucene/src/java/org/apache/lucene/util/cache/SimpleMapCache.java (revision 942059) +++ lucene/src/java/org/apache/lucene/util/cache/SimpleMapCache.java (working copy) @@ -1,117 +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 java.util.HashMap; -import java.util.Map; -import java.util.Set; - -/** - * 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}. - * - * @lucene.internal - */ -@Deprecated -public class SimpleMapCache extends Cache { - protected Map map; - - public SimpleMapCache() { - this(new HashMap()); - } - - public SimpleMapCache(Map map) { - this.map = map; - } - - @Override - public V get(Object key) { - return map.get(key); - } - - @Override - public void put(K key, V value) { - map.put(key, value); - } - - @Override - public void close() { - // NOOP - } - - @Override - public boolean containsKey(Object key) { - return map.containsKey(key); - } - - /** - * Returns a Set containing all keys in this cache. - */ - public Set keySet() { - return map.keySet(); - } - - @Override - Cache getSynchronizedCache() { - return new SynchronizedSimpleMapCache(this); - } - - private static class SynchronizedSimpleMapCache extends SimpleMapCache { - private Object mutex; - private SimpleMapCache cache; - - SynchronizedSimpleMapCache(SimpleMapCache cache) { - this.cache = cache; - this.mutex = this; - } - - @Override - public void put(K key, V value) { - synchronized(mutex) {cache.put(key, value);} - } - - @Override - public V get(Object key) { - synchronized(mutex) {return cache.get(key);} - } - - @Override - public boolean containsKey(Object key) { - synchronized(mutex) {return cache.containsKey(key);} - } - - @Override - public void close() { - synchronized(mutex) {cache.close();} - } - - @Override - public Set keySet() { - synchronized(mutex) {return cache.keySet();} - } - - @Override - Cache getSynchronizedCache() { - return this; - } - } -}