You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by md...@apache.org on 2016/07/20 07:58:48 UTC

svn commit: r1753464 - in /jackrabbit/oak/trunk/oak-segment-tar/src: main/java/org/apache/jackrabbit/oak/segment/ main/java/org/apache/jackrabbit/oak/segment/file/ main/java/org/apache/jackrabbit/oak/segment/http/ main/java/org/apache/jackrabbit/oak/se...

Author: mduerig
Date: Wed Jul 20 07:58:48 2016
New Revision: 1753464

URL: http://svn.apache.org/viewvc?rev=1753464&view=rev
Log:
OAK-4451: Implement a proper template cache
Generalise former StringCache to ReaderCache and derive StringCache and TemplateCache from it

Added:
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/ReaderCache.java
      - copied, changed from r1753449, jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/TemplateCache.java
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/RecordCacheTest.java
      - copied, changed from r1753449, jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/StringCacheTest.java
Removed:
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/StringCacheTest.java
Modified:
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/CachingSegmentReader.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/Segment.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreService.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/http/HttpStore.java
    jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/memory/MemoryStore.java
    jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentCompactionIT.java

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/CachingSegmentReader.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/CachingSegmentReader.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/CachingSegmentReader.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/CachingSegmentReader.java Wed Jul 20 07:58:48 2016
@@ -32,18 +32,17 @@ import org.apache.jackrabbit.oak.cache.C
 import org.apache.jackrabbit.oak.spi.blob.BlobStore;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
 
-// FIXME OAK-4451: Implement a proper template cache
-// - move the template cache into this class, implement monitoring, management, logging, tests
-
 /**
  * This {@code SegmentReader} implementation implements caching for
  * strings and templates. It can also optionally rely on a {@link BlobStore} for resolving blobs.
  */
 public class CachingSegmentReader implements SegmentReader {
     public static final int DEFAULT_STRING_CACHE_MB = 256;
-
     public static final String STRING_CACHE_MB = "oak.segment.stringCacheMB";
 
+    public static final int DEFAULT_TEMPLATE_CACHE_MB = 64;
+    public static final String TEMPLATE_CACHE_MB = "oak.segment.templateCacheMB";
+
     @Nonnull
     private final Supplier<SegmentWriter> writer;
 
@@ -57,6 +56,12 @@ public class CachingSegmentReader implem
     private final StringCache stringCache;
 
     /**
+     * Cache for template records
+     */
+    @Nonnull
+    private final TemplateCache templateCache;
+
+    /**
      * Create a new instance based on the supplied arguments.
      * @param writer          A {@code Supplier} for a the {@code SegmentWriter} used by the segment
      *                        builders returned from {@link NodeState#builder()} to write ahead changes.
@@ -64,14 +69,17 @@ public class CachingSegmentReader implem
      * @param blobStore       {@code BlobStore} instance of the underlying {@link SegmentStore}, or
      *                        {@code null} if none.
      * @param stringCacheMB   the size of the string cache in MBs or {@code 0} for no cache.
+     * @param templateCacheMB the size of the template cache in MBs or {@code 0} for no cache.
      */
     public CachingSegmentReader(
             @Nonnull Supplier<SegmentWriter> writer,
             @Nullable BlobStore blobStore,
-            long stringCacheMB) {
+            long stringCacheMB,
+            long templateCacheMB) {
         this.writer = checkNotNull(writer);
         this.blobStore = blobStore;
         stringCache = new StringCache(getLong(STRING_CACHE_MB, stringCacheMB) * 1024 * 1024);
+        templateCache = new TemplateCache(getLong(TEMPLATE_CACHE_MB, templateCacheMB * 1024 * 1024));
     }
 
     /**
@@ -83,8 +91,8 @@ public class CachingSegmentReader implem
         final SegmentId segmentId = id.getSegmentId();
         long msb = segmentId.getMostSignificantBits();
         long lsb = segmentId.getLeastSignificantBits();
-        return stringCache.getString(msb, lsb, id.getOffset(), new Function<Integer, String>() {
-            @Nullable
+        return stringCache.get(msb, lsb, id.getOffset(), new Function<Integer, String>() {
+            @Nonnull
             @Override
             public String apply(Integer offset) {
                 return segmentId.getSegment().readString(offset);
@@ -104,16 +112,16 @@ public class CachingSegmentReader implem
     @Nonnull
     @Override
     public Template readTemplate(@Nonnull RecordId id) {
-        int offset = id.getOffset();
-        if (id.getSegment().templates == null) {
-            return id.getSegment().readTemplate(offset);
-        }
-        Template template = id.getSegment().templates.get(offset);
-        if (template == null) {
-            template = id.getSegment().readTemplate(offset);
-            id.getSegment().templates.putIfAbsent(offset, template); // only keep the first copy
-        }
-        return template;
+        final SegmentId segmentId = id.getSegmentId();
+        long msb = segmentId.getMostSignificantBits();
+        long lsb = segmentId.getLeastSignificantBits();
+        return templateCache.get(msb, lsb, id.getOffset(), new Function<Integer, Template>() {
+            @Nonnull
+            @Override
+            public Template apply(Integer offset) {
+                return segmentId.getSegment().readTemplate(offset);
+            }
+        });
     }
 
     @Nonnull
@@ -145,4 +153,9 @@ public class CachingSegmentReader implem
     public CacheStats getStringCacheStats() {
         return stringCache.getStats();
     }
+
+    @Nonnull
+    public CacheStats getTemplateCacheStats() {
+        return templateCache.getStats();
+    }
 }

Copied: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/ReaderCache.java (from r1753449, jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java)
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/ReaderCache.java?p2=jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/ReaderCache.java&p1=jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java&r1=1753449&r2=1753464&rev=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/ReaderCache.java Wed Jul 20 07:58:48 2016
@@ -19,49 +19,58 @@
 
 package org.apache.jackrabbit.oak.segment;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import java.util.Arrays;
 
+import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
 
 import com.google.common.base.Function;
+import com.google.common.cache.Weigher;
 import org.apache.jackrabbit.oak.cache.CacheLIRS;
 import org.apache.jackrabbit.oak.cache.CacheStats;
-import static org.apache.jackrabbit.oak.commons.StringUtils.estimateMemoryUsage;
+import org.apache.jackrabbit.oak.cache.EmpiricalWeigher;
 
 /**
- * A string cache. It has two components: a fast cache for small strings, based
- * on an array, and a slow cache that uses a LIRS cache.
+ * A cache consisting of a fast and slow component. The fast cache for small items is based
+ * on an array, and a slow one uses a LIRS cache.
  */
-public class StringCache {
+public abstract class ReaderCache<T> {
+    @Nonnull
+    private final String name;
 
     /**
      * The fast (array based) cache.
      */
-    private final FastCache fastCache;
+    @CheckForNull
+    private final FastCache<T> fastCache;
 
     /**
      * The slower (LIRS) cache.
      */
-    private final CacheLIRS<StringCacheKey, String> cache;
+    @Nonnull
+    private final CacheLIRS<CacheKey<T>, T> cache;
 
     /**
      * Create a new string cache.
      *
      * @param maxSize the maximum memory in bytes.
      */
-    StringCache(long maxSize) {
+    protected ReaderCache(long maxSize, @Nonnull String name) {
+        this.name = checkNotNull(name);
         if (maxSize >= 0) {
-            fastCache = new FastCache();
-            cache = CacheLIRS.<StringCacheKey, String>newBuilder()
-                    .module("StringCache")
+            fastCache = new FastCache<>();
+            cache = CacheLIRS.<CacheKey<T>, T>newBuilder()
+                    .module(name)
                     .maximumWeight(maxSize)
                     .averageWeight(250)
                     .build();
         } else {
             fastCache = null;
             // dummy cache to prevent NPE on the getStats() call
-            cache = CacheLIRS.<StringCacheKey, String> newBuilder()
-                    .module("StringCache")
+            cache = CacheLIRS.<CacheKey<T>, T> newBuilder()
+                    .module(name)
                     .maximumSize(1)
                     .build();
         }
@@ -69,39 +78,55 @@ public class StringCache {
 
     @Nonnull
     public CacheStats getStats() {
-        return new CacheStats(cache, "String Cache", null, -1);
+        Weigher<?, ?> weigher = new Weigher<CacheKey<T>, T>() {
+            @Override
+            public int weigh(CacheKey<T> key, T value) {
+                return getEntryWeight(value);
+            }
+        };
+        return new CacheStats(cache, name, weigher, cache.getMaxMemory());
+    }
+
+    private static int getEntryHash(long lsb, long msb, int offset) {
+        int hash = (int) (msb ^ lsb) + offset;
+        hash = ((hash >>> 16) ^ hash) * 0x45d9f3b;
+        return (hash >>> 16) ^ hash;
     }
 
     /**
-     * Get the string, loading it if necessary.
+     * Get the value, loading it if necessary.
      *
      * @param msb the msb of the segment
      * @param lsb the lsb of the segment
      * @param offset the offset
-     * @param loader the string loader function
-     * @return the string (never null)
+     * @param loader the loader function
+     * @return the value
      */
-    public String getString(long msb, long lsb, int offset, Function<Integer, String> loader) {
+    @Nonnull
+    public T get(long msb, long lsb, int offset, Function<Integer, T> loader) {
         int hash = getEntryHash(msb, lsb, offset);
         if (fastCache == null) {
             // disabled cache
-            return loader.apply(offset);
+            T value = loader.apply(offset);
+            assert value != null;
+            return value;
+        }
+
+        T value = fastCache.get(hash, msb, lsb, offset);
+        if (value != null) {
+            return value;
+        }
+        CacheKey<T> key = new CacheKey<>(hash, msb, lsb, offset);
+        value = cache.getIfPresent(key);
+        if (value == null) {
+            value = loader.apply(offset);
+            assert value != null;
+            cache.put(key, value, getEntryWeight(value));
         }
-
-        String s = fastCache.getString(hash, msb, lsb, offset);
-        if (s != null) {
-            return s;
+        if (isSmall(value)) {
+            fastCache.put(hash, new FastCacheEntry<>(hash, msb, lsb, offset, value));
         }
-        StringCacheKey key = new StringCacheKey(hash, msb, lsb, offset);
-        s = cache.getIfPresent(key);
-        if (s == null) {
-            s = loader.apply(offset);
-            cache.put(key, s, getMemory(s));
-        }
-        if (FastCache.isSmall(s)) {
-            fastCache.addString(hash, new FastCacheEntry(hash, msb, lsb, offset, s));
-        }
-        return s;
+        return value;
     }
 
     /**
@@ -118,28 +143,17 @@ public class StringCache {
      * Estimation includes the key's overhead, see {@link EmpiricalWeigher} for
      * an example
      */
-    private static int getMemory(String s) {
-        int size = 168; // overhead for each cache entry
-        size += 40; // key
-        size += estimateMemoryUsage(s); // value
-        return size;
-    }
+    protected abstract int getEntryWeight(T value);
 
-    private static int getEntryHash(long lsb, long msb, int offset) {
-        int hash = (int) (msb ^ lsb) + offset;
-        hash = ((hash >>> 16) ^ hash) * 0x45d9f3b;
-        return (hash >>> 16) ^ hash;
-    }
+    /**
+     * Determine whether the entry is small, in which case it can be kept in the fast cache.
+     */
+    protected abstract boolean isSmall(T value);
 
     /**
      * A fast cache based on an array.
      */
-    static class FastCache {
-
-        /**
-         * The maximum number of characters in string that are cached.
-         */
-        static final int MAX_STRING_SIZE = 128;
+    private static class FastCache<T> {
 
         /**
          * The number of entries in the cache. Must be a power of 2.
@@ -149,7 +163,8 @@ public class StringCache {
         /**
          * The cache array.
          */
-        private final FastCacheEntry[] cache = new FastCacheEntry[CACHE_SIZE];
+        @SuppressWarnings("unchecked")
+        private final FastCacheEntry<T>[] elements = new FastCacheEntry[CACHE_SIZE];
 
         /**
          * Get the string if it is stored.
@@ -160,42 +175,32 @@ public class StringCache {
          * @param offset the offset
          * @return the string, or null
          */
-        String getString(int hash, long msb, long lsb, int offset) {
+        T get(int hash, long msb, long lsb, int offset) {
             int index = hash & (CACHE_SIZE - 1);
-            FastCacheEntry e = cache[index];
+            FastCacheEntry<T> e = elements[index];
             if (e != null && e.matches(msb, lsb, offset)) {
-                return e.string;
+                return e.value;
             }
             return null;
         }
 
         void clear() {
-            Arrays.fill(cache, null);
-        }
-
-        /**
-         * Whether the entry is small, in which case it can be kept in the fast cache.
-         * 
-         * @param s the string
-         * @return whether the entry is small
-         */
-        static boolean isSmall(String s) {
-            return s.length() <= MAX_STRING_SIZE;
+            Arrays.fill(elements, null);
         }
 
-        void addString(int hash, FastCacheEntry entry) {
+        void put(int hash, FastCacheEntry<T> entry) {
             int index = hash & (CACHE_SIZE - 1);
-            cache[index] = entry;
+            elements[index] = entry;
         }
 
     }
 
-    private static class StringCacheKey {
+    private static class CacheKey<T> {
         private final int hash;
         private final long msb, lsb;
         private final int offset;
 
-        StringCacheKey(int hash, long msb, long lsb, int offset) {
+        CacheKey(int hash, long msb, long lsb, int offset) {
             this.hash = hash;
             this.msb = msb;
             this.lsb = lsb;
@@ -212,10 +217,10 @@ public class StringCache {
             if (other == this) {
                 return true;
             }
-            if (!(other instanceof StringCacheKey)) {
+            if (!(other instanceof ReaderCache.CacheKey)) {
                 return false;
             }
-            StringCacheKey o = (StringCacheKey) other;
+            CacheKey<?> o = (CacheKey<?>) other;
             return o.hash == hash && o.msb == msb && o.lsb == lsb &&
                     o.offset == offset;
         }
@@ -229,19 +234,19 @@ public class StringCache {
 
     }
 
-    private static class FastCacheEntry {
+    private static class FastCacheEntry<T> {
 
         private final int hash;
         private final long msb, lsb;
         private final int offset;
-        private final String string;
+        private final T value;
 
-        FastCacheEntry(int hash, long msb, long lsb, int offset, String string) {
+        FastCacheEntry(int hash, long msb, long lsb, int offset, T value) {
             this.hash = hash;
             this.msb = msb;
             this.lsb = lsb;
             this.offset = offset;
-            this.string = string;
+            this.value = value;
         }
 
         boolean matches(long msb, long lsb, int offset) {
@@ -261,7 +266,7 @@ public class StringCache {
             if (!(other instanceof FastCacheEntry)) {
                 return false;
             }
-            FastCacheEntry o = (FastCacheEntry) other;
+            FastCacheEntry<?> o = (FastCacheEntry<?>) other;
             return o.hash == hash && o.msb == msb && o.lsb == lsb &&
                     o.offset == offset;
         }

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/Segment.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/Segment.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/Segment.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/Segment.java Wed Jul 20 07:58:48 2016
@@ -23,8 +23,6 @@ import static com.google.common.base.Pre
 import static com.google.common.base.Preconditions.checkPositionIndexes;
 import static com.google.common.base.Preconditions.checkState;
 import static com.google.common.collect.Lists.newArrayListWithCapacity;
-import static com.google.common.collect.Maps.newConcurrentMap;
-import static java.lang.Boolean.getBoolean;
 import static org.apache.jackrabbit.oak.commons.IOUtils.closeQuietly;
 import static org.apache.jackrabbit.oak.segment.SegmentId.isDataSegmentId;
 import static org.apache.jackrabbit.oak.segment.SegmentVersion.LATEST_VERSION;
@@ -38,7 +36,6 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentMap;
 
 import javax.annotation.CheckForNull;
 import javax.annotation.Nonnull;
@@ -146,16 +143,6 @@ public class Segment {
     private final SegmentId[] refids;
 
     /**
-     * Template records read from segment. Used to avoid duplicate
-     * copies and repeated parsing of the same templates.
-     * FIXME OAK-4451 move the template cache to the segment reader along side with the string cache
-     */
-    @CheckForNull
-    final ConcurrentMap<Integer, Template> templates;
-
-    private static final boolean DISABLE_TEMPLATE_CACHE = getBoolean("oak.segment.disableTemplateCache");
-
-    /**
      * Unpacks a 4 byte aligned segment offset.
      * @param offset  4 byte aligned segment offset
      * @return unpacked segment offset
@@ -192,12 +179,6 @@ public class Segment {
         this.store = checkNotNull(store);
         this.reader = checkNotNull(reader);
         this.id = checkNotNull(id);
-
-        if (DISABLE_TEMPLATE_CACHE) {
-            templates = null;
-        } else {
-            templates = newConcurrentMap();
-        }
         this.data = checkNotNull(data);
         if (id.isDataSegmentId()) {
             byte segmentVersion = data.get(3);
@@ -241,12 +222,6 @@ public class Segment {
         this.reader = checkNotNull(reader);
         this.id = store.newDataSegmentId();
         this.info = checkNotNull(info);
-        if (DISABLE_TEMPLATE_CACHE) {
-            templates = null;
-        } else {
-            templates = newConcurrentMap();
-        }
-
         this.data = ByteBuffer.wrap(checkNotNull(buffer));
         this.refids = new SegmentId[SEGMENT_REFERENCE_LIMIT + 1];
         this.refids[0] = id;

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreService.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreService.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreService.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/SegmentNodeStoreService.java Wed Jul 20 07:58:48 2016
@@ -370,7 +370,7 @@ public class SegmentNodeStoreService ext
                 segmentCacheStats.getName()
         );
 
-        // Expose stats about the string cache, if available
+        // Expose stats about the string and template caches
 
         CacheStats stringCacheStats = store.getStringCacheStats();
         stringCacheMBean = registerMBean(
@@ -380,6 +380,14 @@ public class SegmentNodeStoreService ext
                 stringCacheStats.getName()
         );
 
+        CacheStats templateCacheStats = store.getTemplateCacheStats();
+        stringCacheMBean = registerMBean(
+                whiteboard,
+                CacheStatsMBean.class,
+                templateCacheStats,CacheStats.TYPE,
+                templateCacheStats.getName()
+        );
+
         // Listen for Executor services on the whiteboard
 
         executor = new WhiteboardExecutor();

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/StringCache.java Wed Jul 20 07:58:48 2016
@@ -19,30 +19,13 @@
 
 package org.apache.jackrabbit.oak.segment;
 
-import java.util.Arrays;
-
-import javax.annotation.Nonnull;
-
-import com.google.common.base.Function;
-import org.apache.jackrabbit.oak.cache.CacheLIRS;
-import org.apache.jackrabbit.oak.cache.CacheStats;
 import static org.apache.jackrabbit.oak.commons.StringUtils.estimateMemoryUsage;
 
-/**
- * A string cache. It has two components: a fast cache for small strings, based
- * on an array, and a slow cache that uses a LIRS cache.
- */
-public class StringCache {
-
-    /**
-     * The fast (array based) cache.
-     */
-    private final FastCache fastCache;
-
+public class StringCache extends ReaderCache<String> {
     /**
-     * The slower (LIRS) cache.
+     * The maximum number of characters of string so they go into the fast cache
      */
-    private final CacheLIRS<StringCacheKey, String> cache;
+    private static final int MAX_STRING_SIZE = 128;
 
     /**
      * Create a new string cache.
@@ -50,222 +33,20 @@ public class StringCache {
      * @param maxSize the maximum memory in bytes.
      */
     StringCache(long maxSize) {
-        if (maxSize >= 0) {
-            fastCache = new FastCache();
-            cache = CacheLIRS.<StringCacheKey, String>newBuilder()
-                    .module("StringCache")
-                    .maximumWeight(maxSize)
-                    .averageWeight(250)
-                    .build();
-        } else {
-            fastCache = null;
-            // dummy cache to prevent NPE on the getStats() call
-            cache = CacheLIRS.<StringCacheKey, String> newBuilder()
-                    .module("StringCache")
-                    .maximumSize(1)
-                    .build();
-        }
-    }
-
-    @Nonnull
-    public CacheStats getStats() {
-        return new CacheStats(cache, "String Cache", null, -1);
-    }
-
-    /**
-     * Get the string, loading it if necessary.
-     *
-     * @param msb the msb of the segment
-     * @param lsb the lsb of the segment
-     * @param offset the offset
-     * @param loader the string loader function
-     * @return the string (never null)
-     */
-    public String getString(long msb, long lsb, int offset, Function<Integer, String> loader) {
-        int hash = getEntryHash(msb, lsb, offset);
-        if (fastCache == null) {
-            // disabled cache
-            return loader.apply(offset);
-        }
-
-        String s = fastCache.getString(hash, msb, lsb, offset);
-        if (s != null) {
-            return s;
-        }
-        StringCacheKey key = new StringCacheKey(hash, msb, lsb, offset);
-        s = cache.getIfPresent(key);
-        if (s == null) {
-            s = loader.apply(offset);
-            cache.put(key, s, getMemory(s));
-        }
-        if (FastCache.isSmall(s)) {
-            fastCache.addString(hash, new FastCacheEntry(hash, msb, lsb, offset, s));
-        }
-        return s;
-    }
-
-    /**
-     * Clear the cache.
-     */
-    public void clear() {
-        if (fastCache != null) {
-            cache.invalidateAll();
-            fastCache.clear();
-        }
+        super(maxSize, "String Cache");
     }
 
-    /**
-     * Estimation includes the key's overhead, see {@link EmpiricalWeigher} for
-     * an example
-     */
-    private static int getMemory(String s) {
+    @Override
+    protected int getEntryWeight(String string) {
         int size = 168; // overhead for each cache entry
         size += 40; // key
-        size += estimateMemoryUsage(s); // value
+        size += estimateMemoryUsage(string); // value
         return size;
     }
 
-    private static int getEntryHash(long lsb, long msb, int offset) {
-        int hash = (int) (msb ^ lsb) + offset;
-        hash = ((hash >>> 16) ^ hash) * 0x45d9f3b;
-        return (hash >>> 16) ^ hash;
-    }
-
-    /**
-     * A fast cache based on an array.
-     */
-    static class FastCache {
-
-        /**
-         * The maximum number of characters in string that are cached.
-         */
-        static final int MAX_STRING_SIZE = 128;
-
-        /**
-         * The number of entries in the cache. Must be a power of 2.
-         */
-        private static final int CACHE_SIZE = 16 * 1024;
-
-        /**
-         * The cache array.
-         */
-        private final FastCacheEntry[] cache = new FastCacheEntry[CACHE_SIZE];
-
-        /**
-         * Get the string if it is stored.
-         *
-         * @param hash the hash
-         * @param msb
-         * @param lsb
-         * @param offset the offset
-         * @return the string, or null
-         */
-        String getString(int hash, long msb, long lsb, int offset) {
-            int index = hash & (CACHE_SIZE - 1);
-            FastCacheEntry e = cache[index];
-            if (e != null && e.matches(msb, lsb, offset)) {
-                return e.string;
-            }
-            return null;
-        }
-
-        void clear() {
-            Arrays.fill(cache, null);
-        }
-
-        /**
-         * Whether the entry is small, in which case it can be kept in the fast cache.
-         * 
-         * @param s the string
-         * @return whether the entry is small
-         */
-        static boolean isSmall(String s) {
-            return s.length() <= MAX_STRING_SIZE;
-        }
-
-        void addString(int hash, FastCacheEntry entry) {
-            int index = hash & (CACHE_SIZE - 1);
-            cache[index] = entry;
-        }
-
-    }
-
-    private static class StringCacheKey {
-        private final int hash;
-        private final long msb, lsb;
-        private final int offset;
-
-        StringCacheKey(int hash, long msb, long lsb, int offset) {
-            this.hash = hash;
-            this.msb = msb;
-            this.lsb = lsb;
-            this.offset = offset;
-        }
-
-        @Override
-        public int hashCode() {
-            return hash;
-        }
-
-        @Override
-        public boolean equals(Object other) {
-            if (other == this) {
-                return true;
-            }
-            if (!(other instanceof StringCacheKey)) {
-                return false;
-            }
-            StringCacheKey o = (StringCacheKey) other;
-            return o.hash == hash && o.msb == msb && o.lsb == lsb &&
-                    o.offset == offset;
-        }
-
-        @Override
-        public String toString() {
-            return Long.toHexString(msb) +
-                ':' + Long.toHexString(lsb) +
-                '+' + Integer.toHexString(offset);
-        }
-
-    }
-
-    private static class FastCacheEntry {
-
-        private final int hash;
-        private final long msb, lsb;
-        private final int offset;
-        private final String string;
-
-        FastCacheEntry(int hash, long msb, long lsb, int offset, String string) {
-            this.hash = hash;
-            this.msb = msb;
-            this.lsb = lsb;
-            this.offset = offset;
-            this.string = string;
-        }
-
-        boolean matches(long msb, long lsb, int offset) {
-            return this.offset == offset && this.msb == msb && this.lsb == lsb;
-        }
-
-        @Override
-        public int hashCode() {
-            return hash;
-        }
-
-        @Override
-        public boolean equals(Object other) {
-            if (other == this) {
-                return true;
-            }
-            if (!(other instanceof FastCacheEntry)) {
-                return false;
-            }
-            FastCacheEntry o = (FastCacheEntry) other;
-            return o.hash == hash && o.msb == msb && o.lsb == lsb &&
-                    o.offset == offset;
-        }
-
+    @Override
+    protected boolean isSmall(String string) {
+        return string.length() <= MAX_STRING_SIZE;
     }
 
 }
\ No newline at end of file

Added: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/TemplateCache.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/TemplateCache.java?rev=1753464&view=auto
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/TemplateCache.java (added)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/TemplateCache.java Wed Jul 20 07:58:48 2016
@@ -0,0 +1,84 @@
+/*
+ * 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.oak.segment;
+
+import static org.apache.jackrabbit.oak.api.Type.STRING;
+
+import org.apache.jackrabbit.oak.api.PropertyState;
+import org.apache.jackrabbit.oak.commons.StringUtils;
+
+public class TemplateCache extends ReaderCache<Template> {
+
+    /**
+     * Create a new template cache.
+     *
+     * @param maxSize the maximum memory in bytes.
+     */
+    TemplateCache(long maxSize) {
+        super(maxSize, "Template Cache");
+    }
+
+    @Override
+    protected int getEntryWeight(Template template) {
+        int size = 168; // overhead for each cache entry
+        size += 40;     // key
+
+        size += estimateMemoryUsage(template.getPrimaryType());
+        size += estimateMemoryUsage(template.getMixinTypes());
+        size += estimateMemoryUsage(template.getChildName());
+        for (PropertyTemplate property : template.getPropertyTemplates()) {
+            size += estimateMemoryUsage(property);
+        }
+        return size;
+    }
+
+    private static int estimateMemoryUsage(PropertyTemplate propertyTemplate) {
+        return 4 + // index
+            estimateMemoryUsage(propertyTemplate.getName());
+    }
+
+    private static int estimateMemoryUsage(PropertyState propertyState) {
+        if (propertyState == null) {
+            return 0;
+        }
+
+        int size = estimateMemoryUsage(propertyState.getName());
+        for (int k = 0; k < propertyState.count(); k++) {
+            size += estimateMemoryUsage(propertyState.getValue(STRING, k));
+        }
+        return size;
+    }
+
+    private static int estimateMemoryUsage(String string) {
+        if (string == null) {
+            return 0;
+        }
+
+        return StringUtils.estimateMemoryUsage(string);
+    }
+
+    @Override
+    protected boolean isSmall(Template template) {
+        PropertyTemplate[] properties = template.getPropertyTemplates();
+        PropertyState mixins = template.getMixinTypes();
+        return properties.length == 0 && (mixins == null || mixins.count() == 0);
+    }
+
+}
\ No newline at end of file

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/file/FileStore.java Wed Jul 20 07:58:48 2016
@@ -37,6 +37,7 @@ import static java.util.concurrent.TimeU
 import static org.apache.jackrabbit.oak.commons.IOUtils.humanReadableByteCount;
 import static org.apache.jackrabbit.oak.plugins.memory.EmptyNodeState.EMPTY_NODE;
 import static org.apache.jackrabbit.oak.segment.CachingSegmentReader.DEFAULT_STRING_CACHE_MB;
+import static org.apache.jackrabbit.oak.segment.CachingSegmentReader.DEFAULT_TEMPLATE_CACHE_MB;
 import static org.apache.jackrabbit.oak.segment.SegmentId.isDataSegmentId;
 import static org.apache.jackrabbit.oak.segment.SegmentWriterBuilder.segmentWriterBuilder;
 import static org.apache.jackrabbit.oak.segment.file.GCListener.Status.FAILURE;
@@ -258,11 +259,13 @@ public class FileStore implements Segmen
         // FIXME OAK-4451: Implement a proper template cache: inject caches
         // from the outside so we can get rid of the cache stat accessors
         if (builder.getCacheSize() < 0) {
-            this.segmentReader = new CachingSegmentReader(getWriter, blobStore, 0);
+            this.segmentReader = new CachingSegmentReader(getWriter, blobStore, 0, 0);
         } else if (builder.getCacheSize() > 0) {
-            this.segmentReader = new CachingSegmentReader(getWriter, blobStore, (long) builder.getCacheSize());
+            this.segmentReader = new CachingSegmentReader(getWriter, blobStore,
+                    (long) builder.getCacheSize(), (long) builder.getCacheSize());
         } else {
-            this.segmentReader = new CachingSegmentReader(getWriter, blobStore, (long) DEFAULT_STRING_CACHE_MB);
+            this.segmentReader = new CachingSegmentReader(getWriter, blobStore,
+                    (long) DEFAULT_STRING_CACHE_MB, (long) DEFAULT_TEMPLATE_CACHE_MB);
         }
 
         Supplier<Integer> getGeneration = new Supplier<Integer>() {
@@ -434,6 +437,11 @@ public class FileStore implements Segmen
         return segmentReader.getStringCacheStats();
     }
 
+    @Nonnull
+    public CacheStats getTemplateCacheStats() {
+        return segmentReader.getTemplateCacheStats();
+    }
+
     public void maybeCompact(boolean cleanup) throws IOException {
         gcListener.info("TarMK GC #{}: started", GC_COUNT.incrementAndGet());
 

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/http/HttpStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/http/HttpStore.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/http/HttpStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/http/HttpStore.java Wed Jul 20 07:58:48 2016
@@ -19,6 +19,7 @@
 package org.apache.jackrabbit.oak.segment.http;
 
 import static org.apache.jackrabbit.oak.segment.CachingSegmentReader.DEFAULT_STRING_CACHE_MB;
+import static org.apache.jackrabbit.oak.segment.CachingSegmentReader.DEFAULT_TEMPLATE_CACHE_MB;
 import static org.apache.jackrabbit.oak.segment.SegmentWriterBuilder.segmentWriterBuilder;
 
 import java.io.IOException;
@@ -65,7 +66,7 @@ public class HttpStore implements Segmen
     };
     @Nonnull
     private final SegmentReader segmentReader = new CachingSegmentReader(
-            getWriter, null, DEFAULT_STRING_CACHE_MB);
+            getWriter, null, DEFAULT_STRING_CACHE_MB, DEFAULT_TEMPLATE_CACHE_MB);
 
     private final SegmentIdFactory segmentIdFactory = new SegmentIdFactory() {
 

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/memory/MemoryStore.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/memory/MemoryStore.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/memory/MemoryStore.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/main/java/org/apache/jackrabbit/oak/segment/memory/MemoryStore.java Wed Jul 20 07:58:48 2016
@@ -81,7 +81,7 @@ public class MemoryStore implements Segm
                 return getWriter();
             }
         };
-        this.segmentReader = new CachingSegmentReader(getWriter, null, 16);
+        this.segmentReader = new CachingSegmentReader(getWriter, null, 16, 2);
         this.segmentWriter = segmentWriterBuilder("sys").withWriterPool().build(this);
         revisions.bind(this);
         segmentWriter.flush();

Copied: jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/RecordCacheTest.java (from r1753449, jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/StringCacheTest.java)
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/RecordCacheTest.java?p2=jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/RecordCacheTest.java&p1=jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/StringCacheTest.java&r1=1753449&r2=1753464&rev=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/StringCacheTest.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/RecordCacheTest.java Wed Jul 20 07:58:48 2016
@@ -19,6 +19,7 @@
 
 package org.apache.jackrabbit.oak.segment;
 
+import static java.lang.String.valueOf;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -26,35 +27,35 @@ import java.util.ArrayList;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
-import org.apache.jackrabbit.oak.segment.StringCache;
 import org.junit.Test;
 
 import com.google.common.base.Function;
 
-public class StringCacheTest {
+public class RecordCacheTest {
 
     @Test
     public void empty() {
         final AtomicInteger counter = new AtomicInteger();
         Function<Integer, String> loader = new Function<Integer, String>() {
-            @Override @Nullable
+            @Override @Nonnull
             public String apply(@Nullable Integer input) {
                 counter.incrementAndGet();
-                return "" + input;
+                return valueOf(input);
             }
         };
         StringCache c = new StringCache(0);
         for (int repeat = 0; repeat < 10; repeat++) {
             for (int i = 0; i < 1000; i++) {
-                assertEquals("" + i, c.getString(i, i, i, loader));
+                assertEquals(valueOf(i), c.get(i, i, i, loader));
             }
         }
         // the LIRS cache should be almost empty (low hit rate there)
-        assertTrue("" + counter, counter.get() > 1000);
+        assertTrue(valueOf(counter), counter.get() > 1000);
         // but the fast cache should improve the total hit rate
-        assertTrue("" + counter, counter.get() < 5000);
+        assertTrue(valueOf(counter), counter.get() < 5000);
     }
     
     @Test
@@ -71,14 +72,14 @@ public class StringCacheTest {
         StringCache c = new StringCache(1024);
         for (int repeat = 0; repeat < 10; repeat++) {
             for (int i = 0; i < 1000; i++) {
-                assertEquals(large + i, c.getString(i, i, i, loader));
-                assertEquals(large + 0, c.getString(0, 0, 0, loader));
+                assertEquals(large + i, c.get(i, i, i, loader));
+                assertEquals(large + 0, c.get(0, 0, 0, loader));
             }
         }
         // the LIRS cache should be almost empty (low hit rate there)
         // and large strings are not kept in the fast cache, so hit rate should be bad
-        assertTrue("" + counter, counter.get() > 9000);
-        assertTrue("" + counter, counter.get() < 10000);
+        assertTrue(valueOf(counter), counter.get() > 9000);
+        assertTrue(valueOf(counter), counter.get() < 10000);
     }
     
     @Test
@@ -87,18 +88,18 @@ public class StringCacheTest {
         Function<Integer, String> uniqueLoader = new Function<Integer, String>() {
             @Override @Nullable
             public String apply(@Nullable Integer input) {
-                return "" + counter.incrementAndGet();
+                return valueOf(counter.incrementAndGet());
             }
         };
         StringCache c = new StringCache(0);
         // load a new entry
-        assertEquals("1", c.getString(0, 0, 0, uniqueLoader));
+        assertEquals("1", c.get(0, 0, 0, uniqueLoader));
         // but only once
-        assertEquals("1", c.getString(0, 0, 0, uniqueLoader));
+        assertEquals("1", c.get(0, 0, 0, uniqueLoader));
         c.clear();
         // after clearing the cache, load a new entry
-        assertEquals("2", c.getString(0, 0, 0, uniqueLoader));
-        assertEquals("2", c.getString(0, 0, 0, uniqueLoader));
+        assertEquals("2", c.get(0, 0, 0, uniqueLoader));
+        assertEquals("2", c.get(0, 0, 0, uniqueLoader));
     }
     
     @Test
@@ -121,7 +122,7 @@ public class StringCacheTest {
             int segment = r.nextInt(segmentCount);
             int offset = r.nextInt(10);
             Function<Integer, String> loader = loaderList.get(segment);
-            String x = c.getString(segment, segment, offset, loader);
+            String x = c.get(segment, segment, offset, loader);
             assertEquals(loader.apply(offset), x);
         }
     }

Modified: jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentCompactionIT.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentCompactionIT.java?rev=1753464&r1=1753463&r2=1753464&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentCompactionIT.java (original)
+++ jackrabbit/oak/trunk/oak-segment-tar/src/test/java/org/apache/jackrabbit/oak/segment/SegmentCompactionIT.java Wed Jul 20 07:58:48 2016
@@ -235,8 +235,6 @@ public class SegmentCompactionIT {
                 .build();
         nodeStore = SegmentNodeStoreBuilders.builder(fileStore).build();
 
-        CacheStats segmentCacheStats = fileStore.getSegmentCacheStats();
-        CacheStats stringCacheStats = fileStore.getStringCacheStats();
         List<Registration> registrations = newArrayList();
         registrations.add(registerMBean(segmentCompactionMBean,
                 new ObjectName("IT:TYPE=Segment Compaction")));
@@ -244,11 +242,15 @@ public class SegmentCompactionIT {
                 new ObjectName("IT:TYPE=Segment Revision GC")));
         registrations.add(registerMBean(fileStoreGCMonitor,
                 new ObjectName("IT:TYPE=GC Monitor")));
-        registrations.add(registerMBean(segmentCacheStats, new ObjectName("IT:TYPE=" + segmentCacheStats.getName())));
-        if (stringCacheStats != null) {
-            registrations.add(registerMBean(stringCacheStats,
-                    new ObjectName("IT:TYPE=" + stringCacheStats.getName())));
-        }
+        CacheStats segmentCacheStats = fileStore.getSegmentCacheStats();
+        registrations.add(registerMBean(segmentCacheStats,
+                new ObjectName("IT:TYPE=" + segmentCacheStats.getName())));
+        CacheStats stringCacheStats = fileStore.getStringCacheStats();
+        registrations.add(registerMBean(stringCacheStats,
+                new ObjectName("IT:TYPE=" + stringCacheStats.getName())));
+        CacheStats templateCacheStats = fileStore.getTemplateCacheStats();
+        registrations.add(registerMBean(templateCacheStats,
+                new ObjectName("IT:TYPE=" + templateCacheStats.getName())));
         mBeanRegistration = new CompositeRegistration(registrations);
     }