diff --git hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java index 8c28534..5327022 100644 --- hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java +++ hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java @@ -847,7 +847,11 @@ public final class HConstants { public static final String STATUS_MULTICAST_PORT = "hbase.status.multicast.port"; public static final int DEFAULT_STATUS_MULTICAST_PORT = 60100; - + /** + * The period (in milliseconds) between computing region server point in time metrics + */ + public static final String REGIONSERVER_METRICS_PERIOD = "hbase.regionserver.metrics.period"; + public static final long DEFAULT_REGIONSERVER_METRICS_PERIOD = 5000; private HConstants() { // Can't be instantiated with this ctor. diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index f44ea43..d36cbab 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -17,11 +17,16 @@ */ package org.apache.hadoop.hbase.regionserver; +import java.util.Collection; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hbase.CompatibilitySingletonFactory; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HDFSBlocksDistribution; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.io.hfile.BlockCache; @@ -31,10 +36,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.metrics2.MetricsExecutor; -import java.util.Collection; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; - /** * Impl for exposing HRegionServer Information through Hadoop's metrics 2 system. */ @@ -44,8 +45,6 @@ class MetricsRegionServerWrapperImpl public static final Log LOG = LogFactory.getLog(MetricsRegionServerWrapperImpl.class); - public static final int PERIOD = 15; - private final HRegionServer regionServer; private BlockCache blockCache; @@ -69,14 +68,24 @@ class MetricsRegionServerWrapperImpl private CacheStats cacheStats; private ScheduledExecutorService executor; private Runnable runnable; + private long period; public MetricsRegionServerWrapperImpl(final HRegionServer regionServer) { this.regionServer = regionServer; initBlockCache(); + this.period = + regionServer.conf.getLong(HConstants.REGIONSERVER_METRICS_PERIOD, + HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD); + this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor(); this.runnable = new RegionServerMetricsWrapperRunnable(); - this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS); + this.executor.scheduleWithFixedDelay(this.runnable, this.period, this.period, + TimeUnit.MILLISECONDS); + + if (LOG.isInfoEnabled()) { + LOG.info("Computing regionserver metrics every " + this.period + " milliseconds"); + } } /** @@ -380,12 +389,12 @@ class MetricsRegionServerWrapperImpl // assume that it took PERIOD seconds to start the executor. // this is a guess but it's a pretty good one. if (lastRan == 0) { - lastRan = currentTime - (PERIOD*1000); + lastRan = currentTime - (period * 1000); } //If we've time traveled keep the last requests per second. - if ((currentTime - lastRan) > 10) { + if ((currentTime - lastRan) > 0) { long currentRequestCount = getTotalRequestCount(); requestsPerSecond = (currentRequestCount - lastRequestCount) / ((currentTime - lastRan) / 1000.0); lastRequestCount = currentRequestCount;