mirror of https://github.com/apache/lucene.git
SOLR-1610 -- Generify SolrCache
git-svn-id: https://svn.apache.org/repos/asf/lucene/solr/trunk@890250 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
68ed49e294
commit
8e2e344157
|
@ -155,6 +155,8 @@ Other Changes
|
|||
|
||||
* Upgraded to Lucene 2.9-dev r888785 (shalin)
|
||||
|
||||
* SOLR-1610: Generify SolrCache (Jason Rutherglen via shalin)
|
||||
|
||||
Build
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -44,7 +44,7 @@ import java.lang.ref.WeakReference;
|
|||
public class ConcurrentLRUCache<K,V> {
|
||||
private static Logger log = LoggerFactory.getLogger(ConcurrentLRUCache.class);
|
||||
|
||||
private final ConcurrentHashMap<Object, CacheEntry> map;
|
||||
private final ConcurrentHashMap<Object, CacheEntry<K,V>> map;
|
||||
private final int upperWaterMark, lowerWaterMark;
|
||||
private final ReentrantLock markAndSweepLock = new ReentrantLock(true);
|
||||
private boolean isCleaning = false; // not volatile... piggybacked on other volatile vars
|
||||
|
@ -62,7 +62,7 @@ public class ConcurrentLRUCache<K,V> {
|
|||
if (upperWaterMark < 1) throw new IllegalArgumentException("upperWaterMark must be > 0");
|
||||
if (lowerWaterMark >= upperWaterMark)
|
||||
throw new IllegalArgumentException("lowerWaterMark must be < upperWaterMark");
|
||||
map = new ConcurrentHashMap<Object, CacheEntry>(initialSize);
|
||||
map = new ConcurrentHashMap<Object, CacheEntry<K,V>>(initialSize);
|
||||
newThreadForCleanup = runNewThreadForCleanup;
|
||||
this.upperWaterMark = upperWaterMark;
|
||||
this.lowerWaterMark = lowerWaterMark;
|
||||
|
@ -102,10 +102,10 @@ public class ConcurrentLRUCache<K,V> {
|
|||
return null;
|
||||
}
|
||||
|
||||
public Object put(K key, V val) {
|
||||
public V put(K key, V val) {
|
||||
if (val == null) return null;
|
||||
CacheEntry e = new CacheEntry(key, val, stats.accessCounter.incrementAndGet());
|
||||
CacheEntry oldCacheEntry = map.put(key, e);
|
||||
CacheEntry<K,V> e = new CacheEntry<K,V>(key, val, stats.accessCounter.incrementAndGet());
|
||||
CacheEntry<K,V> oldCacheEntry = map.put(key, e);
|
||||
int currentSize;
|
||||
if (oldCacheEntry == null) {
|
||||
currentSize = stats.size.incrementAndGet();
|
||||
|
@ -401,7 +401,7 @@ public class ConcurrentLRUCache<K,V> {
|
|||
Map<K, V> result = new LinkedHashMap<K, V>();
|
||||
TreeSet<CacheEntry> tree = new TreeSet<CacheEntry>();
|
||||
try {
|
||||
for (Map.Entry<Object, CacheEntry> entry : map.entrySet()) {
|
||||
for (Map.Entry<Object, CacheEntry<K,V>> entry : map.entrySet()) {
|
||||
CacheEntry ce = entry.getValue();
|
||||
ce.lastAccessedCopy = ce.lastAccessed;
|
||||
if (tree.size() < n) {
|
||||
|
@ -428,8 +428,8 @@ public class ConcurrentLRUCache<K,V> {
|
|||
Map<K,V> result = new LinkedHashMap<K,V>();
|
||||
TreeSet<CacheEntry> tree = new TreeSet<CacheEntry>();
|
||||
try {
|
||||
for (Map.Entry<Object, CacheEntry> entry : map.entrySet()) {
|
||||
CacheEntry ce = entry.getValue();
|
||||
for (Map.Entry<Object, CacheEntry<K,V>> entry : map.entrySet()) {
|
||||
CacheEntry<K,V> ce = entry.getValue();
|
||||
ce.lastAccessedCopy = ce.lastAccessed;
|
||||
if (tree.size() < n) {
|
||||
tree.add(ce);
|
||||
|
@ -457,11 +457,11 @@ public class ConcurrentLRUCache<K,V> {
|
|||
map.clear();
|
||||
}
|
||||
|
||||
public Map<Object, CacheEntry> getMap() {
|
||||
public Map<Object, CacheEntry<K,V>> getMap() {
|
||||
return map;
|
||||
}
|
||||
|
||||
private static class CacheEntry<K,V> implements Comparable<CacheEntry> {
|
||||
private static class CacheEntry<K,V> implements Comparable<CacheEntry<K,V>> {
|
||||
K key;
|
||||
V value;
|
||||
volatile long lastAccessed = 0;
|
||||
|
@ -478,7 +478,7 @@ public class ConcurrentLRUCache<K,V> {
|
|||
this.lastAccessed = lastAccessed;
|
||||
}
|
||||
|
||||
public int compareTo(CacheEntry that) {
|
||||
public int compareTo(CacheEntry<K,V> that) {
|
||||
if (this.lastAccessedCopy == that.lastAccessedCopy) return 0;
|
||||
return this.lastAccessedCopy < that.lastAccessedCopy ? 1 : -1;
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
|
|||
* @see org.apache.solr.search.SolrCache
|
||||
* @since solr 1.4
|
||||
*/
|
||||
public class FastLRUCache implements SolrCache {
|
||||
public class FastLRUCache<K,V> implements SolrCache<K,V> {
|
||||
|
||||
private List<ConcurrentLRUCache.Stats> cumulativeStats;
|
||||
|
||||
|
@ -54,7 +54,7 @@ public class FastLRUCache implements SolrCache {
|
|||
private State state;
|
||||
private CacheRegenerator regenerator;
|
||||
private String description = "Concurrent LRU Cache";
|
||||
private ConcurrentLRUCache cache;
|
||||
private ConcurrentLRUCache<K,V> cache;
|
||||
private int showItems = 0;
|
||||
|
||||
public Object init(Map args, Object persistence, CacheRegenerator regenerator) {
|
||||
|
@ -101,7 +101,7 @@ public class FastLRUCache implements SolrCache {
|
|||
}
|
||||
description += ')';
|
||||
|
||||
cache = new ConcurrentLRUCache(limit, minLimit, acceptableLimit, initialSize, newThread, false, null);
|
||||
cache = new ConcurrentLRUCache<K,V>(limit, minLimit, acceptableLimit, initialSize, newThread, false, null);
|
||||
cache.setAlive(false);
|
||||
|
||||
if (persistence == null) {
|
||||
|
@ -125,11 +125,11 @@ public class FastLRUCache implements SolrCache {
|
|||
|
||||
}
|
||||
|
||||
public Object put(Object key, Object value) {
|
||||
public V put(K key, V value) {
|
||||
return cache.put(key, value);
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
public V get(K key) {
|
||||
return cache.get(key);
|
||||
|
||||
}
|
||||
|
|
|
@ -31,7 +31,7 @@ import java.net.URL;
|
|||
/**
|
||||
* @version $Id$
|
||||
*/
|
||||
public class LRUCache implements SolrCache {
|
||||
public class LRUCache<K,V> implements SolrCache<K,V> {
|
||||
|
||||
/* An instance of this class will be shared across multiple instances
|
||||
* of an LRUCache at the same time. Make sure everything is thread safe.
|
||||
|
@ -54,7 +54,7 @@ public class LRUCache implements SolrCache {
|
|||
|
||||
private long warmupTime = 0;
|
||||
|
||||
private Map map;
|
||||
private Map<K,V> map;
|
||||
private String name;
|
||||
private int autowarmCount;
|
||||
private State state;
|
||||
|
@ -79,7 +79,7 @@ public class LRUCache implements SolrCache {
|
|||
}
|
||||
description += ')';
|
||||
|
||||
map = new LinkedHashMap(initialSize, 0.75f, true) {
|
||||
map = new LinkedHashMap<K,V>(initialSize, 0.75f, true) {
|
||||
protected boolean removeEldestEntry(Map.Entry eldest) {
|
||||
if (size() > limit) {
|
||||
// increment evictions regardless of state.
|
||||
|
@ -113,7 +113,7 @@ public class LRUCache implements SolrCache {
|
|||
}
|
||||
}
|
||||
|
||||
public Object put(Object key, Object value) {
|
||||
public V put(K key, V value) {
|
||||
synchronized (map) {
|
||||
if (state == State.LIVE) {
|
||||
stats.inserts.incrementAndGet();
|
||||
|
@ -126,9 +126,9 @@ public class LRUCache implements SolrCache {
|
|||
}
|
||||
}
|
||||
|
||||
public Object get(Object key) {
|
||||
public V get(K key) {
|
||||
synchronized (map) {
|
||||
Object val = map.get(key);
|
||||
V val = map.get(key);
|
||||
if (state == State.LIVE) {
|
||||
// only increment lookups and hits if we are live.
|
||||
lookups++;
|
||||
|
@ -156,10 +156,10 @@ public class LRUCache implements SolrCache {
|
|||
return state;
|
||||
}
|
||||
|
||||
public void warm(SolrIndexSearcher searcher, SolrCache old) throws IOException {
|
||||
public void warm(SolrIndexSearcher searcher, SolrCache<K,V> old) throws IOException {
|
||||
if (regenerator==null) return;
|
||||
long warmingStartTime = System.currentTimeMillis();
|
||||
LRUCache other = (LRUCache)old;
|
||||
LRUCache<K,V> other = (LRUCache<K,V>)old;
|
||||
|
||||
// warm entries
|
||||
if (autowarmCount != 0) {
|
||||
|
@ -172,7 +172,7 @@ public class LRUCache implements SolrCache {
|
|||
keys = new Object[sz];
|
||||
vals = new Object[sz];
|
||||
|
||||
Iterator iter = other.map.entrySet().iterator();
|
||||
Iterator<Map.Entry<K, V>> iter = other.map.entrySet().iterator();
|
||||
|
||||
// iteration goes from oldest (least recently used) to most recently used,
|
||||
// so we need to skip over the oldest entries.
|
||||
|
@ -181,7 +181,7 @@ public class LRUCache implements SolrCache {
|
|||
|
||||
|
||||
for (int i=0; i<sz; i++) {
|
||||
Map.Entry entry = (Map.Entry)iter.next();
|
||||
Map.Entry<K,V> entry = iter.next();
|
||||
keys[i]=entry.getKey();
|
||||
vals[i]=entry.getValue();
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ import java.io.IOException;
|
|||
*
|
||||
* @version $Id$
|
||||
*/
|
||||
public interface SolrCache extends SolrInfoMBean {
|
||||
public interface SolrCache<K,V> extends SolrInfoMBean {
|
||||
public final static Logger log = LoggerFactory.getLogger(SolrCache.class);
|
||||
|
||||
|
||||
|
@ -82,10 +82,10 @@ public interface SolrCache extends SolrInfoMBean {
|
|||
public int size();
|
||||
|
||||
/** :TODO: copy from Map */
|
||||
public Object put(Object key, Object value);
|
||||
public V put(K key, V value);
|
||||
|
||||
/** :TODO: copy from Map */
|
||||
public Object get(Object key);
|
||||
public V get(K key);
|
||||
|
||||
/** :TODO: copy from Map */
|
||||
public void clear();
|
||||
|
@ -125,7 +125,7 @@ public interface SolrCache extends SolrInfoMBean {
|
|||
* Warm this cache associated with <code>searcher</code> using the <code>old</code>
|
||||
* cache object. <code>this</code> and <code>old</code> will have the same concrete type.
|
||||
*/
|
||||
void warm(SolrIndexSearcher searcher, SolrCache old) throws IOException;
|
||||
void warm(SolrIndexSearcher searcher, SolrCache<K,V> old) throws IOException;
|
||||
// Q: an alternative to passing the searcher here would be to pass it in
|
||||
// init and have the cache implementation save it.
|
||||
|
||||
|
|
|
@ -67,10 +67,10 @@ public class SolrIndexSearcher extends IndexSearcher implements SolrInfoMBean {
|
|||
public final boolean enableLazyFieldLoading;
|
||||
|
||||
private final boolean cachingEnabled;
|
||||
private final SolrCache filterCache;
|
||||
private final SolrCache queryResultCache;
|
||||
private final SolrCache documentCache;
|
||||
private final SolrCache fieldValueCache;
|
||||
private final SolrCache<Query,DocSet> filterCache;
|
||||
private final SolrCache<QueryResultKey,DocList> queryResultCache;
|
||||
private final SolrCache<Integer,Document> documentCache;
|
||||
private final SolrCache<String,Object> fieldValueCache;
|
||||
|
||||
private final LuceneQueryOptimizer optimizer;
|
||||
|
||||
|
|
Loading…
Reference in New Issue