You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jackrabbit.apache.org by ju...@apache.org on 2010/10/01 15:57:08 UTC

svn commit: r1003542 - in /jackrabbit/trunk/jackrabbit-core/src: main/java/org/apache/jackrabbit/core/cache/ main/java/org/apache/jackrabbit/core/persistence/bundle/ main/java/org/apache/jackrabbit/core/persistence/pool/ main/java/org/apache/jackrabbit...

Author: jukka
Date: Fri Oct  1 13:57:07 2010
New Revision: 1003542

URL: http://svn.apache.org/viewvc?rev=1003542&view=rev
Log:
JCR-2699: Improve read/write concurrency

Refactor the LRU cache handling from MLRUItemStateCache and BundleCache to a new LRUCache class. Add a test case for the cache.

Switch from LinkedHashMap to ConcurrentHashMap with explicitly managed LRU linked list to reduce the size of the synchronized blocks.

Added:
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/cache/LRUCache.java   (with props)
    jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/
    jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/LRUCacheTest.java   (with props)
    jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/TestAll.java   (with props)
Removed:
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/util/BundleCache.java
Modified:
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/AbstractBundlePersistenceManager.java
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/pool/AbstractBundlePersistenceManager.java
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateCache.java
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateReferenceCache.java
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/MLRUItemStateCache.java
    jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ManagedMLRUItemStateCacheFactory.java

Added: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/cache/LRUCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/cache/LRUCache.java?rev=1003542&view=auto
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/cache/LRUCache.java (added)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/cache/LRUCache.java Fri Oct  1 13:57:07 2010
@@ -0,0 +1,282 @@
+/*
+ * 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.
+ */
+package org.apache.jackrabbit.core.cache;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.collections.map.LinkedMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An <code>ItemStateCache</code> implementation that internally uses a
+ * {@link LinkedMap} to maintain a cache of <code>ItemState</code> objects. The
+ * cache uses a rough estimate of the memory consumption of the cached item
+ * states for calculating the maximum number of entries. The oldest entries
+ * are flushed once the cache size has exceeded a certain limit.
+ * <p/>
+ * TODO rename class to something more appropriate, e.g. FIFOItemSateCache since
+ * it doesn't use a LRU eviction policy anymore.
+ */
+public class LRUCache<Key, Value> implements Cache {
+
+    /** Logger instance */
+    private static Logger log = LoggerFactory.getLogger(LRUCache.class);
+
+    /**
+     * Map of cache entries.
+     */
+    private final ConcurrentHashMap<Key, Entry<Key, Value>> entries =
+        new ConcurrentHashMap<Key, Entry<Key, Value>>();
+
+    /**
+     * Most recently used entry, or <code>null</code> if the cache is empty.
+     */
+    private Entry<Key, Value> first = null;
+
+    /**
+     * Least recently used entry, or <code>null</code> if the cache is empty.
+     */
+    private Entry<Key, Value> last = null;
+
+    /**
+     * Current size of the cache; sum of the sizes of all the cached entries.
+     */
+    private long currentSize = 0;
+
+    /**
+     * Maximum size of the cache
+     */
+    private long maximumSize;
+
+    /**
+     * Cache access listener
+     */
+    private CacheAccessListener listener = null;
+
+    /**
+     * Access count used to fire {@link CacheAccessListener#cacheAccessed()}
+     * calls once every {@link CacheAccessListener#ACCESS_INTERVAL} hits.
+     */
+    private int accessCount;
+
+    /**
+     * Constructs a new, empty <code>ItemStateCache</code> with the specified
+     * maximum memory.
+     *
+     * @param maximumSize maximum size of the cache
+     */
+    public LRUCache(long maximumSize) {
+        this.maximumSize = maximumSize;
+    }
+
+    public boolean containsKey(Key key) {
+        return entries.containsKey(key);
+    }
+
+    public Value get(Key key) {
+        Entry<Key, Value> entry = entries.get(key);
+        if (entry != null) {
+            boolean notifyAccessListener;
+            synchronized (this) {
+                // Check if we should notify the access listener
+                notifyAccessListener =
+                    (++accessCount % CacheAccessListener.ACCESS_INTERVAL) == 0;
+
+                // Move this entry to the beginning of the LRU linked list
+                if (entry.prev != null) {
+                    entry.prev.next = entry.next;
+                    if (entry.next != null) {
+                        entry.next.prev = entry.prev;
+                    } else if (entry.prev != null) {
+                        last = entry.prev;
+                        entry.prev = null;
+                    }
+                    first.prev = entry;
+                    entry.next = first;
+                    first = entry;
+                }
+            }
+
+            // Notify the access listener outside the synchronized block
+            if (notifyAccessListener && listener != null) {
+                listener.cacheAccessed();
+            }
+
+            return entry.value;
+        } else {
+            return null;
+        }
+    }
+
+    @SuppressWarnings("unchecked")
+    public synchronized Value[] values() {
+        Object[] values = new Object[entries.size()];
+        Entry<Key, Value> entry = first;
+        for (int i = 0; i < values.length && entry != null; i++) {
+            values[i] = entry.value;
+            entry = entry.next;
+        }
+        return (Value[]) values;
+    }
+
+    public synchronized void put(Key key, Value value, long size) {
+        Entry<Key, Value> entry = new Entry<Key, Value>(key, value, size);
+        if (first != null) {
+            first.prev = entry;
+        }
+        entry.next = first;
+        first = entry;
+        if (last == null) {
+            last = entry;
+        }
+
+        currentSize += size;
+
+        Entry<Key, Value> previous = entries.put(key, entry);
+        if (previous != null) {
+            log.warn("Overwriting cached entry {}", key);
+            currentSize -= previous.size;
+        }
+
+        shrinkIfNeeded();
+    }
+
+    public synchronized Value remove(Key key) {
+        Entry<Key, Value> entry = entries.remove(key);
+        if (entry != null) {
+            if (entry.prev != null) {
+                entry.prev.next = entry.next;
+            } else {
+                first = entry.next;
+            }
+            if (entry.next != null) {
+                entry.next.prev = entry.prev;
+            } else {
+                last = entry.prev;
+            }
+
+            currentSize -= entry.size;
+
+            return entry.value;
+        } else {
+            return null;
+        }
+    }
+
+    public synchronized void clear() {
+        entries.clear();
+        first = null;
+        last = null;
+        currentSize = 0;
+    }
+
+    public boolean isEmpty() {
+        return entries.isEmpty();
+    }
+
+    public synchronized long getAccessCount() {
+        return accessCount;
+    }
+
+    public synchronized void resetAccessCount() {
+        accessCount = 0;
+    }
+
+    public synchronized long getMemoryUsed() {
+        return currentSize;
+    }
+
+    public synchronized long getMaxMemorySize() {
+        return maximumSize;
+    }
+
+    public synchronized void setMaxMemorySize(long size) {
+        this.maximumSize = size;
+        shrinkIfNeeded();
+    }
+
+    private void shrinkIfNeeded() {
+        while (currentSize > maximumSize && last != null) {
+            entries.remove(last.key);
+            currentSize -= last.size;
+            if (last.prev != null) {
+                last.prev.next = null;
+                last = last.prev;
+            } else {
+                first = null;
+                last = null;
+            }
+        }
+    }
+
+    /**
+     * Set the cache access listener. Only one listener per cache is supported.
+     *
+     * @param listener the new listener
+     */
+    public synchronized void setAccessListener(CacheAccessListener listener) {
+        this.listener = listener;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    public synchronized void dispose() {
+        if (listener != null) {
+            listener.disposeCache(this);
+        }
+    }
+
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("{ ");
+        Entry<Key, Value> entry = first;
+        while (entry != null) {
+            builder.append(entry.key);
+            builder.append(" => ");
+            builder.append(entry.value);
+            builder.append(" ");
+        }
+        builder.append("}");
+        return builder.toString();
+    }
+
+    /**
+     * Internal cache entry.
+     */
+    private static class Entry<Key, Value> {
+
+        final Key key;
+
+        final Value value;
+
+        final long size;
+
+        Entry<Key, Value> prev = null;
+
+        Entry<Key, Value> next = null;
+
+        public Entry(Key key, Value value, long size) {
+            this.key = key;
+            this.value = value;
+            this.size = size;
+        }
+
+    }
+
+}

Propchange: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/cache/LRUCache.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/AbstractBundlePersistenceManager.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/AbstractBundlePersistenceManager.java?rev=1003542&r1=1003541&r2=1003542&view=diff
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/AbstractBundlePersistenceManager.java (original)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/bundle/AbstractBundlePersistenceManager.java Fri Oct  1 13:57:07 2010
@@ -18,6 +18,7 @@ package org.apache.jackrabbit.core.persi
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.jackrabbit.core.cache.LRUCache;
 import org.apache.jackrabbit.core.fs.FileSystemResource;
 import org.apache.jackrabbit.core.fs.FileSystem;
 import org.apache.jackrabbit.core.state.ItemState;
@@ -37,7 +38,6 @@ import org.apache.jackrabbit.core.persis
 import org.apache.jackrabbit.core.persistence.PersistenceManager;
 import org.apache.jackrabbit.core.util.StringIndex;
 import org.apache.jackrabbit.core.persistence.util.BLOBStore;
-import org.apache.jackrabbit.core.persistence.util.BundleCache;
 import org.apache.jackrabbit.core.persistence.util.FileBasedIndex;
 import org.apache.jackrabbit.core.persistence.util.LRUNodeIdCache;
 import org.apache.jackrabbit.core.persistence.util.NodePropBundle;
@@ -68,7 +68,7 @@ import javax.jcr.PropertyType;
  * included in the bundle but generated when required.
  * <p/>
  * In order to increase performance, there are 2 caches maintained. One is the
- * {@link BundleCache} that caches already loaded bundles. The other is the
+ * bundle cache that caches already loaded bundles. The other is the
  * {@link LRUNodeIdCache} that caches non-existent bundles. This is useful
  * because a lot of {@link #exists(NodeId)} calls are issued that would result
  * in a useless SQL execution if the desired bundle does not exist.
@@ -103,7 +103,7 @@ public abstract class AbstractBundlePers
     private StringIndex nameIndex;
 
     /** the cache of loaded bundles */
-    private BundleCache bundles;
+    private LRUCache<NodeId, NodePropBundle> bundles;
 
     /** the cache of non-existent bundles */
     private LRUNodeIdCache missing;
@@ -387,7 +387,7 @@ public abstract class AbstractBundlePers
     public void init(PMContext context) throws Exception {
         this.context = context;
         // init bundle cache
-        bundles = new BundleCache(bundleCacheSize);
+        bundles = new LRUCache<NodeId, NodePropBundle>(bundleCacheSize);
         missing = new LRUNodeIdCache();
     }
     
@@ -656,7 +656,7 @@ public abstract class AbstractBundlePers
                 bundle = loadBundle(id);
                 if (bundle != null) {
                     bundle.markOld();
-                    bundles.put(bundle);
+                    bundles.put(id, bundle, bundle.getSize());
                 } else {
                     missing.put(id);
                 }
@@ -692,8 +692,9 @@ public abstract class AbstractBundlePers
         missing.remove(bundle.getId());
         // only put to cache if already exists. this is to ensure proper overwrite
         // and not creating big contention during bulk loads
-        if (bundles.contains(bundle.getId())) {
-            bundles.put(bundle);
+        if (bundles.containsKey(bundle.getId())) {
+            bundles.remove(bundle.getId());
+            bundles.put(bundle.getId(), bundle, bundle.getSize());
         }
     }
 

Modified: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/pool/AbstractBundlePersistenceManager.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/pool/AbstractBundlePersistenceManager.java?rev=1003542&r1=1003541&r2=1003542&view=diff
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/pool/AbstractBundlePersistenceManager.java (original)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/persistence/pool/AbstractBundlePersistenceManager.java Fri Oct  1 13:57:07 2010
@@ -24,6 +24,7 @@ import javax.jcr.PropertyType;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.apache.jackrabbit.core.cache.LRUCache;
 import org.apache.jackrabbit.core.fs.FileSystemResource;
 import org.apache.jackrabbit.core.fs.FileSystem;
 import org.apache.jackrabbit.core.id.ItemId;
@@ -34,7 +35,6 @@ import org.apache.jackrabbit.core.persis
 import org.apache.jackrabbit.core.persistence.PMContext;
 import org.apache.jackrabbit.core.persistence.PersistenceManager;
 import org.apache.jackrabbit.core.persistence.util.BLOBStore;
-import org.apache.jackrabbit.core.persistence.util.BundleCache;
 import org.apache.jackrabbit.core.persistence.util.FileBasedIndex;
 import org.apache.jackrabbit.core.persistence.util.LRUNodeIdCache;
 import org.apache.jackrabbit.core.persistence.util.NodePropBundle;
@@ -68,7 +68,7 @@ import org.apache.jackrabbit.spi.commons
  * included in the bundle but generated when required.
  * <p/>
  * In order to increase performance, there are 2 caches maintained. One is the
- * {@link BundleCache} that caches already loaded bundles. The other is the
+ * bundle cache that caches already loaded bundles. The other is the
  * {@link LRUNodeIdCache} that caches non-existent bundles. This is useful
  * because a lot of {@link #exists(NodeId)} calls are issued that would result
  * in a useless SQL execution if the desired bundle does not exist.
@@ -103,7 +103,7 @@ public abstract class AbstractBundlePers
     private StringIndex nameIndex;
 
     /** the cache of loaded bundles */
-    private BundleCache bundles;
+    private LRUCache<NodeId, NodePropBundle> bundles;
 
     /** the cache of non-existent bundles */
     private LRUNodeIdCache missing;
@@ -387,7 +387,7 @@ public abstract class AbstractBundlePers
     public void init(PMContext context) throws Exception {
         this.context = context;
         // init bundle cache
-        bundles = new BundleCache(bundleCacheSize);
+        bundles = new LRUCache<NodeId, NodePropBundle>(bundleCacheSize);
         missing = new LRUNodeIdCache();
     }
 
@@ -654,7 +654,7 @@ public abstract class AbstractBundlePers
                 bundle = loadBundle(id);
                 if (bundle != null) {
                     bundle.markOld();
-                    bundles.put(bundle);
+                    bundles.put(id, bundle, bundle.getSize());
                 } else {
                     missing.put(id);
                 }
@@ -690,8 +690,9 @@ public abstract class AbstractBundlePers
         missing.remove(bundle.getId());
         // only put to cache if already exists. this is to ensure proper overwrite
         // and not creating big contention during bulk loads
-        if (bundles.contains(bundle.getId())) {
-            bundles.put(bundle);
+        if (bundles.containsKey(bundle.getId())) {
+            bundles.remove(bundle.getId());
+            bundles.put(bundle.getId(), bundle, bundle.getSize());
         }
     }
 

Modified: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateCache.java?rev=1003542&r1=1003541&r2=1003542&view=diff
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateCache.java (original)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateCache.java Fri Oct  1 13:57:07 2010
@@ -83,14 +83,6 @@ public interface ItemStateCache {
     boolean isEmpty();
 
     /**
-     * Informs the cache that the item was modified and the cache might need to
-     * recalculate the items caching weight.
-     *
-     * @param id the id of the item that was modified.
-     */
-    void update(ItemId id);
-
-    /**
      * Informs the cache that it is no longer in use.
      */
     void dispose();

Modified: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateReferenceCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateReferenceCache.java?rev=1003542&r1=1003541&r2=1003542&view=diff
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateReferenceCache.java (original)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ItemStateReferenceCache.java Fri Oct  1 13:57:07 2010
@@ -162,14 +162,6 @@ public class ItemStateReferenceCache imp
     /**
      * {@inheritDoc}
      */
-    public synchronized void update(ItemId id) {
-        // delegate
-        cache.update(id);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
     public synchronized boolean isEmpty() {
         // check primary cache
         return refs.isEmpty();

Modified: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/MLRUItemStateCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/MLRUItemStateCache.java?rev=1003542&r1=1003541&r2=1003542&view=diff
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/MLRUItemStateCache.java (original)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/MLRUItemStateCache.java Fri Oct  1 13:57:07 2010
@@ -16,14 +16,9 @@
  */
 package org.apache.jackrabbit.core.state;
 
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.commons.collections.map.LinkedMap;
-import org.apache.jackrabbit.core.cache.Cache;
-import org.apache.jackrabbit.core.cache.CacheAccessListener;
+import org.apache.jackrabbit.core.cache.CacheManager;
+import org.apache.jackrabbit.core.cache.LRUCache;
 import org.apache.jackrabbit.core.id.ItemId;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,139 +33,58 @@ import org.slf4j.LoggerFactory;
  * TODO rename class to something more appropriate, e.g. FIFOItemSateCache since
  * it doesn't use a LRU eviction policy anymore.
  */
-public class MLRUItemStateCache implements ItemStateCache, Cache {
+public class MLRUItemStateCache implements ItemStateCache {
+
     /** Logger instance */
     private static Logger log = LoggerFactory.getLogger(MLRUItemStateCache.class);
 
     /** default maximum memory to use */
     public static final int DEFAULT_MAX_MEM = 4 * 1024 * 1024;
 
-    /** the amount of memory the entries use */
-    private volatile long totalMem;
-
-    /** the maximum of memory the cache may use */
-    private volatile long maxMem;
-
     /** the number of writes */
-    private volatile long numWrites;
-
-    /** the access count */
-    private volatile long accessCount;
+    private volatile long numWrites = 0;
 
-    /** the cache access listeners */
-    private CacheAccessListener accessListener;
+    private final LRUCache<ItemId, ItemState> cache =
+        new LRUCache<ItemId, ItemState>(DEFAULT_MAX_MEM);
 
-    /**
-     * A cache for <code>ItemState</code> instances
-     */
-    private final Map<ItemId, Entry> cache;
-
-    /**
-     * Constructs a new, empty <code>ItemStateCache</code> with a maximum amount
-     * of memory of {@link #DEFAULT_MAX_MEM}.
-     */
-    public MLRUItemStateCache() {
-        this(DEFAULT_MAX_MEM);
-    }
-
-    /**
-     * Constructs a new, empty <code>ItemStateCache</code> with the specified
-     * maximum memory.
-     *
-     * @param maxMem the maximum amount of memory this cache may use.
-     */
-    @SuppressWarnings("serial")
-    private MLRUItemStateCache(int maxMem) {
-        this.maxMem = maxMem;
-        this.cache = new LinkedHashMap<ItemId, MLRUItemStateCache.Entry>(
-                maxMem / 1024, 0.75f, true /* access-ordered */) {
-            @Override
-            protected boolean removeEldestEntry(Map.Entry<ItemId, Entry> e) {
-                long maxMem = MLRUItemStateCache.this.maxMem;
-                if (totalMem <= maxMem) {
-                    return false;
-                } else if (totalMem - e.getValue().size <= maxMem) {
-                    totalMem -= e.getValue().size;
-                    return true;
-                } else {
-                    shrink();
-                    return false;
-                }
-            }
-        };
+    public MLRUItemStateCache(CacheManager cacheMgr) {
+        cacheMgr.add(cache);
+        cache.setAccessListener(cacheMgr);
     }
 
     //-------------------------------------------------------< ItemStateCache >
+
     /**
      * {@inheritDoc}
      */
     public boolean isCached(ItemId id) {
-        synchronized (cache) {
-            return cache.containsKey(id);
-        }
+        return cache.containsKey(id);
     }
 
     /**
      * {@inheritDoc}
      */
     public ItemState retrieve(ItemId id) {
-        touch();
-        synchronized (cache) {
-            Entry entry = cache.get(id);
-            if (entry != null) {
-                return entry.state;
-            } else {
-                return null;
-            }
-        }
+        return cache.get(id);
     }
 
     /**
      * {@inheritDoc}
      */
     public ItemState[] retrieveAll() {
-        synchronized (cache) {
-            ItemState[] states = new ItemState[cache.size()];
-            int i = 0;
-            for (Entry entry : cache.values()) {
-                states[i++] = entry.state;
-            }
-            return states;
-        }
+        return cache.values();
     }
 
     /**
      * {@inheritDoc}
      */
-    public void update(ItemId id) {
-        touch();
-        synchronized (cache) {
-            Entry entry = cache.get(id);
-            if (entry != null) {
-                totalMem -= entry.size;
-                entry.recalc();
-                totalMem += entry.size;
-            }
-        }
-    }
+    public synchronized void cache(ItemState state) {
+        cache.put(state.getId(), state, state.calculateMemoryFootprint());
 
-    /**
-     * {@inheritDoc}
-     */
-    public void cache(ItemState state) {
-        touch();
-        synchronized (cache) {
-            ItemId id = state.getId();
-            if (cache.containsKey(id)) {
-                log.warn("overwriting cached entry " + id);
-                evict(id);
-            }
-            Entry entry = new Entry(state);
-            totalMem += entry.size;
-            cache.put(id, entry);
-            if (numWrites++ % 10000 == 0 && log.isDebugEnabled()) {
-                log.debug(this + " size=" + cache.size() + ", " + totalMem + "/" + maxMem);
-            }
+        if (numWrites++ % 10000 == 0 && log.isDebugEnabled()) {
+            log.debug("Item state cache size: {}% of {} bytes",
+                    cache.getMemoryUsed() * 100 / cache.getMaxMemorySize(),
+                    cache.getMaxMemorySize());
         }
     }
 
@@ -178,135 +92,28 @@ public class MLRUItemStateCache implemen
      * {@inheritDoc}
      */
     public void evict(ItemId id) {
-        touch();
-        synchronized (cache) {
-            Entry entry = cache.remove(id);
-            if (entry != null) {
-                totalMem -= entry.size;
-            }
-        }
+        cache.remove(id);
     }
 
     /**
      * {@inheritDoc}
      */
     public void evictAll() {
-        synchronized (cache) {
-            cache.clear();
-            totalMem = 0;
-        }
+        cache.clear();
     }
 
     /**
      * {@inheritDoc}
      */
     public boolean isEmpty() {
-        synchronized (cache) {
-            return cache.isEmpty();
-        }
-    }
-
-    private void touch() {
-        accessCount++;
-        if ((accessCount % CacheAccessListener.ACCESS_INTERVAL) == 0) {
-            if (accessListener != null) {
-                accessListener.cacheAccessed();
-            }
-        }
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    public long getAccessCount() {
-        return accessCount;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    public long getMaxMemorySize() {
-        return maxMem;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    public long getMemoryUsed() {
-        return totalMem;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    public void resetAccessCount() {
-        synchronized (cache) {
-            accessCount = 0;
-        }
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    public void setMaxMemorySize(long size) {
-        synchronized (cache) {
-            this.maxMem = size;
-
-            // remove items, if too many
-            if (totalMem > maxMem) {
-                shrink();
-            }
-        }
-    }
-
-    private void shrink() {
-        List<Map.Entry<ItemId, Entry>> list =
-            new ArrayList<Map.Entry<ItemId, Entry>>(cache.entrySet());
-        for (int i = list.size() - 1; totalMem > maxMem && i >= 0; i--) {
-            Map.Entry<ItemId, Entry> last = list.get(i);
-            totalMem -= last.getValue().size;
-            cache.remove(last.getKey());
-        }
-    }
-
-    /**
-     * Set the cache access listener. Only one listener per cache is supported.
-     *
-     * @param listener the new listener
-     */
-    public void setAccessListener(CacheAccessListener listener) {
-        this.accessListener = listener;
+        return cache.isEmpty();
     }
 
     /**
      * {@inheritDoc}
      */
     public void dispose() {
-        synchronized (cache) {
-            if (accessListener != null) {
-                accessListener.disposeCache(this);
-            }
-        }
-    }
-
-
-    /**
-     * Internal cache entry.
-     */
-    private static class Entry {
-
-        private final ItemState state;
-
-        private long size;
-
-        public Entry(ItemState state) {
-            this.state = state;
-            this.size = 64 + state.calculateMemoryFootprint();
-        }
-
-        public void recalc() {
-            size = 64 + state.calculateMemoryFootprint();
-        }
+        cache.dispose();
     }
 
 }

Modified: jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ManagedMLRUItemStateCacheFactory.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ManagedMLRUItemStateCacheFactory.java?rev=1003542&r1=1003541&r2=1003542&view=diff
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ManagedMLRUItemStateCacheFactory.java (original)
+++ jackrabbit/trunk/jackrabbit-core/src/main/java/org/apache/jackrabbit/core/state/ManagedMLRUItemStateCacheFactory.java Fri Oct  1 13:57:07 2010
@@ -41,10 +41,7 @@ public class ManagedMLRUItemStateCacheFa
      * Create a new cache instance and link it to the cache manager.
      */
     public ItemStateCache newItemStateCache() {
-        MLRUItemStateCache cache = new MLRUItemStateCache();
-        cacheMgr.add(cache);
-        cache.setAccessListener(cacheMgr);
-        return cache;
+        return new MLRUItemStateCache(cacheMgr);
     }
 
 }

Added: jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/LRUCacheTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/LRUCacheTest.java?rev=1003542&view=auto
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/LRUCacheTest.java (added)
+++ jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/LRUCacheTest.java Fri Oct  1 13:57:07 2010
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+package org.apache.jackrabbit.core.cache;
+
+import junit.framework.TestCase;
+
+/**
+ * Test cases for the {@link LRUCache} class.
+ */
+public class LRUCacheTest extends TestCase {
+
+    public void testLRUCache() {
+        LRUCache<String, String> cache = new LRUCache<String, String>(3);
+
+        cache.put("foo", "1", 1);
+        assertTrue(cache.containsKey("foo"));
+        assertEquals("1", cache.get("foo"));
+        assertEquals(1, cache.getMemoryUsed());
+
+        cache.put("bar", "12", 2);
+        assertTrue(cache.containsKey("foo"));
+        assertEquals("1", cache.get("foo"));
+        assertTrue(cache.containsKey("bar"));
+        assertEquals("12", cache.get("bar"));
+        assertEquals(3, cache.getMemoryUsed());
+
+        cache.put("baz", "123", 3);
+        assertFalse(cache.containsKey("foo"));
+        assertFalse(cache.containsKey("bar"));
+        assertTrue(cache.containsKey("baz"));
+        assertEquals("123", cache.get("baz"));
+        assertEquals(3, cache.getMemoryUsed());
+
+        cache.put("foo", "1", 1);
+        assertTrue(cache.containsKey("foo"));
+        assertFalse(cache.containsKey("bar"));
+        assertFalse(cache.containsKey("baz"));
+        assertEquals("1", cache.get("foo"));
+        assertEquals(1, cache.getMemoryUsed());
+
+        cache.put("bar", "12", 2);
+        assertTrue(cache.containsKey("foo"));
+        assertEquals("1", cache.get("foo"));
+        assertTrue(cache.containsKey("bar"));
+        assertEquals("12", cache.get("bar"));
+        assertEquals(3, cache.getMemoryUsed());
+
+        cache.remove("foo");
+        assertFalse(cache.containsKey("foo"));
+        assertTrue(cache.containsKey("bar"));
+        assertEquals("12", cache.get("bar"));
+        assertEquals(2, cache.getMemoryUsed());
+
+        cache.put("foo", "1", 1);
+        assertTrue(cache.containsKey("foo"));
+        assertEquals("1", cache.get("foo"));
+        assertTrue(cache.containsKey("bar"));
+        assertEquals("12", cache.get("bar"));
+        assertEquals(3, cache.getMemoryUsed());
+    }
+
+}

Propchange: jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/LRUCacheTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/TestAll.java
URL: http://svn.apache.org/viewvc/jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/TestAll.java?rev=1003542&view=auto
==============================================================================
--- jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/TestAll.java (added)
+++ jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/TestAll.java Fri Oct  1 13:57:07 2010
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+package org.apache.jackrabbit.core.cache;
+
+import junit.framework.TestCase;
+import junit.framework.Test;
+import junit.framework.TestSuite;
+
+public class TestAll extends TestCase {
+
+    public static Test suite() {
+        TestSuite suite = new TestSuite();
+
+        suite.addTestSuite(LRUCacheTest.class);
+
+        return suite;
+    }
+
+}

Propchange: jackrabbit/trunk/jackrabbit-core/src/test/java/org/apache/jackrabbit/core/cache/TestAll.java
------------------------------------------------------------------------------
    svn:eol-style = native