HBASE-1022 Add storefile index size to hbase metrics; add store count also

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@720302 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Andrew Kyle Purtell 2008-11-24 21:49:25 +00:00
parent 2826eeeeef
commit 95d2837346
2 changed files with 22 additions and 2 deletions

View File

@ -652,21 +652,33 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
// Is this too expensive every three seconds getting a lock on onlineRegions
// and then per store carried? Can I make metrics be sloppier and avoid
// the synchronizations?
int stores = 0;
int storefiles = 0;
long memcacheSize = 0;
long storefileIndexSize = 0;
synchronized (this.onlineRegions) {
for (Map.Entry<Integer, HRegion> e: this.onlineRegions.entrySet()) {
HRegion r = e.getValue();
memcacheSize += r.memcacheSize.get();
synchronized(r.stores) {
stores += r.stores.size();
for(Map.Entry<Integer, HStore> ee: r.stores.entrySet()) {
storefiles += ee.getValue().getStorefilesCount();
HStore store = ee.getValue();
storefiles += store.getStorefilesCount();
try {
storefileIndexSize += store.getStorefilesIndexSize();
} catch (IOException ex) {
LOG.warn("error getting store file index size for " + store +
": " + StringUtils.stringifyException(ex));
}
}
}
}
}
this.metrics.stores.set(stores);
this.metrics.storefiles.set(storefiles);
this.metrics.memcacheSizeMB.set((int)(memcacheSize/(1024*1024)));
this.metrics.storefileIndexSizeMB.set((int)(storefileIndexSize/(1024*1024)));
}
/**

View File

@ -48,7 +48,12 @@ public class RegionServerMetrics implements Updater {
* Count of requests to the regionservers since last call to metrics update
*/
private final MetricsIntValue requests = new MetricsIntValue("requests");
/**
* Count of stores open on the regionserver.
*/
public final MetricsIntValue stores = new MetricsIntValue("stores");
/**
* Count of storefiles open on the regionserver.
*/
@ -86,6 +91,7 @@ public class RegionServerMetrics implements Updater {
*/
public void doUpdates(@SuppressWarnings("unused") MetricsContext unused) {
synchronized (this) {
this.stores.pushMetric(this.metricsRecord);
this.storefiles.pushMetric(this.metricsRecord);
this.storefileIndexSizeMB.pushMetric(this.metricsRecord);
this.memcacheSizeMB.pushMetric(this.metricsRecord);
@ -131,6 +137,8 @@ public class RegionServerMetrics implements Updater {
sb.append(this.requests.get()/seconds);
sb.append(", regions=");
sb.append(this.regions.get());
sb.append(", stores=");
sb.append(this.stores.get());
sb.append(", storefiles=");
sb.append(this.storefiles.get());
sb.append(", storefileIndexSize=");