mirror of https://github.com/apache/lucene.git
LUCENE-2451: remove dead code in oal.util.cache; keep only DBLRUCache
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@942166 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
4a1c8c37c4
commit
25404004d7
|
@ -26,7 +26,6 @@ import org.apache.lucene.index.IndexFileNames;
|
|||
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 <Term, TermInfo> pairs in a
|
||||
* Directory. Pairs are accessed either by Term or by ordinal position the
|
||||
|
@ -62,7 +61,7 @@ public final class TermInfosReader {
|
|||
}
|
||||
}
|
||||
|
||||
private final Cache<Term,TermInfoAndOrd> termsCache = new DoubleBarrelLRUCache<Term,TermInfoAndOrd>(DEFAULT_CACHE_SIZE);
|
||||
private final DoubleBarrelLRUCache<Term,TermInfoAndOrd> termsCache = new DoubleBarrelLRUCache<Term,TermInfoAndOrd>(DEFAULT_CACHE_SIZE);
|
||||
|
||||
/**
|
||||
* Per-thread resources managed by ThreadLocal
|
||||
|
@ -146,7 +145,6 @@ public final class TermInfosReader {
|
|||
if (origEnum != null)
|
||||
origEnum.close();
|
||||
threadResources.close();
|
||||
termsCache.close();
|
||||
}
|
||||
|
||||
/** Returns the number of term/value pairs in the set. */
|
||||
|
|
|
@ -37,7 +37,6 @@ import org.apache.lucene.index.codecs.FieldsProducer;
|
|||
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 @@ public class StandardTermsDictReader extends FieldsProducer {
|
|||
private final Comparator<BytesRef> termComp;
|
||||
|
||||
// Caches the most recently looked-up field + terms:
|
||||
private final Cache<FieldAndTerm,TermState> termsCache;
|
||||
private final DoubleBarrelLRUCache<FieldAndTerm,TermState> termsCache;
|
||||
|
||||
// Reads the terms index
|
||||
private StandardTermsIndexReader indexReader;
|
||||
|
|
|
@ -41,7 +41,7 @@ import java.util.Map;
|
|||
* @lucene.internal
|
||||
*/
|
||||
|
||||
final public class DoubleBarrelLRUCache<K,V> extends Cache<K,V> {
|
||||
final public class DoubleBarrelLRUCache<K,V> {
|
||||
private final Map<K,V> cache1;
|
||||
private final Map<K,V> cache2;
|
||||
private final AtomicInteger countdown;
|
||||
|
@ -55,17 +55,7 @@ final public class DoubleBarrelLRUCache<K,V> extends Cache<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) {
|
||||
public V get(K key) {
|
||||
final Map<K,V> primary;
|
||||
final Map<K,V> secondary;
|
||||
if (swapped) {
|
||||
|
@ -83,13 +73,12 @@ final public class DoubleBarrelLRUCache<K,V> extends Cache<K,V> {
|
|||
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<K,V> primary;
|
||||
final Map<K,V> secondary;
|
|
@ -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<K,V> implements Closeable {
|
||||
|
||||
/**
|
||||
* Simple Cache wrapper that synchronizes all
|
||||
* calls that access the cache.
|
||||
*/
|
||||
static class SynchronizedCache<K,V> extends Cache<K,V> {
|
||||
private Object mutex;
|
||||
private Cache<K,V> cache;
|
||||
|
||||
SynchronizedCache(Cache<K,V> cache) {
|
||||
this.cache = cache;
|
||||
this.mutex = this;
|
||||
}
|
||||
|
||||
SynchronizedCache(Cache<K,V> 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<K,V> 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 <K,V> Cache<K,V> synchronizedCache(Cache<K,V> 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<K,V> getSynchronizedCache() {
|
||||
return new SynchronizedCache<K,V>(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();
|
||||
|
||||
}
|
|
@ -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<K,V> extends SimpleMapCache<K,V> {
|
||||
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<K,V>((int) Math.ceil(cacheSize / LOADFACTOR) + 1, LOADFACTOR, true) {
|
||||
@Override
|
||||
protected boolean removeEldestEntry(Map.Entry<K, V> eldest) {
|
||||
return size() > cacheSize;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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<K,V> extends Cache<K,V> {
|
||||
protected Map<K,V> map;
|
||||
|
||||
public SimpleMapCache() {
|
||||
this(new HashMap<K,V>());
|
||||
}
|
||||
|
||||
public SimpleMapCache(Map<K,V> 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<K> keySet() {
|
||||
return map.keySet();
|
||||
}
|
||||
|
||||
@Override
|
||||
Cache<K,V> getSynchronizedCache() {
|
||||
return new SynchronizedSimpleMapCache<K,V>(this);
|
||||
}
|
||||
|
||||
private static class SynchronizedSimpleMapCache<K,V> extends SimpleMapCache<K,V> {
|
||||
private Object mutex;
|
||||
private SimpleMapCache<K,V> cache;
|
||||
|
||||
SynchronizedSimpleMapCache(SimpleMapCache<K,V> 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<K> keySet() {
|
||||
synchronized(mutex) {return cache.keySet();}
|
||||
}
|
||||
|
||||
@Override
|
||||
Cache<K,V> getSynchronizedCache() {
|
||||
return this;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,60 +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 abstract class BaseTestLRU extends LuceneTestCase {
|
||||
|
||||
protected void testCache(Cache<Integer,Object> 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)));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -17,8 +17,45 @@ package org.apache.lucene.util.cache;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
public class TestDoubleBarrelLRUCache extends BaseTestLRU {
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
public class TestDoubleBarrelLRUCache extends LuceneTestCase {
|
||||
|
||||
private void testCache(DoubleBarrelLRUCache<Integer,Object> 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<Integer,Object>(n), n);
|
||||
|
@ -26,11 +63,11 @@ public class TestDoubleBarrelLRUCache extends BaseTestLRU {
|
|||
|
||||
private class CacheThread extends Thread {
|
||||
private final Object[] objs;
|
||||
private final Cache<Object,Object> c;
|
||||
private final DoubleBarrelLRUCache<Object,Object> c;
|
||||
private final long endTime;
|
||||
volatile boolean failed;
|
||||
|
||||
public CacheThread(Cache<Object,Object> c,
|
||||
public CacheThread(DoubleBarrelLRUCache<Object,Object> c,
|
||||
Object[] objs, long endTime) {
|
||||
this.c = c;
|
||||
this.objs = objs;
|
||||
|
@ -81,7 +118,7 @@ public class TestDoubleBarrelLRUCache extends BaseTestLRU {
|
|||
final int CACHE_SIZE = 512;
|
||||
final int OBJ_COUNT = 3*CACHE_SIZE;
|
||||
|
||||
Cache<Object,Object> c = new DoubleBarrelLRUCache<Object,Object>(1024);
|
||||
DoubleBarrelLRUCache<Object,Object> c = new DoubleBarrelLRUCache<Object,Object>(1024);
|
||||
|
||||
Object[] objs = new Object[OBJ_COUNT];
|
||||
for(int i=0;i<OBJ_COUNT;i++) {
|
||||
|
|
|
@ -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<Integer,Object>(n), n);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue