Return-Path: X-Original-To: apmail-jackrabbit-oak-commits-archive@minotaur.apache.org Delivered-To: apmail-jackrabbit-oak-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 8167910F89 for ; Thu, 27 Jun 2013 09:46:14 +0000 (UTC) Received: (qmail 46386 invoked by uid 500); 27 Jun 2013 09:46:13 -0000 Delivered-To: apmail-jackrabbit-oak-commits-archive@jackrabbit.apache.org Received: (qmail 46343 invoked by uid 500); 27 Jun 2013 09:46:08 -0000 Mailing-List: contact oak-commits-help@jackrabbit.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: oak-dev@jackrabbit.apache.org Delivered-To: mailing list oak-commits@jackrabbit.apache.org Received: (qmail 46271 invoked by uid 99); 27 Jun 2013 09:46:05 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 27 Jun 2013 09:46:05 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 27 Jun 2013 09:45:43 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 432222388993; Thu, 27 Jun 2013 09:45:24 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1497267 - in /jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak: api/jmx/ cache/ kernel/ osgi/ plugins/mongomk/ plugins/segment/ Date: Thu, 27 Jun 2013 09:45:23 -0000 To: oak-commits@jackrabbit.apache.org From: chetanm@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130627094524.432222388993@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: chetanm Date: Thu Jun 27 09:45:22 2013 New Revision: 1497267 URL: http://svn.apache.org/r1497267 Log: OAK-863 - Enable stats for various caches used in Oak by default -- Enabling the stats collection by default -- Added new CacheStatsMBean class for exposing the cache stats via JMX -- Using the Whiteboard to register the JMX beans Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/api/jmx/ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/api/jmx/CacheStatsMBean.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/cache/ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/cache/CacheStats.java Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/kernel/KernelNodeStore.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/osgi/Activator.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMicroKernelService.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentCache.java jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/api/jmx/CacheStatsMBean.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/api/jmx/CacheStatsMBean.java?rev=1497267&view=auto ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/api/jmx/CacheStatsMBean.java (added) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/api/jmx/CacheStatsMBean.java Thu Jun 27 09:45:22 2013 @@ -0,0 +1,118 @@ +package org.apache.jackrabbit.oak.api.jmx; + +public interface CacheStatsMBean { + String TYPE = "CacheStats"; + + /** + * Returns the number of times {@link com.google.common.cache.Cache} lookup methods have returned either a cached or + * uncached value. This is defined as {@code getHitCount + getMissCount}. + */ + long getRequestCount(); + + /** + * Returns the number of times {@link com.google.common.cache.Cache} lookup methods have returned a cached value. + */ + long getHitCount(); + + /** + * Returns the ratio of cache requests which were hits. This is defined as + * {@code getHitCount / getRequestCount}, or {@code 1.0} when {@code getRequestCount == 0}. + * Note that {@code getHitRate + getMissRate =~ 1.0}. + */ + double getHitRate(); + /** + * Returns the number of times {@link com.google.common.cache.Cache} lookup methods have returned an uncached (newly + * loaded) value, or null. Multiple concurrent calls to {@link com.google.common.cache.Cache} lookup methods on an absent + * value can result in multiple misses, all returning the results of a single cache load + * operation. + */ + long getMissCount(); + + /** + * Returns the ratio of cache requests which were misses. This is defined as + * {@code getMissCount / getRequestCount}, or {@code 0.0} when {@code getRequestCount == 0}. + * Note that {@code getHitRate + getMissRate =~ 1.0}. Cache misses include all requests which + * weren't cache hits, including requests which resulted in either successful or failed loading + * attempts, and requests which waited for other threads to finish loading. It is thus the case + * that {@code getMissCount >= getLoadSuccessCount + getLoadExceptionCount}. Multiple + * concurrent misses for the same key will result in a single load operation. + */ + double getMissRate(); + + /** + * Returns the total number of times that {@link com.google.common.cache.Cache} lookup methods attempted to load new + * values. This includes both successful load operations, as well as those that threw + * exceptions. This is defined as {@code getLoadSuccessCount + getLoadExceptionCount}. + */ + long getLoadCount(); + + /** + * Returns the number of times {@link com.google.common.cache.Cache} lookup methods have successfully loaded a new value. + * This is always incremented in conjunction with {@link #getMissCount}, though {@code getMissCount} + * is also incremented when an exception is encountered during cache loading (see + * {@link #getLoadExceptionCount}). Multiple concurrent misses for the same key will result in a + * single load operation. + */ + long getLoadSuccessCount(); + + /** + * Returns the number of times {@link com.google.common.cache.Cache} lookup methods threw an exception while loading a + * new value. This is always incremented in conjunction with {@code getMissCount}, though + * {@code getMissCount} is also incremented when cache loading completes successfully (see + * {@link #getLoadSuccessCount}). Multiple concurrent misses for the same key will result in a + * single load operation. + */ + long getLoadExceptionCount(); + + /** + * Returns the ratio of cache loading attempts which threw exceptions. This is defined as + * {@code getLoadExceptionCount / (getLoadSuccessCount + getLoadExceptionCount)}, or + * {@code 0.0} when {@code getLoadSuccessCount + getLoadExceptionCount == 0}. + */ + double getLoadExceptionRate(); + + /** + * Returns the total number of nanoseconds the cache has spent loading new values. This can be + * used to calculate the miss penalty. This value is increased every time + * {@code getLoadSuccessCount} or {@code getLoadExceptionCount} is incremented. + */ + long getTotalLoadTime(); + + /** + * Returns the average time spent loading new values. This is defined as + * {@code getTotalLoadTime / (getLoadSuccessCount + getLoadExceptionCount)}. + */ + double getAverageLoadPenalty(); + + /** + * Returns the number of times an entry has been evicted. This count does not include manual + * {@linkplain com.google.common.cache.Cache#invalidate invalidations}. + */ + long getEvictionCount(); + + /** + * Get the number of elements/objects in the cache. + * @return the number of elements + */ + long getElementCount(); + + + /** + * Total weight of the complete cache. Depending on implementation it might be the amount + * of RAM taken by the cache + * @return to weight of the cache + */ + //Computing weight is costly hence its an operation + long estimateCurrentWeight(); + + /** + * The maximum weight of entries the cache may contain. + * @return the maximum total weight of entries the cache may contain + */ + long getMaxTotalWeight(); + + /** + * Gathers the stats of the cache for logging. + */ + String cacheInfoAsString(); +} Added: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/cache/CacheStats.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/cache/CacheStats.java?rev=1497267&view=auto ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/cache/CacheStats.java (added) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/cache/CacheStats.java Thu Jun 27 09:45:22 2013 @@ -0,0 +1,160 @@ +/* + * 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.cache; + +import java.util.Map; + +import com.google.common.base.Objects; +import com.google.common.cache.Cache; +import com.google.common.cache.Weigher; +import org.apache.jackrabbit.oak.api.jmx.CacheStatsMBean; + +public class CacheStats implements CacheStatsMBean{ + private final Cache cache; + private final Weigher weigher; + private final long maxWeight; + private final String name; + + public CacheStats(Cache cache, String name, Weigher weigher, long maxWeight) { + this.cache = cache; + this.name = name; + this.weigher = weigher; + this.maxWeight = maxWeight; + } + + @Override + public long getRequestCount() { + return stats().requestCount(); + } + + @Override + public long getHitCount() { + return stats().hitCount(); + } + + @Override + public double getHitRate() { + return stats().hitRate(); + } + + @Override + public long getMissCount() { + return stats().missCount(); + } + + @Override + public double getMissRate() { + return stats().missRate(); + } + + @Override + public long getLoadCount() { + return stats().loadCount(); + } + + @Override + public long getLoadSuccessCount() { + return stats().loadSuccessCount(); + } + + @Override + public long getLoadExceptionCount() { + return stats().loadExceptionCount(); + } + + @Override + public double getLoadExceptionRate() { + return stats().loadExceptionRate(); + } + + @Override + public long getTotalLoadTime() { + return stats().totalLoadTime(); + } + + @Override + public double getAverageLoadPenalty() { + return stats().averageLoadPenalty(); + } + + @Override + public long getEvictionCount() { + return stats().evictionCount(); + } + + @Override + public long getElementCount() { + return cache.size(); + } + + @Override + public long estimateCurrentWeight() { + if(weigher == null){ + return -1; + } + long size = 0; + for(Map.Entry e : cache.asMap().entrySet()){ + size += weigher.weigh(e.getKey(),e.getValue()); + } + return size; + } + + @Override + public long getMaxTotalWeight() { + return maxWeight; + } + + @Override + public String cacheInfoAsString() { + return Objects.toStringHelper("CacheStats") + .add("hitCount", getHitCount()) + .add("missCount", getMissCount()) + .add("loadSuccessCount", getLoadSuccessCount()) + .add("lLoadExceptionCount", getLoadExceptionCount()) + .add("totalLoadTime", getTotalLoadTime()) + .add("evictionCount", getEvictionCount()) + .add("elementCount", getElementCount()) + .add("totalWeight", humanReadableByteCount(estimateCurrentWeight(), true)) + .add("maxWeight", humanReadableByteCount(getMaxTotalWeight(),true)) + .toString(); + } + + public String getName() { + return name; + } + + private com.google.common.cache.CacheStats stats() { + return cache.stats(); + } + + /** + * Based on http://stackoverflow.com/a/3758880/1035417 + */ + private static String humanReadableByteCount(long bytes, boolean si) { + if(bytes < 0){ + return "0"; + } + int unit = si ? 1000 : 1024; + if (bytes < unit) return bytes + " B"; + int exp = (int) (Math.log(bytes) / Math.log(unit)); + String pre = (si ? "kMGTPE" : "KMGTPE").charAt(exp-1) + (si ? "" : "i"); + return String.format("%.1f %sB", bytes / Math.pow(unit, exp), pre); + } +} Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/kernel/KernelNodeStore.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/kernel/KernelNodeStore.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/kernel/KernelNodeStore.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/kernel/KernelNodeStore.java Thu Jun 27 09:45:22 2013 @@ -37,6 +37,7 @@ import org.apache.jackrabbit.oak.spi.com import org.apache.jackrabbit.oak.spi.state.AbstractNodeStore; import org.apache.jackrabbit.oak.spi.state.NodeState; import org.apache.jackrabbit.oak.spi.state.NodeStoreBranch; +import org.apache.jackrabbit.oak.cache.CacheStats; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; @@ -61,6 +62,8 @@ public class KernelNodeStore extends Abs private final LoadingCache cache; + private final CacheStats cacheStats; + /** * State of the current root node. */ @@ -68,14 +71,17 @@ public class KernelNodeStore extends Abs public KernelNodeStore(final MicroKernel kernel, long cacheSize) { this.kernel = checkNotNull(kernel); + + Weigher weigher = new Weigher() { + @Override + public int weigh(String key, KernelNodeState state) { + return state.getMemory(); + } + }; this.cache = CacheBuilder.newBuilder() .maximumWeight(cacheSize) - .weigher(new Weigher() { - @Override - public int weigh(String key, KernelNodeState state) { - return state.getMemory(); - } - }).build(new CacheLoader() { + .weigher(weigher) + .build(new CacheLoader() { @Override public KernelNodeState load(String key) { int slash = key.indexOf('/'); @@ -83,6 +89,7 @@ public class KernelNodeStore extends Abs String path = key.substring(slash); return new KernelNodeState(kernel, path, revision, cache); } + @Override public ListenableFuture reload( String key, KernelNodeState oldValue) { @@ -95,6 +102,8 @@ public class KernelNodeStore extends Abs } }); + cacheStats = new CacheStats(cache, "NodeStore", weigher, cacheSize); + try { this.root = cache.get(kernel.getHeadRevision() + '/'); } catch (Exception e) { @@ -162,6 +171,10 @@ public class KernelNodeStore extends Abs } } + public CacheStats getCacheStats(){ + return cacheStats; + } + //-----------------------------------------------------------< internal >--- @Nonnull Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/osgi/Activator.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/osgi/Activator.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/osgi/Activator.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/osgi/Activator.java Thu Jun 27 09:45:22 2013 @@ -25,6 +25,7 @@ import java.util.Properties; import org.apache.jackrabbit.mk.api.MicroKernel; import org.apache.jackrabbit.oak.Oak; import org.apache.jackrabbit.oak.api.ContentRepository; +import org.apache.jackrabbit.oak.api.jmx.CacheStatsMBean; import org.apache.jackrabbit.oak.core.ContentRepositoryImpl; import org.apache.jackrabbit.oak.kernel.KernelNodeStore; import org.apache.jackrabbit.oak.osgi.OsgiRepositoryInitializer.RepositoryInitializerObserver; @@ -32,6 +33,9 @@ import org.apache.jackrabbit.oak.securit import org.apache.jackrabbit.oak.spi.lifecycle.OakInitializer; import org.apache.jackrabbit.oak.spi.lifecycle.RepositoryInitializer; import org.apache.jackrabbit.oak.spi.state.NodeStore; +import org.apache.jackrabbit.oak.spi.whiteboard.OsgiWhiteboard; +import org.apache.jackrabbit.oak.spi.whiteboard.Registration; +import org.apache.jackrabbit.oak.spi.whiteboard.Whiteboard; import org.osgi.framework.BundleActivator; import org.osgi.framework.BundleContext; import org.osgi.framework.ServiceReference; @@ -39,13 +43,17 @@ import org.osgi.framework.ServiceRegistr import org.osgi.util.tracker.ServiceTracker; import org.osgi.util.tracker.ServiceTrackerCustomizer; +import static org.apache.jackrabbit.oak.spi.whiteboard.WhiteboardUtils.registerMBean; + public class Activator implements BundleActivator, ServiceTrackerCustomizer, RepositoryInitializerObserver { private BundleContext context; private ServiceTracker microKernelTracker; - // see OAK-795 for a reason why the nodeStore tracker is disabled + private Whiteboard whiteboard; + + // see OAK-795 for a reason why the nodeStore tracker is disabled // private ServiceTracker nodeStoreTracker; private final OsgiIndexProvider indexProvider = new OsgiIndexProvider(); @@ -58,12 +66,14 @@ public class Activator implements Bundle private final Map services = new HashMap(); + private final List registrations = new ArrayList(); + //----------------------------------------------------< BundleActivator >--- @Override public void start(BundleContext bundleContext) throws Exception { context = bundleContext; - + whiteboard = new OsgiWhiteboard(bundleContext); indexProvider.start(bundleContext); indexEditorProvider.start(bundleContext); validatorProvider.start(bundleContext); @@ -85,6 +95,10 @@ public class Activator implements Bundle indexEditorProvider.stop(); validatorProvider.stop(); repositoryInitializerTracker.stop(); + + for(Registration r : registrations){ + r.unregister(); + } } //-------------------------------------------< ServiceTrackerCustomizer >--- @@ -94,10 +108,13 @@ public class Activator implements Bundle Object service = context.getService(reference); if (service instanceof MicroKernel) { MicroKernel kernel = (MicroKernel) service; + KernelNodeStore store = new KernelNodeStore(kernel); services.put(reference, context.registerService( NodeStore.class.getName(), - new KernelNodeStore(kernel), + store, new Properties())); + registrations.add(registerMBean(whiteboard, CacheStatsMBean.class, + store.getCacheStats(), CacheStatsMBean.TYPE, store.getCacheStats().getName())); } else if (service instanceof NodeStore) { NodeStore store = (NodeStore) service; OakInitializer.initialize(store, repositoryInitializerTracker, indexEditorProvider); @@ -106,6 +123,7 @@ public class Activator implements Bundle .with(new SecurityProviderImpl()) .with(validatorProvider) .with(indexProvider) + .with(whiteboard) .with(indexEditorProvider); services.put(reference, context.registerService( ContentRepository.class.getName(), Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoDocumentStore.java Thu Jun 27 09:45:22 2013 @@ -30,6 +30,7 @@ import javax.annotation.Nonnull; import org.apache.jackrabbit.mk.api.MicroKernelException; import org.apache.jackrabbit.oak.plugins.mongomk.UpdateOp.Operation; import org.apache.jackrabbit.oak.plugins.mongomk.util.Utils; +import org.apache.jackrabbit.oak.cache.CacheStats; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -63,6 +64,8 @@ public class MongoDocumentStore implemen private long timeSum; private final Cache nodesCache; + + private final CacheStats cacheStats; public MongoDocumentStore(DB db, MongoMK.Builder builder) { nodes = db.getCollection( @@ -82,10 +85,12 @@ public class MongoDocumentStore implemen // TODO expire entries if the parent was changed nodesCache = CacheBuilder.newBuilder() .weigher(builder.getWeigher()) - //.recordStats() FIXME: OAK-863 + .recordStats() .maximumWeight(builder.getDocumentCacheSize()) .build(); - + + cacheStats = new CacheStats(nodesCache, "MongoMk-Documents", builder.getWeigher(), + builder.getDocumentCacheSize()); } private static long start() { @@ -446,7 +451,11 @@ public class MongoDocumentStore implemen } nodes.getDB().getMongo().close(); } - + + public CacheStats getCacheStats() { + return cacheStats; + } + private static void log(String message, Object... args) { if (LOG.isDebugEnabled()) { String argList = Arrays.toString(args); Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMK.java Thu Jun 27 09:45:22 2013 @@ -56,6 +56,7 @@ import org.apache.jackrabbit.oak.plugins import org.apache.jackrabbit.oak.plugins.mongomk.util.TimingDocumentStoreWrapper; import org.apache.jackrabbit.oak.plugins.mongomk.util.Utils; import org.apache.jackrabbit.oak.commons.PathUtils; +import org.apache.jackrabbit.oak.cache.CacheStats; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -136,6 +137,7 @@ public class MongoMK implements MicroKer * Key: path@rev, value: node */ private final Cache nodeCache; + private final CacheStats nodeCacheStats; /** * Child node cache. @@ -143,11 +145,12 @@ public class MongoMK implements MicroKer * Key: path@rev, value: children */ private final Cache nodeChildrenCache; - + private final CacheStats nodeChildrenCacheStats; /** * Diff cache. */ private final Cache diffCache; + private final CacheStats diffCacheStats; /** * The unsaved last revisions. This contains the parents of all changed @@ -191,7 +194,7 @@ public class MongoMK implements MicroKer private final UnmergedBranches branches; private boolean stopBackground; - + MongoMK(Builder builder) { DocumentStore s = builder.getDocumentStore(); if (builder.getTiming()) { @@ -222,20 +225,27 @@ public class MongoMK implements MicroKer nodeCache = CacheBuilder.newBuilder() .weigher(builder.getWeigher()) .maximumWeight(builder.getNodeCacheSize()) - // .recordStats() FIXME: OAK-863 + .recordStats() .build(); + nodeCacheStats = new CacheStats(nodeCache, "MongoMk-Node", + builder.getWeigher(), builder.getNodeCacheSize()); nodeChildrenCache = CacheBuilder.newBuilder() .weigher(builder.getWeigher()) - //.recordStats() FIXME: OAK-863 + .recordStats() .maximumWeight(builder.getChildrenCacheSize()) .build(); - + nodeChildrenCacheStats = new CacheStats(nodeChildrenCache, "MongoMk-NodeChildren", + builder.getWeigher(), builder.getNodeCacheSize()); + diffCache = CacheBuilder.newBuilder() + .recordStats() .weigher(builder.getWeigher()) .maximumWeight(builder.getDiffCacheSize()) .build(); - + diffCacheStats = new CacheStats(diffCache, "MongoMk-DiffCache", + builder.getWeigher(), builder.getNodeCacheSize()); + init(); // initial reading of the revisions of other cluster nodes backgroundRead(); @@ -243,7 +253,7 @@ public class MongoMK implements MicroKer headRevision = newRevision(); LOG.info("Initialized MongoMK with clusterNodeId: {}", clusterId); } - + void init() { headRevision = newRevision(); Node n = readNode("/", headRevision); @@ -1521,7 +1531,7 @@ public class MongoMK implements MicroKer nodeChildrenCache.put(key, c2); } } - + /** * A background thread. */ @@ -1555,6 +1565,18 @@ public class MongoMK implements MicroKer } } + public CacheStats getNodeCacheStats() { + return nodeCacheStats; + } + + public CacheStats getNodeChildrenCacheStats() { + return nodeChildrenCacheStats; + } + + public CacheStats getDiffCacheStats() { + return diffCacheStats; + } + /** * A builder for a MongoMK instance. */ @@ -1676,7 +1698,7 @@ public class MongoMK implements MicroKer return weigher; } - public Builder weigher(Weigher weigher) { + public Builder withWeigher(Weigher weigher) { this.weigher = weigher; return this; } Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMicroKernelService.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMicroKernelService.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMicroKernelService.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/mongomk/MongoMicroKernelService.java Thu Jun 27 09:45:22 2013 @@ -19,6 +19,8 @@ package org.apache.jackrabbit.oak.plugins.mongomk; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Properties; @@ -27,7 +29,11 @@ import org.apache.felix.scr.annotations. import org.apache.felix.scr.annotations.ConfigurationPolicy; import org.apache.felix.scr.annotations.Property; import org.apache.jackrabbit.mk.api.MicroKernel; +import org.apache.jackrabbit.oak.api.jmx.CacheStatsMBean; import org.apache.jackrabbit.oak.plugins.mongomk.util.MongoConnection; +import org.apache.jackrabbit.oak.spi.whiteboard.OsgiWhiteboard; +import org.apache.jackrabbit.oak.spi.whiteboard.Registration; +import org.apache.jackrabbit.oak.spi.whiteboard.Whiteboard; import org.apache.sling.commons.osgi.PropertiesUtil; import org.osgi.framework.BundleContext; import org.osgi.framework.ServiceRegistration; @@ -37,6 +43,8 @@ import org.slf4j.LoggerFactory; import com.mongodb.DB; +import static org.apache.jackrabbit.oak.spi.whiteboard.WhiteboardUtils.registerMBean; + /** * The OSGi service to start/stop a MongoMK instance. */ @@ -69,6 +77,7 @@ public class MongoMicroKernelService { private ServiceRegistration reg; private MongoMK mk; + private final List registrations = new ArrayList(); @Activate private void activate(BundleContext context, Map config) @@ -86,15 +95,59 @@ public class MongoMicroKernelService { logger.info("Connected to database {}", mongoDB); - mk = new MongoMK.Builder().memoryCacheSize(cacheSize * MB).setMongoDB(mongoDB).open(); + mk = new MongoMK.Builder() + .memoryCacheSize(cacheSize * MB) + .setMongoDB(mongoDB) + .open(); + + registerJMXBeans(mk, context); - Properties props = new Properties(); - props.setProperty("oak.mk.type", "mongo"); - reg = context.registerService(MicroKernel.class.getName(), mk, props); + reg = context.registerService(MicroKernel.class.getName(), mk, new Properties()); + } + + private void registerJMXBeans(MongoMK mk, BundleContext context) { + Whiteboard wb = new OsgiWhiteboard(context); + registrations.add( + registerMBean(wb, + CacheStatsMBean.class, + mk.getNodeCacheStats(), + CacheStatsMBean.TYPE, + mk.getNodeCacheStats().getName()) + ); + registrations.add( + registerMBean(wb, + CacheStatsMBean.class, + mk.getNodeChildrenCacheStats(), + CacheStatsMBean.TYPE, + mk.getNodeCacheStats().getName()) + ); + registrations.add( + registerMBean(wb, + CacheStatsMBean.class, + mk.getDiffCacheStats(), + CacheStatsMBean.TYPE, + mk.getDiffCacheStats().getName()) + ); + + DocumentStore ds = mk.getDocumentStore(); + if(ds instanceof MongoDocumentStore){ + MongoDocumentStore mds = (MongoDocumentStore) ds; + registrations.add( + registerMBean(wb, + CacheStatsMBean.class, + mds.getCacheStats(), + CacheStatsMBean.TYPE, + mds.getCacheStats().getName()) + ); + } } @Deactivate private void deactivate() { + for(Registration r : registrations){ + r.unregister(); + } + if (reg != null) { reg.unregister(); } Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentCache.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentCache.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentCache.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentCache.java Thu Jun 27 09:45:22 2013 @@ -22,6 +22,7 @@ import java.util.concurrent.ExecutionExc import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; +import org.apache.jackrabbit.oak.cache.CacheStats; /** * Combined memory and disk cache for segments. @@ -32,6 +33,8 @@ public class SegmentCache { private final Cache memoryCache; + private final CacheStats cacheStats; + // private final Cache diskCache; // private final File diskCacheDirectory; @@ -48,6 +51,7 @@ public class SegmentCache { // }).build(); this.memoryCache = CacheBuilder.newBuilder() .maximumWeight(memoryCacheSize) + .recordStats() .weigher(Segment.WEIGHER) // .removalListener(new RemovalListener() { // @Override @@ -57,6 +61,8 @@ public class SegmentCache { // } // }) .build(); + + cacheStats = new CacheStats(memoryCache, "Segment", Segment.WEIGHER, memoryCacheSize); } public SegmentCache() { @@ -80,4 +86,7 @@ public class SegmentCache { memoryCache.invalidate(segmentId); } + public CacheStats getCacheStats() { + return cacheStats; + } } Modified: jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java?rev=1497267&r1=1497266&r2=1497267&view=diff ============================================================================== --- jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java (original) +++ jackrabbit/oak/trunk/oak-core/src/main/java/org/apache/jackrabbit/oak/plugins/segment/SegmentNodeStoreService.java Thu Jun 27 09:45:22 2013 @@ -30,16 +30,21 @@ import org.apache.felix.scr.annotations. import org.apache.felix.scr.annotations.Property; import org.apache.felix.scr.annotations.Service; import org.apache.jackrabbit.oak.api.Blob; +import org.apache.jackrabbit.oak.api.jmx.CacheStatsMBean; import org.apache.jackrabbit.oak.plugins.segment.file.FileStore; import org.apache.jackrabbit.oak.plugins.segment.mongo.MongoStore; import org.apache.jackrabbit.oak.spi.state.AbstractNodeStore; import org.apache.jackrabbit.oak.spi.state.NodeState; import org.apache.jackrabbit.oak.spi.state.NodeStore; import org.apache.jackrabbit.oak.spi.state.NodeStoreBranch; +import org.apache.jackrabbit.oak.spi.whiteboard.OsgiWhiteboard; +import org.apache.jackrabbit.oak.spi.whiteboard.Registration; import org.osgi.service.component.ComponentContext; import com.mongodb.Mongo; +import static org.apache.jackrabbit.oak.spi.whiteboard.WhiteboardUtils.registerMBean; + @Component(policy = ConfigurationPolicy.REQUIRE) @Service(NodeStore.class) public class SegmentNodeStoreService extends AbstractNodeStore { @@ -72,6 +77,8 @@ public class SegmentNodeStoreService ext private NodeStore delegate; + private Registration cacheStatsReg; + private synchronized NodeStore getDelegate() { assert delegate != null : "service must be activated when used"; return delegate; @@ -95,7 +102,11 @@ public class SegmentNodeStoreService ext int cache = Integer.parseInt(String.valueOf(properties.get(CACHE))); mongo = new Mongo(host, port); - store = new MongoStore(mongo.getDB(db), cache * MB); + SegmentCache sc = new SegmentCache(cache * MB); + store = new MongoStore(mongo.getDB(db), sc); + + cacheStatsReg = registerMBean(new OsgiWhiteboard(context.getBundleContext()), CacheStatsMBean.class, + sc.getCacheStats(), CacheStatsMBean.TYPE, sc.getCacheStats().getName()); } delegate = new SegmentNodeStore(store); @@ -105,6 +116,10 @@ public class SegmentNodeStoreService ext public synchronized void deactivate() { delegate = null; + if(cacheStatsReg != null){ + cacheStatsReg.unregister(); + } + store.close(); if (mongo != null) { mongo.close();