You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by er...@apache.org on 2012/01/12 21:06:00 UTC

svn commit: r1230744 - in /lucene/dev/branches/branch_3x/solr: ./ core/src/java/org/apache/solr/search/ core/src/java/org/apache/solr/util/ core/src/test-files/solr/conf/ core/src/test/org/apache/solr/search/

Author: erick
Date: Thu Jan 12 20:05:59 2012
New Revision: 1230744

URL: http://svn.apache.org/viewvc?rev=1230744&view=rev
Log:
Fixes for SOLR-2906. Many thanks to Shawn Heisey

Added:
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/LFUCache.java
    lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/util/ConcurrentLFUCache.java
    lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-caching.xml
    lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
Modified:
    lucene/dev/branches/branch_3x/solr/CHANGES.txt

Modified: lucene/dev/branches/branch_3x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/CHANGES.txt?rev=1230744&r1=1230743&r2=1230744&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_3x/solr/CHANGES.txt Thu Jan 12 20:05:59 2012
@@ -52,6 +52,8 @@ New Features
   Solr will use it as the root name for all MBeans Solr exposes via
   JMX.  The default root name is "solr" followed by the core name.
   (Constantijn Visinescu, hossman)
+  
+* SOLR-2906: Added LFU cache options to Solr. (Shawn Heisey via Erick Erickson)
 
 Optimizations
 ----------------------

Added: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/LFUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/LFUCache.java?rev=1230744&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/LFUCache.java (added)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/LFUCache.java Thu Jan 12 20:05:59 2012
@@ -0,0 +1,293 @@
+package org.apache.solr.search;
+/**
+ * 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.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.util.ConcurrentLFUCache;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URL;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * SolrCache based on ConcurrentLFUCache implementation.
+ * <p/>
+ * This implementation does not use a separate cleanup thread. Instead it uses the calling thread
+ * itself to do the cleanup when the size of the cache exceeds certain limits.
+ * <p/>
+ * Also see <a href="http://wiki.apache.org/solr/SolrCaching">SolrCaching</a>
+ * <p/>
+ * <b>This API is experimental and subject to change</b>
+ *
+ * @version $Id: LFUCache.java 1170772 2011-09-14 19:09:56Z sarowe $
+ * @see org.apache.solr.util.ConcurrentLFUCache
+ * @see org.apache.solr.search.SolrCache
+ * @since solr 3.6
+ */
+public class LFUCache<K, V> implements SolrCache<K, V> {
+
+  // contains the statistics objects for all open caches of the same type
+  private List<ConcurrentLFUCache.Stats> statsList;
+
+  private long warmupTime = 0;
+
+  private String name;
+  private int autowarmCount;
+  private State state;
+  private CacheRegenerator regenerator;
+  private String description = "Concurrent LFU Cache";
+  private ConcurrentLFUCache<K, V> cache;
+  private int showItems = 0;
+  private Boolean timeDecay = true;
+
+  public Object init(Map args, Object persistence, CacheRegenerator regenerator) {
+    state = State.CREATED;
+    this.regenerator = regenerator;
+    name = (String) args.get("name");
+    String str = (String) args.get("size");
+    int limit = str == null ? 1024 : Integer.parseInt(str);
+    int minLimit;
+    str = (String) args.get("minSize");
+    if (str == null) {
+      minLimit = (int) (limit * 0.9);
+    } else {
+      minLimit = Integer.parseInt(str);
+    }
+    if (minLimit == 0) minLimit = 1;
+    if (limit <= minLimit) limit = minLimit + 1;
+
+    int acceptableSize;
+    str = (String) args.get("acceptableSize");
+    if (str == null) {
+      acceptableSize = (int) (limit * 0.95);
+    } else {
+      acceptableSize = Integer.parseInt(str);
+    }
+    // acceptable limit should be somewhere between minLimit and limit
+    acceptableSize = Math.max(minLimit, acceptableSize);
+
+    str = (String) args.get("initialSize");
+    final int initialSize = str == null ? limit : Integer.parseInt(str);
+    str = (String) args.get("autowarmCount");
+    autowarmCount = str == null ? 0 : Integer.parseInt(str);
+    str = (String) args.get("cleanupThread");
+    boolean newThread = str == null ? false : Boolean.parseBoolean(str);
+
+    str = (String) args.get("showItems");
+    showItems = str == null ? 0 : Integer.parseInt(str);
+
+    // Don't make this "efficient" by removing the test, default is true and omitting the param will make it false.
+    str = (String) args.get("timeDecay");
+    timeDecay = (str == null) ? true : Boolean.parseBoolean(str);
+
+    description = "Concurrent LFU Cache(maxSize=" + limit + ", initialSize=" + initialSize +
+        ", minSize=" + minLimit + ", acceptableSize=" + acceptableSize + ", cleanupThread=" + newThread +
+        ", timeDecay=" + Boolean.toString(timeDecay);
+    if (autowarmCount > 0) {
+      description += ", autowarmCount=" + autowarmCount + ", regenerator=" + regenerator;
+    }
+    description += ')';
+
+    cache = new ConcurrentLFUCache<K, V>(limit, minLimit, acceptableSize, initialSize, newThread, false, null, timeDecay);
+    cache.setAlive(false);
+
+    statsList = (List<ConcurrentLFUCache.Stats>) persistence;
+    if (statsList == null) {
+      // must be the first time a cache of this type is being created
+      // Use a CopyOnWriteArrayList since puts are very rare and iteration may be a frequent operation
+      // because it is used in getStatistics()
+      statsList = new CopyOnWriteArrayList<ConcurrentLFUCache.Stats>();
+
+      // the first entry will be for cumulative stats of caches that have been closed.
+      statsList.add(new ConcurrentLFUCache.Stats());
+    }
+    statsList.add(cache.getStats());
+    return statsList;
+  }
+
+  public String name() {
+    return name;
+  }
+
+  public int size() {
+    return cache.size();
+
+  }
+
+  public V put(K key, V value) {
+    return cache.put(key, value);
+  }
+
+  public V get(K key) {
+    return cache.get(key);
+  }
+
+  public void clear() {
+    cache.clear();
+  }
+
+  public void setState(State state) {
+    this.state = state;
+    cache.setAlive(state == State.LIVE);
+  }
+
+  public State getState() {
+    return state;
+  }
+
+  public void warm(SolrIndexSearcher searcher, SolrCache old) throws IOException {
+    if (regenerator == null) return;
+    long warmingStartTime = System.currentTimeMillis();
+    LFUCache other = (LFUCache) old;
+    // warm entries
+    if (autowarmCount != 0) {
+      int sz = other.size();
+      if (autowarmCount != -1) sz = Math.min(sz, autowarmCount);
+      Map items = other.cache.getMostUsedItems(sz);
+      Map.Entry[] itemsArr = new Map.Entry[items.size()];
+      int counter = 0;
+      for (Object mapEntry : items.entrySet()) {
+        itemsArr[counter++] = (Map.Entry) mapEntry;
+      }
+      for (int i = itemsArr.length - 1; i >= 0; i--) {
+        try {
+          boolean continueRegen = regenerator.regenerateItem(searcher,
+              this, old, itemsArr[i].getKey(), itemsArr[i].getValue());
+          if (!continueRegen) break;
+        } catch (Throwable e) {
+          SolrException.log(log, "Error during auto-warming of key:" + itemsArr[i].getKey(), e);
+        }
+      }
+    }
+    warmupTime = System.currentTimeMillis() - warmingStartTime;
+  }
+
+
+  public void close() {
+    // add the stats to the cumulative stats object (the first in the statsList)
+    statsList.get(0).add(cache.getStats());
+    statsList.remove(cache.getStats());
+    cache.destroy();
+  }
+
+  //////////////////////// SolrInfoMBeans methods //////////////////////
+  public String getName() {
+    return LFUCache.class.getName();
+  }
+
+  public String getVersion() {
+    return SolrCore.version;
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  public Category getCategory() {
+    return Category.CACHE;
+  }
+
+  public String getSourceId() {
+    return "$Id: LFUCache.java 1170772 2011-09-14 19:09:56Z sarowe $";
+  }
+
+  public String getSource() {
+    return "$URL: http://svn.apache.org/repos/asf/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/search/LFUCache.java $";
+  }
+
+  public URL[] getDocs() {
+    return null;
+  }
+
+  // returns a ratio, not a percent.
+  private static String calcHitRatio(long lookups, long hits) {
+    if (lookups == 0) return "0.00";
+    if (lookups == hits) return "1.00";
+    int hundredths = (int) (hits * 100 / lookups);   // rounded down
+    if (hundredths < 10) return "0.0" + hundredths;
+    return "0." + hundredths;
+  }
+
+  public NamedList getStatistics() {
+    NamedList<Serializable> lst = new SimpleOrderedMap<Serializable>();
+    if (cache == null) return lst;
+    ConcurrentLFUCache.Stats stats = cache.getStats();
+    long lookups = stats.getCumulativeLookups();
+    long hits = stats.getCumulativeHits();
+    long inserts = stats.getCumulativePuts();
+    long evictions = stats.getCumulativeEvictions();
+    long size = stats.getCurrentSize();
+
+    lst.add("lookups", lookups);
+    lst.add("hits", hits);
+    lst.add("hitratio", calcHitRatio(lookups, hits));
+    lst.add("inserts", inserts);
+    lst.add("evictions", evictions);
+    lst.add("size", size);
+
+    lst.add("warmupTime", warmupTime);
+    lst.add("timeDecay", timeDecay);
+
+    long clookups = 0;
+    long chits = 0;
+    long cinserts = 0;
+    long cevictions = 0;
+
+    // NOTE: It is safe to iterate on a CopyOnWriteArrayList
+    for (ConcurrentLFUCache.Stats statistiscs : statsList) {
+      clookups += statistiscs.getCumulativeLookups();
+      chits += statistiscs.getCumulativeHits();
+      cinserts += statistiscs.getCumulativePuts();
+      cevictions += statistiscs.getCumulativeEvictions();
+    }
+    lst.add("cumulative_lookups", clookups);
+    lst.add("cumulative_hits", chits);
+    lst.add("cumulative_hitratio", calcHitRatio(clookups, chits));
+    lst.add("cumulative_inserts", cinserts);
+    lst.add("cumulative_evictions", cevictions);
+
+    if (showItems != 0) {
+      Map items = cache.getMostUsedItems(showItems == -1 ? Integer.MAX_VALUE : showItems);
+      for (Map.Entry e : (Set<Map.Entry>) items.entrySet()) {
+        Object k = e.getKey();
+        Object v = e.getValue();
+
+        String ks = "item_" + k;
+        String vs = v.toString();
+        lst.add(ks, vs);
+      }
+
+    }
+
+    return lst;
+  }
+
+  @Override
+  public String toString() {
+    return name + getStatistics().toString();
+  }
+}
+
+
+

Added: lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/util/ConcurrentLFUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/util/ConcurrentLFUCache.java?rev=1230744&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/util/ConcurrentLFUCache.java (added)
+++ lucene/dev/branches/branch_3x/solr/core/src/java/org/apache/solr/util/ConcurrentLFUCache.java Thu Jan 12 20:05:59 2012
@@ -0,0 +1,475 @@
+package org.apache.solr.util;
+/**
+ * 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
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.ref.WeakReference;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A LFU cache implementation based upon ConcurrentHashMap.
+ * <p/>
+ * This is not a terribly efficient implementation.  The tricks used in the
+ * LRU version were not directly usable, perhaps it might be possible to
+ * rewrite them with LFU in mind.
+ * <p/>
+ * <b>This API is experimental and subject to change</b>
+ *
+ * @version $Id: ConcurrentLFUCache.java 1170772 2011-09-14 19:09:56Z sarowe $
+ * @since solr 1.6
+ */
+public class ConcurrentLFUCache<K, V> {
+  private static Logger log = LoggerFactory.getLogger(ConcurrentLFUCache.class);
+
+  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
+  private final boolean newThreadForCleanup;
+  private volatile boolean islive = true;
+  private final Stats stats = new Stats();
+  private final int acceptableWaterMark;
+  private long lowHitCount = 0;  // not volatile, only accessed in the cleaning method
+  private final EvictionListener<K, V> evictionListener;
+  private CleanupThread cleanupThread;
+  private final boolean timeDecay;
+
+  public ConcurrentLFUCache(int upperWaterMark, final int lowerWaterMark, int acceptableSize,
+                            int initialSize, boolean runCleanupThread, boolean runNewThreadForCleanup,
+                            EvictionListener<K, V> evictionListener, boolean timeDecay) {
+    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<K, V>>(initialSize);
+    newThreadForCleanup = runNewThreadForCleanup;
+    this.upperWaterMark = upperWaterMark;
+    this.lowerWaterMark = lowerWaterMark;
+    this.acceptableWaterMark = acceptableSize;
+    this.evictionListener = evictionListener;
+    this.timeDecay = timeDecay;
+    if (runCleanupThread) {
+      cleanupThread = new CleanupThread(this);
+      cleanupThread.start();
+    }
+  }
+
+  public ConcurrentLFUCache(int size, int lowerWatermark) {
+    this(size, lowerWatermark, (int) Math.floor((lowerWatermark + size) / 2),
+        (int) Math.ceil(0.75 * size), false, false, null, true);
+  }
+
+  public void setAlive(boolean live) {
+    islive = live;
+  }
+
+  public V get(K key) {
+    CacheEntry<K, V> e = map.get(key);
+    if (e == null) {
+      if (islive) stats.missCounter.incrementAndGet();
+      return null;
+    }
+    if (islive) {
+      e.lastAccessed = stats.accessCounter.incrementAndGet();
+      e.hits.incrementAndGet();
+    }
+    return e.value;
+  }
+
+  public V remove(K key) {
+    CacheEntry<K, V> cacheEntry = map.remove(key);
+    if (cacheEntry != null) {
+      stats.size.decrementAndGet();
+      return cacheEntry.value;
+    }
+    return null;
+  }
+
+  public V put(K key, V val) {
+    if (val == null) return null;
+    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();
+    } else {
+      currentSize = stats.size.get();
+    }
+    if (islive) {
+      stats.putCounter.incrementAndGet();
+    } else {
+      stats.nonLivePutCounter.incrementAndGet();
+    }
+
+    // Check if we need to clear out old entries from the cache.
+    // isCleaning variable is checked instead of markAndSweepLock.isLocked()
+    // for performance because every put invokation will check until
+    // the size is back to an acceptable level.
+    //
+    // There is a race between the check and the call to markAndSweep, but
+    // it's unimportant because markAndSweep actually aquires the lock or returns if it can't.
+    //
+    // Thread safety note: isCleaning read is piggybacked (comes after) other volatile reads
+    // in this method.
+    if (currentSize > upperWaterMark && !isCleaning) {
+      if (newThreadForCleanup) {
+        new Thread() {
+          @Override
+          public void run() {
+            markAndSweep();
+          }
+        }.start();
+      } else if (cleanupThread != null) {
+        cleanupThread.wakeThread();
+      } else {
+        markAndSweep();
+      }
+    }
+    return oldCacheEntry == null ? null : oldCacheEntry.value;
+  }
+
+  /**
+   * Removes items from the cache to bring the size down
+   * to an acceptable value ('acceptableWaterMark').
+   * <p/>
+   * It is done in two stages. In the first stage, least recently used items are evicted.
+   * If, after the first stage, the cache size is still greater than 'acceptableSize'
+   * config parameter, the second stage takes over.
+   * <p/>
+   * The second stage is more intensive and tries to bring down the cache size
+   * to the 'lowerWaterMark' config parameter.
+   */
+  private void markAndSweep() {
+    if (!markAndSweepLock.tryLock()) return;
+    try {
+      long lowHitCount = this.lowHitCount;
+      isCleaning = true;
+      this.lowHitCount = lowHitCount;     // volatile write to make isCleaning visible
+
+      int sz = stats.size.get();
+
+      int wantToRemove = sz - lowerWaterMark;
+
+      TreeSet<CacheEntry> tree = new TreeSet<CacheEntry>();
+
+      for (CacheEntry<K, V> ce : map.values()) {
+        // set hitsCopy to avoid later Atomic reads
+        ce.hitsCopy = ce.hits.get();
+        ce.lastAccessedCopy = ce.lastAccessed;
+        if (timeDecay) {
+          ce.hits.set(ce.hitsCopy >>> 1);
+        }
+
+        if (tree.size() < wantToRemove) {
+          tree.add(ce);
+        } else {
+          // If the hits are not equal, we can remove before adding
+          // which is slightly faster
+          if (ce.hitsCopy < tree.first().hitsCopy) {
+            tree.remove(tree.first());
+            tree.add(ce);
+          } else if (ce.hitsCopy == tree.first().hitsCopy) {
+            tree.add(ce);
+            tree.remove(tree.first());
+          }
+        }
+      }
+
+      for (CacheEntry<K, V> e : tree) {
+        evictEntry(e.key);
+      }
+    } finally {
+      isCleaning = false;  // set before markAndSweep.unlock() for visibility
+      markAndSweepLock.unlock();
+    }
+  }
+
+  private void evictEntry(K key) {
+    CacheEntry<K, V> o = map.remove(key);
+    if (o == null) return;
+    stats.size.decrementAndGet();
+    stats.evictionCounter.incrementAndGet();
+    if (evictionListener != null) evictionListener.evictedEntry(o.key, o.value);
+  }
+
+  /**
+   * Returns 'n' number of least used entries present in this cache.
+   * <p/>
+   * This uses a TreeSet to collect the 'n' least used items ordered by ascending hitcount
+   * and returns a LinkedHashMap containing 'n' or less than 'n' entries.
+   *
+   * @param n the number of items needed
+   * @return a LinkedHashMap containing 'n' or less than 'n' entries
+   */
+  public Map<K, V> getLeastUsedItems(int n) {
+    Map<K, V> result = new LinkedHashMap<K, V>();
+    if (n <= 0)
+      return result;
+    TreeSet<CacheEntry> tree = new TreeSet<CacheEntry>();
+    // we need to grab the lock since we are changing the copy variables
+    markAndSweepLock.lock();
+    try {
+      for (Map.Entry<Object, CacheEntry<K, V>> entry : map.entrySet()) {
+        CacheEntry ce = entry.getValue();
+        ce.hitsCopy = ce.hits.get();
+        ce.lastAccessedCopy = ce.lastAccessed;
+        if (tree.size() < n) {
+          tree.add(ce);
+        } else {
+          // If the hits are not equal, we can remove before adding
+          // which is slightly faster
+          if (ce.hitsCopy < tree.first().hitsCopy) {
+            tree.remove(tree.first());
+            tree.add(ce);
+          } else if (ce.hitsCopy == tree.first().hitsCopy) {
+            tree.add(ce);
+            tree.remove(tree.first());
+          }
+        }
+      }
+    } finally {
+      markAndSweepLock.unlock();
+    }
+    for (CacheEntry<K, V> e : tree) {
+      result.put(e.key, e.value);
+    }
+    return result;
+  }
+
+  /**
+   * Returns 'n' number of most used entries present in this cache.
+   * <p/>
+   * This uses a TreeSet to collect the 'n' most used items ordered by descending hitcount
+   * and returns a LinkedHashMap containing 'n' or less than 'n' entries.
+   *
+   * @param n the number of items needed
+   * @return a LinkedHashMap containing 'n' or less than 'n' entries
+   */
+  public Map<K, V> getMostUsedItems(int n) {
+    Map<K, V> result = new LinkedHashMap<K, V>();
+    if (n <= 0)
+      return result;
+    TreeSet<CacheEntry> tree = new TreeSet<CacheEntry>();
+    // we need to grab the lock since we are changing the copy variables
+    markAndSweepLock.lock();
+    try {
+      for (Map.Entry<Object, CacheEntry<K, V>> entry : map.entrySet()) {
+        CacheEntry<K, V> ce = entry.getValue();
+        ce.hitsCopy = ce.hits.get();
+        ce.lastAccessedCopy = ce.lastAccessed;
+        if (tree.size() < n) {
+          tree.add(ce);
+        } else {
+          // If the hits are not equal, we can remove before adding
+          // which is slightly faster
+          if (ce.hitsCopy > tree.last().hitsCopy) {
+            tree.remove(tree.last());
+            tree.add(ce);
+          } else if (ce.hitsCopy == tree.last().hitsCopy) {
+            tree.add(ce);
+            tree.remove(tree.last());
+          }
+        }
+      }
+    } finally {
+      markAndSweepLock.unlock();
+    }
+    for (CacheEntry<K, V> e : tree) {
+      result.put(e.key, e.value);
+    }
+    return result;
+  }
+
+  public int size() {
+    return stats.size.get();
+  }
+
+  public void clear() {
+    map.clear();
+  }
+
+  public Map<Object, CacheEntry<K, V>> getMap() {
+    return map;
+  }
+
+  private static class CacheEntry<K, V> implements Comparable<CacheEntry<K, V>> {
+    K key;
+    V value;
+    volatile AtomicLong hits = new AtomicLong(0);
+    long hitsCopy = 0;
+    volatile long lastAccessed = 0;
+    long lastAccessedCopy = 0;
+
+    public CacheEntry(K key, V value, long lastAccessed) {
+      this.key = key;
+      this.value = value;
+      this.lastAccessed = lastAccessed;
+    }
+
+    public int compareTo(CacheEntry<K, V> that) {
+      if (this.hitsCopy == that.hitsCopy) {
+        if (this.lastAccessedCopy == that.lastAccessedCopy) {
+          return 0;
+        }
+        return this.lastAccessedCopy < that.lastAccessedCopy ? 1 : -1;
+      }
+      return this.hitsCopy < that.hitsCopy ? 1 : -1;
+    }
+
+    @Override
+    public int hashCode() {
+      return value.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return value.equals(obj);
+    }
+
+    @Override
+    public String toString() {
+      return "key: " + key + " value: " + value + " hits:" + hits.get();
+    }
+  }
+
+  private boolean isDestroyed = false;
+
+  public void destroy() {
+    try {
+      if (cleanupThread != null) {
+        cleanupThread.stopThread();
+      }
+    } finally {
+      isDestroyed = true;
+    }
+  }
+
+  public Stats getStats() {
+    return stats;
+  }
+
+
+  public static class Stats {
+    private final AtomicLong accessCounter = new AtomicLong(0),
+        putCounter = new AtomicLong(0),
+        nonLivePutCounter = new AtomicLong(0),
+        missCounter = new AtomicLong();
+    private final AtomicInteger size = new AtomicInteger();
+    private AtomicLong evictionCounter = new AtomicLong();
+
+    public long getCumulativeLookups() {
+      return (accessCounter.get() - putCounter.get() - nonLivePutCounter.get()) + missCounter.get();
+    }
+
+    public long getCumulativeHits() {
+      return accessCounter.get() - putCounter.get() - nonLivePutCounter.get();
+    }
+
+    public long getCumulativePuts() {
+      return putCounter.get();
+    }
+
+    public long getCumulativeEvictions() {
+      return evictionCounter.get();
+    }
+
+    public int getCurrentSize() {
+      return size.get();
+    }
+
+    public long getCumulativeNonLivePuts() {
+      return nonLivePutCounter.get();
+    }
+
+    public long getCumulativeMisses() {
+      return missCounter.get();
+    }
+
+    public void add(Stats other) {
+      accessCounter.addAndGet(other.accessCounter.get());
+      putCounter.addAndGet(other.putCounter.get());
+      nonLivePutCounter.addAndGet(other.nonLivePutCounter.get());
+      missCounter.addAndGet(other.missCounter.get());
+      evictionCounter.addAndGet(other.evictionCounter.get());
+      size.set(Math.max(size.get(), other.size.get()));
+    }
+  }
+
+  public static interface EvictionListener<K, V> {
+    public void evictedEntry(K key, V value);
+  }
+
+  private static class CleanupThread extends Thread {
+    private WeakReference<ConcurrentLFUCache> cache;
+
+    private boolean stop = false;
+
+    public CleanupThread(ConcurrentLFUCache c) {
+      cache = new WeakReference<ConcurrentLFUCache>(c);
+    }
+
+    @Override
+    public void run() {
+      while (true) {
+        synchronized (this) {
+          if (stop) break;
+          try {
+            this.wait();
+          } catch (InterruptedException e) {
+          }
+        }
+        if (stop) break;
+        ConcurrentLFUCache c = cache.get();
+        if (c == null) break;
+        c.markAndSweep();
+      }
+    }
+
+    void wakeThread() {
+      synchronized (this) {
+        this.notify();
+      }
+    }
+
+    void stopThread() {
+      synchronized (this) {
+        stop = true;
+        this.notify();
+      }
+    }
+  }
+
+  @Override
+  protected void finalize() throws Throwable {
+    try {
+      if (!isDestroyed) {
+        log.error("ConcurrentLFUCache was not destroyed prior to finalize(), indicates a bug -- POSSIBLE RESOURCE LEAK!!!");
+        destroy();
+      }
+    } finally {
+      super.finalize();
+    }
+  }
+}

Added: lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-caching.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-caching.xml?rev=1230744&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-caching.xml (added)
+++ lucene/dev/branches/branch_3x/solr/core/src/test-files/solr/conf/solrconfig-caching.xml Thu Jan 12 20:05:59 2012
@@ -0,0 +1,37 @@
+<!--
+  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.
+  -->
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <query>
+    <cache name="lfuCacheDecayFalse"
+           class="solr.search.LFUCache"
+           size="10"
+           initialSize="9"
+           timeDecay="false" />
+
+    <cache name="lfuCacheDecayTrue"
+           class="solr.search.LFUCache"
+           size="10"
+           initialSize="9"
+           timeDecay="true" />
+
+    <cache name="lfuCacheDecayDefault"
+           class="solr.search.LFUCache"
+           size="10"
+           initialSize="9" />
+  </query>
+</config>
\ No newline at end of file

Added: lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/search/TestLFUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/search/TestLFUCache.java?rev=1230744&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/search/TestLFUCache.java (added)
+++ lucene/dev/branches/branch_3x/solr/core/src/test/org/apache/solr/search/TestLFUCache.java Thu Jan 12 20:05:59 2012
@@ -0,0 +1,511 @@
+package org.apache.solr.search;
+
+/*
+ * 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.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.ConcurrentLFUCache;
+import org.apache.solr.util.RefCounted;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+
+/**
+ * Test for LFUCache
+ *
+ * @version $Id: TestFastLFUCache.java 1170772 2011-09-14 19:09:56Z sarowe $
+ * @see org.apache.solr.search.LFUCache
+ * @since solr 3.6
+ */
+public class TestLFUCache extends SolrTestCaseJ4 {
+
+  private class LFURegenerator implements CacheRegenerator {
+    public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache,
+                                  SolrCache oldCache, Object oldKey, Object oldVal) throws IOException {
+      newCache.put(oldKey, oldVal);
+      return true;
+    }
+  }
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-caching.xml", "schema.xml");
+  }
+
+  @Test
+  public void testTimeDecayParams() throws IOException {
+    RefCounted<SolrIndexSearcher> holder = h.getCore().getSearcher();
+    try {
+      SolrIndexSearcher searcher = holder.get();
+      LFUCache cacheDecayTrue = (LFUCache) searcher.getCache("lfuCacheDecayTrue");
+      assertNotNull(cacheDecayTrue);
+      NamedList stats = cacheDecayTrue.getStatistics();
+      assertTrue((Boolean) stats.get("timeDecay"));
+      addCache(cacheDecayTrue, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+      for (int idx = 0; idx < 64; ++idx) {
+        assertCache(cacheDecayTrue, 1, 2, 3, 4, 5);
+      }
+      addCache(cacheDecayTrue, 11, 12, 13, 14, 15);
+      assertCache(cacheDecayTrue, 1, 2, 3, 4, 5, 12, 13, 14, 15);
+
+      LFUCache cacheDecayDefault = (LFUCache) searcher.getCache("lfuCacheDecayDefault");
+      assertNotNull(cacheDecayDefault);
+      stats = cacheDecayDefault.getStatistics();
+      assertTrue((Boolean) stats.get("timeDecay"));
+      addCache(cacheDecayDefault, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+      assertCache(cacheDecayDefault, 1, 2, 3, 4, 5);
+      for (int idx = 0; idx < 64; ++idx) {
+        assertCache(cacheDecayDefault, 1, 2, 3, 4, 5);
+      }
+      addCache(cacheDecayDefault, 11, 12, 13, 14, 15);
+      assertCache(cacheDecayDefault, 1, 2, 3, 4, 5, 12, 13, 14, 15);
+      addCache(cacheDecayDefault, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21);
+      assertCache(cacheDecayDefault, 1, 2, 3, 4, 5, 17, 18, 19, 20, 21);
+
+      LFUCache cacheDecayFalse = (LFUCache) searcher.getCache("lfuCacheDecayFalse");
+      assertNotNull(cacheDecayFalse);
+      stats = cacheDecayFalse.getStatistics();
+      assertFalse((Boolean) stats.get("timeDecay"));
+      addCache(cacheDecayFalse, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
+      assertCache(cacheDecayFalse, 1, 2, 3, 4, 5);
+      for (int idx = 0; idx < 16; ++idx) {
+        assertCache(cacheDecayFalse, 1, 2, 3, 4, 5);
+      }
+      addCache(cacheDecayFalse, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21);
+
+      assertCache(cacheDecayFalse, 1, 2, 3, 4, 5);
+      assertNotCache(cacheDecayFalse, 6, 7, 8, 9, 10);
+      for (int idx = 22; idx < 256; ++idx) {
+        addCache(cacheDecayFalse, idx);
+      }
+      assertCache(cacheDecayFalse, 1, 2, 3, 4, 5);
+
+    } finally {
+      holder.decref();
+    }
+  }
+
+  private void addCache(LFUCache cache, int... inserts) {
+    for (int idx : inserts) {
+      cache.put(idx, Integer.toString(idx));
+    }
+  }
+
+  private void assertCache(LFUCache cache, int... gets) {
+    for (int idx : gets) {
+      if (cache.get(idx) == null) {
+        log.error(String.format("Expected entry %d not in cache", idx));
+        assertTrue(false);
+      }
+    }
+  }
+  private void assertNotCache(LFUCache cache, int... gets) {
+    for (int idx : gets) {
+      if (cache.get(idx) != null) {
+        log.error(String.format("Unexpected entry %d in cache", idx));
+        assertTrue(false);
+      }
+    }
+  }
+
+
+  @Test
+  public void testSimple() throws IOException {
+    LFUCache lfuCache = new LFUCache();
+    LFUCache newLFUCache = new LFUCache();
+    LFUCache noWarmLFUCache = new LFUCache();
+    try {
+      Map params = new HashMap();
+      params.put("size", "100");
+      params.put("initialSize", "10");
+      params.put("autowarmCount", "25");
+      LFURegenerator regenerator = new LFURegenerator();
+      Object initObj = lfuCache.init(params, null, regenerator);
+      lfuCache.setState(SolrCache.State.LIVE);
+      for (int i = 0; i < 101; i++) {
+        lfuCache.put(i + 1, "" + (i + 1));
+      }
+      assertEquals("15", lfuCache.get(15));
+      assertEquals("75", lfuCache.get(75));
+      assertEquals(null, lfuCache.get(110));
+      NamedList nl = lfuCache.getStatistics();
+      assertEquals(3L, nl.get("lookups"));
+      assertEquals(2L, nl.get("hits"));
+      assertEquals(101L, nl.get("inserts"));
+
+      assertEquals(null, lfuCache.get(1));  // first item put in should be the first out
+
+      // Test autowarming
+      newLFUCache.init(params, initObj, regenerator);
+      newLFUCache.warm(null, lfuCache);
+      newLFUCache.setState(SolrCache.State.LIVE);
+
+      newLFUCache.put(103, "103");
+      assertEquals("15", newLFUCache.get(15));
+      assertEquals("75", newLFUCache.get(75));
+      assertEquals(null, newLFUCache.get(50));
+      nl = newLFUCache.getStatistics();
+      assertEquals(3L, nl.get("lookups"));
+      assertEquals(2L, nl.get("hits"));
+      assertEquals(1L, nl.get("inserts"));
+      assertEquals(0L, nl.get("evictions"));
+
+      assertEquals(7L, nl.get("cumulative_lookups"));
+      assertEquals(4L, nl.get("cumulative_hits"));
+      assertEquals(102L, nl.get("cumulative_inserts"));
+      newLFUCache.close();
+
+      // Test no autowarming
+
+      params.put("autowarmCount", "0");
+      noWarmLFUCache.init(params, initObj, regenerator);
+      noWarmLFUCache.warm(null, lfuCache);
+      noWarmLFUCache.setState(SolrCache.State.LIVE);
+
+      noWarmLFUCache.put(103, "103");
+      assertNull(noWarmLFUCache.get(15));
+      assertNull(noWarmLFUCache.get(75));
+      assertEquals("103", noWarmLFUCache.get(103));
+    } finally {
+      if (newLFUCache != null) newLFUCache.close();
+      if (noWarmLFUCache != null) noWarmLFUCache.close();
+      if (lfuCache != null) lfuCache.close();
+    }
+  }
+
+  @Test
+  public void testItemOrdering() {
+    ConcurrentLFUCache<Integer, String> cache = new ConcurrentLFUCache<Integer, String>(100, 90);
+    try {
+      for (int i = 0; i < 50; i++) {
+        cache.put(i + 1, "" + (i + 1));
+      }
+      for (int i = 0; i < 44; i++) {
+        cache.get(i + 1);
+        cache.get(i + 1);
+      }
+      cache.get(1);
+      cache.get(1);
+      cache.get(1);
+      cache.get(3);
+      cache.get(3);
+      cache.get(3);
+      cache.get(5);
+      cache.get(5);
+      cache.get(5);
+      cache.get(7);
+      cache.get(7);
+      cache.get(7);
+      cache.get(9);
+      cache.get(9);
+      cache.get(9);
+      cache.get(48);
+      cache.get(48);
+      cache.get(48);
+      cache.get(50);
+      cache.get(50);
+      cache.get(50);
+      cache.get(50);
+      cache.get(50);
+
+      Map<Integer, String> m;
+
+      m = cache.getMostUsedItems(5);
+      //System.out.println(m);
+      // 50 9 7 5 3 1
+      assertNotNull(m.get(50));
+      assertNotNull(m.get(9));
+      assertNotNull(m.get(7));
+      assertNotNull(m.get(5));
+      assertNotNull(m.get(3));
+
+      m = cache.getLeastUsedItems(5);
+      //System.out.println(m);
+      // 49 47 46 45 2
+      assertNotNull(m.get(49));
+      assertNotNull(m.get(47));
+      assertNotNull(m.get(46));
+      assertNotNull(m.get(45));
+      assertNotNull(m.get(2));
+
+      m = cache.getLeastUsedItems(0);
+      assertTrue(m.isEmpty());
+
+      //test this too
+      m = cache.getMostUsedItems(0);
+      assertTrue(m.isEmpty());
+    } finally {
+      cache.destroy();
+    }
+  }
+
+  @Test
+  public void testTimeDecay() {
+    ConcurrentLFUCache<Integer, String> cacheDecay = new ConcurrentLFUCache<Integer, String>(10, 9);
+    try {
+      for (int i = 1; i < 21; i++) {
+        cacheDecay.put(i, Integer.toString(i));
+      }
+      Map<Integer, String> itemsDecay;
+
+      //11-20 now in cache.
+      itemsDecay = cacheDecay.getMostUsedItems(10);
+      for (int i = 11; i < 21; ++i) {
+        assertNotNull(itemsDecay.get(i));
+      }
+
+      // Now increase the freq count for 5 items
+      for (int i = 0; i < 5; ++i) {
+        for (int jdx = 0; jdx < 63; ++jdx) {
+          cacheDecay.get(i + 13);
+        }
+      }
+      // OK, 13 - 17 should have larger counts and should stick past next few collections. One collection should
+      // be triggered for each two insertions
+      cacheDecay.put(22, "22");
+      cacheDecay.put(23, "23"); // Surplus count at 32
+      cacheDecay.put(24, "24");
+      cacheDecay.put(25, "25"); // Surplus count at 16
+      itemsDecay = cacheDecay.getMostUsedItems(10);
+      // 13 - 17 should be in cache, but 11 and 18 (among others) should not Testing that elements before and
+      // after the ones with increased counts are removed, and all the increased count ones are still in the cache
+      assertNull(itemsDecay.get(11));
+      assertNull(itemsDecay.get(18));
+      assertNotNull(itemsDecay.get(13));
+      assertNotNull(itemsDecay.get(14));
+      assertNotNull(itemsDecay.get(15));
+      assertNotNull(itemsDecay.get(16));
+      assertNotNull(itemsDecay.get(17));
+
+
+      // Testing that all the elements in front of the ones with increased counts are gone
+      for (int idx = 26; idx < 32; ++idx) {
+        cacheDecay.put(idx, Integer.toString(idx));
+      }
+      //Surplus count should be at 0
+      itemsDecay = cacheDecay.getMostUsedItems(10);
+      assertNull(itemsDecay.get(20));
+      assertNull(itemsDecay.get(24));
+      assertNotNull(itemsDecay.get(13));
+      assertNotNull(itemsDecay.get(14));
+      assertNotNull(itemsDecay.get(15));
+      assertNotNull(itemsDecay.get(16));
+      assertNotNull(itemsDecay.get(17));
+
+      for (int idx = 32; idx < 40; ++idx) {
+        cacheDecay.put(idx, Integer.toString(idx));
+      }
+
+      // All the entries with increased counts should be gone.
+      itemsDecay = cacheDecay.getMostUsedItems(10);
+      System.out.println(itemsDecay);
+      assertNull(itemsDecay.get(13));
+      assertNull(itemsDecay.get(14));
+      assertNull(itemsDecay.get(15));
+      assertNull(itemsDecay.get(16));
+      assertNull(itemsDecay.get(17));
+      for (int idx = 30; idx < 40; ++idx) {
+        assertNotNull(itemsDecay.get(idx));
+      }
+    } finally {
+      cacheDecay.destroy();
+    }
+  }
+
+  @Test
+  public void testTimeNoDecay() {
+
+    ConcurrentLFUCache<Integer, String> cacheNoDecay = new ConcurrentLFUCache<Integer, String>(10, 9,
+        (int) Math.floor((9 + 10) / 2), (int) Math.ceil(0.75 * 10), false, false, null, false);
+    try {
+      for (int i = 1; i < 21; i++) {
+        cacheNoDecay.put(i, Integer.toString(i));
+      }
+      Map<Integer, String> itemsNoDecay;
+
+      //11-20 now in cache.
+      itemsNoDecay = cacheNoDecay.getMostUsedItems(10);
+      for (int i = 11; i < 21; ++i) {
+        assertNotNull(itemsNoDecay.get(i));
+      }
+
+      // Now increase the freq count for 5 items
+      for (int i = 0; i < 5; ++i) {
+        for (int jdx = 0; jdx < 10; ++jdx) {
+          cacheNoDecay.get(i + 13);
+        }
+      }
+      // OK, 13 - 17 should have larger counts but that shouldn't matter since timeDecay=false
+      cacheNoDecay.put(22, "22");
+      cacheNoDecay.put(23, "23");
+      cacheNoDecay.put(24, "24");
+      cacheNoDecay.put(25, "25");
+      itemsNoDecay = cacheNoDecay.getMostUsedItems(10);
+      for (int idx = 15; idx < 25; ++idx) {
+        assertNotNull(itemsNoDecay.get(15));
+      }
+    } finally {
+      cacheNoDecay.destroy();
+    }
+  }
+
+// From the original LRU cache tests, they're commented out there too because they take a while.
+//  void doPerfTest(int iter, int cacheSize, int maxKey) {
+//    long start = System.currentTimeMillis();
+//
+//    int lowerWaterMark = cacheSize;
+//    int upperWaterMark = (int) (lowerWaterMark * 1.1);
+//
+//    Random r = random;
+//    ConcurrentLFUCache cache = new ConcurrentLFUCache(upperWaterMark, lowerWaterMark,
+//        (upperWaterMark + lowerWaterMark) / 2, upperWaterMark, false, false, null, true);
+//    boolean getSize = false;
+//    int minSize = 0, maxSize = 0;
+//    for (int i = 0; i < iter; i++) {
+//      cache.put(r.nextInt(maxKey), "TheValue");
+//      int sz = cache.size();
+//      if (!getSize && sz >= cacheSize) {
+//        getSize = true;
+//        minSize = sz;
+//      } else {
+//        if (sz < minSize) minSize = sz;
+//        else if (sz > maxSize) maxSize = sz;
+//      }
+//    }
+//    cache.destroy();
+//
+//    long end = System.currentTimeMillis();
+//    System.out.println("time=" + (end - start) + ", minSize=" + minSize + ",maxSize=" + maxSize);
+//  }
+//
+//
+//  @Test
+//  public void testPerf() {
+//    doPerfTest(1000000, 100000, 200000); // big cache, warmup
+//    doPerfTest(2000000, 100000, 200000); // big cache
+//    doPerfTest(2000000, 100000, 120000);  // smaller key space increases distance between oldest, newest and makes the first passes less effective.
+//    doPerfTest(6000000, 1000, 2000);    // small cache, smaller hit rate
+//    doPerfTest(6000000, 1000, 1200);    // small cache, bigger hit rate
+//  }
+//
+//
+//  // returns number of puts
+//  int useCache(SolrCache sc, int numGets, int maxKey, int seed) {
+//    int ret = 0;
+//    Random r = new Random(seed);
+//
+//    // use like a cache... gets and a put if not found
+//    for (int i = 0; i < numGets; i++) {
+//      Integer k = r.nextInt(maxKey);
+//      Integer v = (Integer) sc.get(k);
+//      if (v == null) {
+//        sc.put(k, k);
+//        ret++;
+//      }
+//    }
+//
+//    return ret;
+//  }
+//
+//  void fillCache(SolrCache sc, int cacheSize, int maxKey) {
+//    for (int i = 0; i < cacheSize; i++) {
+//      Integer kv = random.nextInt(maxKey);
+//      sc.put(kv, kv);
+//    }
+//  }
+//
+//
+//  void cachePerfTest(final SolrCache sc, final int nThreads, final int numGets, int cacheSize, final int maxKey) {
+//    Map l = new HashMap();
+//    l.put("size", "" + cacheSize);
+//    l.put("initialSize", "" + cacheSize);
+//
+//    Object o = sc.init(l, null, null);
+//    sc.setState(SolrCache.State.LIVE);
+//
+//    fillCache(sc, cacheSize, maxKey);
+//
+//    long start = System.currentTimeMillis();
+//
+//    Thread[] threads = new Thread[nThreads];
+//    final AtomicInteger puts = new AtomicInteger(0);
+//    for (int i = 0; i < threads.length; i++) {
+//      final int seed = random.nextInt();
+//      threads[i] = new Thread() {
+//        @Override
+//        public void run() {
+//          int ret = useCache(sc, numGets / nThreads, maxKey, seed);
+//          puts.addAndGet(ret);
+//        }
+//      };
+//    }
+//
+//    for (Thread thread : threads) {
+//      try {
+//        thread.start();
+//      } catch (Exception e) {
+//        e.printStackTrace();
+//      }
+//    }
+//
+//    for (Thread thread : threads) {
+//      try {
+//        thread.join();
+//      } catch (Exception e) {
+//        e.printStackTrace();
+//      }
+//    }
+//
+//    long end = System.currentTimeMillis();
+//    System.out.println("time=" + (end - start) + " impl=" + sc.getClass().getSimpleName()
+//        + " nThreads= " + nThreads + " size=" + cacheSize + " maxKey=" + maxKey + " gets=" + numGets
+//        + " hitRatio=" + (1 - (((double) puts.get()) / numGets)));
+//  }
+//
+//  void perfTestBoth(int nThreads, int numGets, int cacheSize, int maxKey) {
+//    cachePerfTest(new LFUCache(), nThreads, numGets, cacheSize, maxKey);
+//  }
+//
+//
+//  public void testCachePerf() {
+//    // warmup
+//    perfTestBoth(2, 100000, 100000, 120000);
+//    perfTestBoth(1, 2000000, 100000, 100000); // big cache, 100% hit ratio
+//    perfTestBoth(2, 2000000, 100000, 100000); // big cache, 100% hit ratio
+//    perfTestBoth(1, 2000000, 100000, 120000); // big cache, bigger hit ratio
+//    perfTestBoth(2, 2000000, 100000, 120000); // big cache, bigger hit ratio
+//    perfTestBoth(1, 2000000, 100000, 200000); // big cache, ~50% hit ratio
+//    perfTestBoth(2, 2000000, 100000, 200000); // big cache, ~50% hit ratio
+//    perfTestBoth(1, 2000000, 100000, 1000000); // big cache, ~10% hit ratio
+//    perfTestBoth(2, 2000000, 100000, 1000000); // big cache, ~10% hit ratio
+//
+//    perfTestBoth(1, 2000000, 1000, 1000); // small cache, ~100% hit ratio
+//    perfTestBoth(2, 2000000, 1000, 1000); // small cache, ~100% hit ratio
+//    perfTestBoth(1, 2000000, 1000, 1200); // small cache, bigger hit ratio
+//    perfTestBoth(2, 2000000, 1000, 1200); // small cache, bigger hit ratio
+//    perfTestBoth(1, 2000000, 1000, 2000); // small cache, ~50% hit ratio
+//    perfTestBoth(2, 2000000, 1000, 2000); // small cache, ~50% hit ratio
+//    perfTestBoth(1, 2000000, 1000, 10000); // small cache, ~10% hit ratio
+//    perfTestBoth(2, 2000000, 1000, 10000); // small cache, ~10% hit ratio
+//  }
+
+}