You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2011/08/31 15:46:02 UTC
svn commit: r1163620 - in /hbase/trunk: CHANGES.txt
src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java
Author: tedyu
Date: Wed Aug 31 13:46:01 2011
New Revision: 1163620
URL: http://svn.apache.org/viewvc?rev=1163620&view=rev
Log:
HBASE-4310 SlabCache metrics bugfix (Li Pi)
Modified:
hbase/trunk/CHANGES.txt
hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java
Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1163620&r1=1163619&r2=1163620&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Wed Aug 31 13:46:01 2011
@@ -228,6 +228,7 @@ Release 0.91.0 - Unreleased
non-distributed case (todd)
HBASE-4303 HRegionInfo.toString has bad quoting (todd)
HBASE-4307 race condition in CacheTestUtils (Li Pi)
+ HBASE-4310 SlabCache metrics bugfix (Li Pi)
IMPROVEMENTS
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java?rev=1163620&r1=1163619&r2=1163620&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/io/hfile/slab/SlabCache.java Wed Aug 31 13:46:01 2011
@@ -64,7 +64,8 @@ public class SlabCache implements SlabIt
long size;
private final CacheStats stats;
- final SlabStats slabstats;
+ final SlabStats requestStats;
+ final SlabStats successfullyCachedStats;
private final long avgBlockSize;
private static final long CACHE_FIXED_OVERHEAD = ClassSize.estimateBase(
SlabCache.class, false);
@@ -80,7 +81,9 @@ public class SlabCache implements SlabIt
this.avgBlockSize = avgBlockSize;
this.size = size;
this.stats = new CacheStats();
- this.slabstats = new SlabStats();
+ this.requestStats = new SlabStats();
+ this.successfullyCachedStats = new SlabStats();
+
backingStore = new ConcurrentHashMap<String, SingleSizeCache>();
sizer = new TreeMap<Integer, SingleSizeCache>();
this.scheduleThreadPool.scheduleAtFixedRate(new StatisticsThread(this),
@@ -191,12 +194,13 @@ public class SlabCache implements SlabIt
Entry<Integer, SingleSizeCache> scacheEntry = getHigherBlock(cachedItem
.getSerializedLength());
- this.slabstats.addin(cachedItem.getSerializedLength());
+ this.requestStats.addin(cachedItem.getSerializedLength());
if (scacheEntry == null) {
return; // we can't cache, something too big.
}
+ this.successfullyCachedStats.addin(cachedItem.getSerializedLength());
SingleSizeCache scache = scacheEntry.getValue();
scache.cacheBlock(blockName, cachedItem); // if this
// fails, due to
@@ -312,7 +316,10 @@ public class SlabCache implements SlabIt
@Override
public void run() {
- ourcache.slabstats.logStats(ourcache);
+ LOG.info("Request Stats");
+ ourcache.requestStats.logStats(ourcache);
+ LOG.info("Successfully Cached Stats");
+ ourcache.successfullyCachedStats.logStats(ourcache);
}
}
@@ -353,17 +360,19 @@ public class SlabCache implements SlabIt
SlabCache.LOG.info("Current heap size is: "
+ StringUtils.humanReadableInt(slabCache.heapSize()));
for (int i = 0; i < fineGrainedStats.length; i++) {
- double lowerbound = Math.pow(Math.E, (double) i / (double) multiplier
- - 0.5);
- double upperbound = Math.pow(Math.E, (double) i / (double) multiplier
- + 0.5);
-
- SlabCache.LOG.info("From "
- + StringUtils.humanReadableInt((long) lowerbound) + "- "
- + StringUtils.humanReadableInt((long) upperbound) + ": "
- + StringUtils.humanReadableInt(fineGrainedStats[i].get())
- + " requests");
+ double lowerbound = Math.pow(Math.E,
+ ((double) i / (double) multiplier) - 0.5);
+ double upperbound = Math.pow(Math.E,
+ ((double) i / (double) multiplier) + 0.5);
+
+ if (fineGrainedStats[i].get() > 0) {
+ SlabCache.LOG.info("From "
+ + StringUtils.humanReadableInt((long) lowerbound) + "- "
+ + StringUtils.humanReadableInt((long) upperbound) + ": "
+ + StringUtils.humanReadableInt(fineGrainedStats[i].get())
+ + " requests");
+ }
}
}
}