HBASE-15222 Use less contended classes for metrics

Summary:
Use less contended things for metrics.
For histogram which was the largest culprit we use FastLongHistogram
For atomic long where possible we now use counter.

Test Plan: unit tests

Reviewers:

Subscribers:

Differential Revision: https://reviews.facebook.net/D54381
This commit is contained in:
Elliott Clark 2016-02-18 09:54:05 -08:00
parent 20e14f449a
commit 630a65825e
40 changed files with 561 additions and 698 deletions

View File

@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.util;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicLongArray;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
@ -31,11 +30,20 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class FastLongHistogram {
/**
* Default number of bins.
*/
public static final int DEFAULT_NBINS = 255;
public static final double[] DEFAULT_QUANTILES =
new double[]{0.25, 0.5, 0.75, 0.90, 0.95, 0.98, 0.99, 0.999};
/**
* Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
*/
private static class Bins {
private final AtomicLongArray counts;
private final Counter[] counts;
// inclusive
private final long binsMin;
// exclusive
@ -43,6 +51,10 @@ public class FastLongHistogram {
private final long bins10XMax;
private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
private final AtomicLong max = new AtomicLong(0L);
private final Counter count = new Counter(0);
private final Counter total = new Counter(0);
// set to true when any of data has been inserted to the Bins. It is set after the counts are
// updated.
private final AtomicBoolean hasData = new AtomicBoolean(false);
@ -50,17 +62,18 @@ public class FastLongHistogram {
/**
* The constructor for creating a Bins without any prior data.
*/
public Bins() {
this.counts = new AtomicLongArray(4);
this.binsMin = 0L;
this.binsMax = Long.MAX_VALUE;
this.bins10XMax = Long.MAX_VALUE;
public Bins(int numBins) {
counts = createCounters(numBins + 3);
this.binsMin = 1L;
// These two numbers are total guesses
// and should be treated as highly suspect.
this.binsMax = 1000;
this.bins10XMax = binsMax * 10;
}
/**
* The constructor for creating a Bins with last Bins.
* @param last the last Bins instance.
* @param quantiles the quantiles for creating the bins of the histogram.
*/
public Bins(Bins last, int numOfBins, double minQ, double maxQ) {
long[] values = last.getQuantiles(new double[] { minQ, maxQ });
@ -72,28 +85,51 @@ public class FastLongHistogram {
this.binsMax = Math.max(binsMax, this.binsMin + numOfBins);
this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
this.counts = new AtomicLongArray(numOfBins + 3);
this.counts = createCounters(numOfBins + 3);
}
private Counter[] createCounters(int num) {
Counter[] counters = new Counter[num];
for (int i = 0; i < num; i++) {
counters[i] = new Counter();
}
return counters;
}
private int getIndex(long value) {
if (value < this.binsMin) {
return 0;
} else if (value > this.bins10XMax) {
return this.counts.length - 1;
} else if (value >= this.binsMax) {
return this.counts.length - 2;
}
// compute the position
return 1 + (int) ((value - this.binsMin) * (this.counts.length - 3) /
(this.binsMax - this.binsMin));
}
/**
* Adds a value to the histogram.
*/
public void add(long value, long count) {
if (value < 0) {
// The whole computation is completely thrown off if there are negative numbers
//
// Normally we would throw an IllegalArgumentException however this is the metrics
// system and it should be completely safe at all times.
// So silently throw it away.
return;
}
AtomicUtils.updateMin(min, value);
AtomicUtils.updateMax(max, value);
if (value < this.binsMin) {
this.counts.addAndGet(0, count);
} else if (value > this.bins10XMax) {
this.counts.addAndGet(this.counts.length() - 1, count);
} else if (value >= this.binsMax) {
this.counts.addAndGet(this.counts.length() - 2, count);
} else {
// compute the position
int pos =
1 + (int) ((value - this.binsMin) * (this.counts.length() - 3) / (this.binsMax - this.binsMin));
this.counts.addAndGet(pos, count);
}
this.count.add(count);
this.total.add(value * count);
int pos = getIndex(value);
this.counts[pos].add(count);
// hasData needs to be updated as last
this.hasData.set(true);
@ -101,7 +137,6 @@ public class FastLongHistogram {
/**
* Computes the quantiles give the ratios.
* @param smooth set to true to have a prior on the distribution. Used for recreating the bins.
*/
public long[] getQuantiles(double[] quantiles) {
if (!this.hasData.get()) {
@ -112,10 +147,10 @@ public class FastLongHistogram {
// Make a snapshot of lowerCounter, higherCounter and bins.counts to counts.
// This is not synchronized, but since the counter are accumulating, the result is a good
// estimation of a snapshot.
long[] counts = new long[this.counts.length()];
long[] counts = new long[this.counts.length];
long total = 0L;
for (int i = 0; i < this.counts.length(); i++) {
counts[i] = this.counts.get(i);
for (int i = 0; i < this.counts.length; i++) {
counts[i] = this.counts[i].get();
total += counts[i];
}
@ -137,8 +172,8 @@ public class FastLongHistogram {
mn = this.binsMax;
mx = this.bins10XMax;
} else {
mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length() - 3);
mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length() - 3);
mn = this.binsMin + (i - 1) * (this.binsMax - this.binsMin) / (this.counts.length - 3);
mx = this.binsMin + i * (this.binsMax - this.binsMin) / (this.counts.length - 3);
}
if (mx < this.min.get()) {
@ -177,12 +212,27 @@ public class FastLongHistogram {
return res;
}
long getNumAtOrBelow(long val) {
final int targetIndex = getIndex(val);
long totalToCurrentIndex = 0;
for (int i = 0; i <= targetIndex; i++) {
totalToCurrentIndex += this.counts[i].get();
}
return totalToCurrentIndex;
}
}
// The bins counting values. It is replaced with a new one in calling of reset().
private volatile Bins bins = new Bins();
// The quantiles for creating a Bins with last Bins.
private final int numOfBins;
private volatile Bins bins;
/**
* Constructor.
*/
public FastLongHistogram() {
this(DEFAULT_NBINS);
}
/**
* Constructor.
@ -190,7 +240,7 @@ public class FastLongHistogram {
* results but with lower efficiency, and vice versus.
*/
public FastLongHistogram(int numOfBins) {
this.numOfBins = numOfBins;
this.bins = new Bins(numOfBins);
}
/**
@ -202,10 +252,14 @@ public class FastLongHistogram {
*/
public FastLongHistogram(int numOfBins, long min, long max) {
this(numOfBins);
Bins bins = new Bins();
Bins bins = new Bins(numOfBins);
bins.add(min, 1);
bins.add(max, 1);
this.bins = new Bins(bins, numOfBins, 0.01, 0.99);
this.bins = new Bins(bins, numOfBins, 0.01, 0.999);
}
private FastLongHistogram(Bins bins) {
this.bins = bins;
}
/**
@ -222,12 +276,46 @@ public class FastLongHistogram {
return this.bins.getQuantiles(quantiles);
}
public long[] getQuantiles() {
return this.bins.getQuantiles(DEFAULT_QUANTILES);
}
public long getMin() {
return this.bins.min.get();
}
public long getMax() {
return this.bins.max.get();
}
public long getCount() {
return this.bins.count.get();
}
public long getMean() {
Bins bins = this.bins;
long count = bins.count.get();
long total = bins.total.get();
if (count == 0) {
return 0;
}
return total / count;
}
public long getNumAtOrBelow(long value) {
return this.bins.getNumAtOrBelow(value);
}
/**
* Resets the histogram for new counting.
*/
public void reset() {
public FastLongHistogram reset() {
if (this.bins.hasData.get()) {
this.bins = new Bins(this.bins, numOfBins, 0.01, 0.99);
Bins oldBins = this.bins;
this.bins = new Bins(this.bins, this.bins.counts.length - 3, 0.01, 0.99);
return new FastLongHistogram(oldBins);
}
return null;
}
}

View File

@ -26,6 +26,8 @@ import org.junit.Assert;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
/**
* Testcases for FastLongHistogram.
*/
@ -88,6 +90,36 @@ public class TestFastLongHistogram {
}
}
@Test
public void testGetNumAtOrBelow() {
long[] VALUES = { 1, 10, 20, 30, 40, 50 };
FastLongHistogram h = new FastLongHistogram();
for (long v : VALUES) {
for (int i = 0; i < 100; i++) {
h.add(v, 1);
}
}
h.add(Integer.MAX_VALUE, 1);
h.reset();
for (long v : VALUES) {
for (int i = 0; i < 100; i++) {
h.add(v, 1);
}
}
// Add something way out there to make sure it doesn't throw off the counts.
h.add(Integer.MAX_VALUE, 1);
assertEquals(100, h.getNumAtOrBelow(1));
assertEquals(200, h.getNumAtOrBelow(11));
assertEquals(601, h.getNumAtOrBelow(Long.MAX_VALUE));
}
@Test
public void testSameValues() {
FastLongHistogram hist = new FastLongHistogram(100);

View File

@ -79,14 +79,6 @@ public interface BaseSource {
void updateHistogram(String name, long value);
/**
* Add some value to a Quantile (An accurate histogram).
*
* @param name the name of the quantile
* @param value the value to add to the quantile
*/
void updateQuantile(String name, long value);
/**
* Get the metrics context. For hadoop metrics2 system this is usually an all lowercased string.
* eg. regionserver, master, thriftserver

View File

@ -30,10 +30,13 @@ public interface MetricHistogram {
String MAX_METRIC_NAME = "_max";
String MEAN_METRIC_NAME = "_mean";
String MEDIAN_METRIC_NAME = "_median";
String TWENTY_FIFTH_PERCENTILE_METRIC_NAME = "_25th_percentile";
String SEVENTY_FIFTH_PERCENTILE_METRIC_NAME = "_75th_percentile";
String NINETIETH_PERCENTILE_METRIC_NAME = "_90th_percentile";
String NINETY_FIFTH_PERCENTILE_METRIC_NAME = "_95th_percentile";
String NINETY_EIGHTH_PERCENTILE_METRIC_NAME = "_98th_percentile";
String NINETY_NINETH_PERCENTILE_METRIC_NAME = "_99th_percentile";
String NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME = "_99.9th_percentile";
/**
* Add a single value to a histogram's stream of values.

View File

@ -160,6 +160,10 @@ limitations under the License.
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-hadoop-compat</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-common</artifactId>
</dependency>
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-hadoop-compat</artifactId>
@ -181,10 +185,6 @@ limitations under the License.
<artifactId>hadoop-common</artifactId>
<version>${hadoop-two.version}</version>
</dependency>
<dependency>
<groupId>io.dropwizard.metrics</groupId>
<artifactId>metrics-core</artifactId>
</dependency>
<dependency>
<groupId>commons-lang</groupId>
<artifactId>commons-lang</artifactId>

View File

@ -21,11 +21,11 @@ package org.apache.hadoop.hbase.ipc;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
@InterfaceAudience.Private
public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
@ -33,29 +33,29 @@ public class MetricsHBaseServerSourceImpl extends BaseSourceImpl
private final MetricsHBaseServerWrapper wrapper;
private final MutableCounterLong authorizationSuccesses;
private final MutableCounterLong authorizationFailures;
private final MutableCounterLong authenticationSuccesses;
private final MutableCounterLong authenticationFailures;
private final MutableCounterLong authenticationFallbacks;
private final MutableCounterLong sentBytes;
private final MutableCounterLong receivedBytes;
private final MutableFastCounter authorizationSuccesses;
private final MutableFastCounter authorizationFailures;
private final MutableFastCounter authenticationSuccesses;
private final MutableFastCounter authenticationFailures;
private final MutableFastCounter authenticationFallbacks;
private final MutableFastCounter sentBytes;
private final MutableFastCounter receivedBytes;
private final MutableCounterLong exceptions;
private final MutableCounterLong exceptionsOOO;
private final MutableCounterLong exceptionsBusy;
private final MutableCounterLong exceptionsUnknown;
private final MutableCounterLong exceptionsSanity;
private final MutableCounterLong exceptionsNSRE;
private final MutableCounterLong exceptionsMoved;
private final MutableCounterLong exceptionsMultiTooLarge;
private final MutableFastCounter exceptions;
private final MutableFastCounter exceptionsOOO;
private final MutableFastCounter exceptionsBusy;
private final MutableFastCounter exceptionsUnknown;
private final MutableFastCounter exceptionsSanity;
private final MutableFastCounter exceptionsNSRE;
private final MutableFastCounter exceptionsMoved;
private final MutableFastCounter exceptionsMultiTooLarge;
private MutableHistogram queueCallTime;
private MutableHistogram processCallTime;
private MutableHistogram totalCallTime;
private MutableHistogram requestSize;
private MutableHistogram responseSize;
private MetricHistogram queueCallTime;
private MetricHistogram processCallTime;
private MetricHistogram totalCallTime;
private MetricHistogram requestSize;
private MetricHistogram responseSize;
public MetricsHBaseServerSourceImpl(String metricsName,
String metricsDescription,

View File

@ -20,17 +20,19 @@ package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
@InterfaceAudience.Private
public class MetricsAssignmentManagerSourceImpl extends BaseSourceImpl implements MetricsAssignmentManagerSource {
public class MetricsAssignmentManagerSourceImpl
extends BaseSourceImpl
implements MetricsAssignmentManagerSource {
private MutableGaugeLong ritGauge;
private MutableGaugeLong ritCountOverThresholdGauge;
private MutableGaugeLong ritOldestAgeGauge;
private MutableHistogram assignTimeHisto;
private MutableHistogram bulkAssignTimeHisto;
private MetricHistogram assignTimeHisto;
private MetricHistogram bulkAssignTimeHisto;
public MetricsAssignmentManagerSourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

View File

@ -20,15 +20,17 @@ package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.MetricHistogram;
@InterfaceAudience.Private
public class MetricsMasterFilesystemSourceImpl extends BaseSourceImpl implements MetricsMasterFileSystemSource {
public class MetricsMasterFilesystemSourceImpl
extends BaseSourceImpl
implements MetricsMasterFileSystemSource {
private MutableHistogram splitSizeHisto;
private MutableHistogram splitTimeHisto;
private MutableHistogram metaSplitTimeHisto;
private MutableHistogram metaSplitSizeHisto;
private MetricHistogram splitSizeHisto;
private MetricHistogram splitTimeHisto;
private MetricHistogram metaSplitTimeHisto;
private MetricHistogram metaSplitSizeHisto;
public MetricsMasterFilesystemSourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

View File

@ -23,7 +23,7 @@ import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
/**
* Hadoop2 implementation of MetricsMasterSource.
@ -35,7 +35,7 @@ public class MetricsMasterSourceImpl
extends BaseSourceImpl implements MetricsMasterSource {
private final MetricsMasterWrapper masterWrapper;
private MutableCounterLong clusterRequestsCounter;
private MutableFastCounter clusterRequestsCounter;
public MetricsMasterSourceImpl(MetricsMasterWrapper masterWrapper) {
this(METRICS_NAME,

View File

@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.master;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.MetricHistogram;
@InterfaceAudience.Private
public class MetricsSnapshotSourceImpl extends BaseSourceImpl implements MetricsSnapshotSource {
private MutableHistogram snapshotTimeHisto;
private MutableHistogram snapshotCloneTimeHisto;
private MutableHistogram snapshotRestoreTimeHisto;
private MetricHistogram snapshotTimeHisto;
private MetricHistogram snapshotCloneTimeHisto;
private MetricHistogram snapshotRestoreTimeHisto;
public MetricsSnapshotSourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

View File

@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.master.balancer;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
@InterfaceAudience.Private
public class MetricsBalancerSourceImpl extends BaseSourceImpl implements MetricsBalancerSource{
private MutableHistogram blanceClusterHisto;
private MutableCounterLong miscCount;
private MetricHistogram blanceClusterHisto;
private MutableFastCounter miscCount;
public MetricsBalancerSourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

View File

@ -24,8 +24,7 @@ import org.apache.hadoop.metrics2.MetricsSource;
import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
import org.apache.hadoop.metrics2.lib.MetricMutableQuantiles;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.source.JvmMetrics;
@ -88,7 +87,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param value the new value of the gauge.
*/
public void setGauge(String gaugeName, long value) {
MutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, value);
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, value);
gaugeInt.set(value);
}
@ -99,7 +98,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param delta The amount to increment the gauge by.
*/
public void incGauge(String gaugeName, long delta) {
MutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0l);
gaugeInt.incr(delta);
}
@ -110,7 +109,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param delta the ammount to subtract from a gauge value.
*/
public void decGauge(String gaugeName, long delta) {
MutableGaugeLong gaugeInt = metricsRegistry.getLongGauge(gaugeName, 0l);
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0l);
gaugeInt.decr(delta);
}
@ -121,7 +120,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param delta the ammount to increment
*/
public void incCounters(String key, long delta) {
MutableCounterLong counter = metricsRegistry.getLongCounter(key, 0l);
MutableFastCounter counter = metricsRegistry.getCounter(key, 0l);
counter.incr(delta);
}
@ -132,12 +131,6 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
histo.add(value);
}
@Override
public void updateQuantile(String name, long value) {
MetricMutableQuantiles histo = metricsRegistry.getQuantile(name);
histo.add(value);
}
/**
* Remove a named gauge.
*

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
/**
* Hadoop2 implementation of MetricsRegionServerSource.
@ -45,13 +45,13 @@ public class MetricsRegionServerSourceImpl
private final MetricHistogram replayHisto;
private final MetricHistogram scanNextHisto;
private final MutableCounterLong slowPut;
private final MutableCounterLong slowDelete;
private final MutableCounterLong slowGet;
private final MutableCounterLong slowIncrement;
private final MutableCounterLong slowAppend;
private final MutableCounterLong splitRequest;
private final MutableCounterLong splitSuccess;
private final MutableFastCounter slowPut;
private final MutableFastCounter slowDelete;
private final MutableFastCounter slowGet;
private final MutableFastCounter slowIncrement;
private final MutableFastCounter slowAppend;
private final MutableFastCounter splitRequest;
private final MutableFastCounter splitSuccess;
private final MetricHistogram splitTimeHisto;
private final MetricHistogram flushTimeHisto;

View File

@ -23,11 +23,11 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
@InterfaceAudience.Private
public class MetricsRegionSourceImpl implements MetricsRegionSource {
@ -53,12 +53,12 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
private final String regionAppendKey;
private final String regionScanNextKey;
private final MutableCounterLong regionPut;
private final MutableCounterLong regionDelete;
private final MutableCounterLong regionIncrement;
private final MutableCounterLong regionAppend;
private final MutableHistogram regionGet;
private final MutableHistogram regionScanNext;
private final MutableFastCounter regionPut;
private final MutableFastCounter regionDelete;
private final MutableFastCounter regionIncrement;
private final MutableFastCounter regionAppend;
private final MetricHistogram regionGet;
private final MetricHistogram regionScanNext;
private final int hashCode;
public MetricsRegionSourceImpl(MetricsRegionWrapper regionWrapper,
@ -80,16 +80,16 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
String suffix = "Count";
regionPutKey = regionNamePrefix + MetricsRegionServerSource.MUTATE_KEY + suffix;
regionPut = registry.getLongCounter(regionPutKey, 0L);
regionPut = registry.getCounter(regionPutKey, 0L);
regionDeleteKey = regionNamePrefix + MetricsRegionServerSource.DELETE_KEY + suffix;
regionDelete = registry.getLongCounter(regionDeleteKey, 0L);
regionDelete = registry.getCounter(regionDeleteKey, 0L);
regionIncrementKey = regionNamePrefix + MetricsRegionServerSource.INCREMENT_KEY + suffix;
regionIncrement = registry.getLongCounter(regionIncrementKey, 0L);
regionIncrement = registry.getCounter(regionIncrementKey, 0L);
regionAppendKey = regionNamePrefix + MetricsRegionServerSource.APPEND_KEY + suffix;
regionAppend = registry.getLongCounter(regionAppendKey, 0L);
regionAppend = registry.getCounter(regionAppendKey, 0L);
regionGetKey = regionNamePrefix + MetricsRegionServerSource.GET_KEY;
regionGet = registry.newTimeHistogram(regionGetKey);

View File

@ -21,7 +21,7 @@ package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
/**
@ -36,10 +36,10 @@ public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSo
private final MetricHistogram appendSizeHisto;
private final MetricHistogram appendTimeHisto;
private final MetricHistogram syncTimeHisto;
private final MutableCounterLong appendCount;
private final MutableCounterLong slowAppendCount;
private final MutableCounterLong logRollRequested;
private final MutableCounterLong lowReplicationLogRollRequested;
private final MutableFastCounter appendCount;
private final MutableFastCounter slowAppendCount;
private final MutableFastCounter logRollRequested;
private final MutableFastCounter lowReplicationLogRollRequested;
public MetricsWALSourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);

View File

@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.replication.regionserver;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
public class MetricsReplicationGlobalSourceSource implements MetricsReplicationSourceSource{
@ -26,38 +26,38 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
private final MutableGaugeLong ageOfLastShippedOpGauge;
private final MutableGaugeLong sizeOfLogQueueGauge;
private final MutableCounterLong logReadInEditsCounter;
private final MutableCounterLong logEditsFilteredCounter;
private final MutableCounterLong shippedBatchesCounter;
private final MutableCounterLong shippedOpsCounter;
private final MutableCounterLong shippedKBsCounter;
private final MutableCounterLong logReadInBytesCounter;
private final MutableCounterLong shippedHFilesCounter;
private final MutableFastCounter logReadInEditsCounter;
private final MutableFastCounter logEditsFilteredCounter;
private final MutableFastCounter shippedBatchesCounter;
private final MutableFastCounter shippedOpsCounter;
private final MutableFastCounter shippedKBsCounter;
private final MutableFastCounter logReadInBytesCounter;
private final MutableFastCounter shippedHFilesCounter;
private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
public MetricsReplicationGlobalSourceSource(MetricsReplicationSourceImpl rms) {
this.rms = rms;
ageOfLastShippedOpGauge = rms.getMetricsRegistry().getLongGauge(SOURCE_AGE_OF_LAST_SHIPPED_OP, 0L);
ageOfLastShippedOpGauge = rms.getMetricsRegistry().getGauge(SOURCE_AGE_OF_LAST_SHIPPED_OP, 0L);
sizeOfLogQueueGauge = rms.getMetricsRegistry().getLongGauge(SOURCE_SIZE_OF_LOG_QUEUE, 0L);
sizeOfLogQueueGauge = rms.getMetricsRegistry().getGauge(SOURCE_SIZE_OF_LOG_QUEUE, 0L);
shippedBatchesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_BATCHES, 0L);
shippedBatchesCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_BATCHES, 0L);
shippedOpsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_OPS, 0L);
shippedOpsCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_OPS, 0L);
shippedKBsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_KBS, 0L);
shippedKBsCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_KBS, 0L);
logReadInBytesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_READ_IN_BYTES, 0L);
logReadInBytesCounter = rms.getMetricsRegistry().getCounter(SOURCE_LOG_READ_IN_BYTES, 0L);
logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_READ_IN_EDITS, 0L);
logReadInEditsCounter = rms.getMetricsRegistry().getCounter(SOURCE_LOG_READ_IN_EDITS, 0L);
logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_LOG_EDITS_FILTERED, 0L);
logEditsFilteredCounter = rms.getMetricsRegistry().getCounter(SOURCE_LOG_EDITS_FILTERED, 0L);
shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(SOURCE_SHIPPED_HFILES, 0L);
shippedHFilesCounter = rms.getMetricsRegistry().getCounter(SOURCE_SHIPPED_HFILES, 0L);
sizeOfHFileRefsQueueGauge =
rms.getMetricsRegistry().getLongGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
rms.getMetricsRegistry().getGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
}
@Override public void setLastShippedAge(long age) {

View File

@ -18,21 +18,21 @@
package org.apache.hadoop.hbase.replication.regionserver;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
public class MetricsReplicationSinkSourceImpl implements MetricsReplicationSinkSource {
private final MutableGaugeLong ageGauge;
private final MutableCounterLong batchesCounter;
private final MutableCounterLong opsCounter;
private final MutableCounterLong hfilesCounter;
private final MutableFastCounter batchesCounter;
private final MutableFastCounter opsCounter;
private final MutableFastCounter hfilesCounter;
public MetricsReplicationSinkSourceImpl(MetricsReplicationSourceImpl rms) {
ageGauge = rms.getMetricsRegistry().getLongGauge(SINK_AGE_OF_LAST_APPLIED_OP, 0L);
batchesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_BATCHES, 0L);
opsCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_OPS, 0L);
hfilesCounter = rms.getMetricsRegistry().getLongCounter(SINK_APPLIED_HFILES, 0L);
ageGauge = rms.getMetricsRegistry().getGauge(SINK_AGE_OF_LAST_APPLIED_OP, 0L);
batchesCounter = rms.getMetricsRegistry().getCounter(SINK_APPLIED_BATCHES, 0L);
opsCounter = rms.getMetricsRegistry().getCounter(SINK_APPLIED_OPS, 0L);
hfilesCounter = rms.getMetricsRegistry().getCounter(SINK_APPLIED_HFILES, 0L);
}
@Override public void setLastAppliedOpAge(long age) {

View File

@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.replication.regionserver;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSourceSource {
@ -37,13 +37,13 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
private final MutableGaugeLong ageOfLastShippedOpGauge;
private final MutableGaugeLong sizeOfLogQueueGauge;
private final MutableCounterLong logReadInEditsCounter;
private final MutableCounterLong logEditsFilteredCounter;
private final MutableCounterLong shippedBatchesCounter;
private final MutableCounterLong shippedOpsCounter;
private final MutableCounterLong shippedKBsCounter;
private final MutableCounterLong logReadInBytesCounter;
private final MutableCounterLong shippedHFilesCounter;
private final MutableFastCounter logReadInEditsCounter;
private final MutableFastCounter logEditsFilteredCounter;
private final MutableFastCounter shippedBatchesCounter;
private final MutableFastCounter shippedOpsCounter;
private final MutableFastCounter shippedKBsCounter;
private final MutableFastCounter logReadInBytesCounter;
private final MutableFastCounter shippedHFilesCounter;
private final MutableGaugeLong sizeOfHFileRefsQueueGauge;
public MetricsReplicationSourceSourceImpl(MetricsReplicationSourceImpl rms, String id) {
@ -51,34 +51,34 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
this.id = id;
ageOfLastShippedOpKey = "source." + id + ".ageOfLastShippedOp";
ageOfLastShippedOpGauge = rms.getMetricsRegistry().getLongGauge(ageOfLastShippedOpKey, 0L);
ageOfLastShippedOpGauge = rms.getMetricsRegistry().getGauge(ageOfLastShippedOpKey, 0L);
sizeOfLogQueueKey = "source." + id + ".sizeOfLogQueue";
sizeOfLogQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfLogQueueKey, 0L);
sizeOfLogQueueGauge = rms.getMetricsRegistry().getGauge(sizeOfLogQueueKey, 0L);
shippedBatchesKey = "source." + this.id + ".shippedBatches";
shippedBatchesCounter = rms.getMetricsRegistry().getLongCounter(shippedBatchesKey, 0L);
shippedBatchesCounter = rms.getMetricsRegistry().getCounter(shippedBatchesKey, 0L);
shippedOpsKey = "source." + this.id + ".shippedOps";
shippedOpsCounter = rms.getMetricsRegistry().getLongCounter(shippedOpsKey, 0L);
shippedOpsCounter = rms.getMetricsRegistry().getCounter(shippedOpsKey, 0L);
shippedKBsKey = "source." + this.id + ".shippedKBs";
shippedKBsCounter = rms.getMetricsRegistry().getLongCounter(shippedKBsKey, 0L);
shippedKBsCounter = rms.getMetricsRegistry().getCounter(shippedKBsKey, 0L);
logReadInBytesKey = "source." + this.id + ".logReadInBytes";
logReadInBytesCounter = rms.getMetricsRegistry().getLongCounter(logReadInBytesKey, 0L);
logReadInBytesCounter = rms.getMetricsRegistry().getCounter(logReadInBytesKey, 0L);
logReadInEditsKey = "source." + id + ".logEditsRead";
logReadInEditsCounter = rms.getMetricsRegistry().getLongCounter(logReadInEditsKey, 0L);
logReadInEditsCounter = rms.getMetricsRegistry().getCounter(logReadInEditsKey, 0L);
logEditsFilteredKey = "source." + id + ".logEditsFiltered";
logEditsFilteredCounter = rms.getMetricsRegistry().getLongCounter(logEditsFilteredKey, 0L);
logEditsFilteredCounter = rms.getMetricsRegistry().getCounter(logEditsFilteredKey, 0L);
shippedHFilesKey = "source." + this.id + ".shippedHFiles";
shippedHFilesCounter = rms.getMetricsRegistry().getLongCounter(shippedHFilesKey, 0L);
shippedHFilesCounter = rms.getMetricsRegistry().getCounter(shippedHFilesKey, 0L);
sizeOfHFileRefsQueueKey = "source." + id + ".sizeOfHFileRefsQueue";
sizeOfHFileRefsQueueGauge = rms.getMetricsRegistry().getLongGauge(sizeOfHFileRefsQueueKey, 0L);
sizeOfHFileRefsQueueGauge = rms.getMetricsRegistry().getGauge(sizeOfHFileRefsQueueKey, 0L);
}
@Override public void setLastShippedAge(long age) {

View File

@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.rest;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
/**
* Hadoop Two implementation of a metrics2 source that will export metrics from the Rest server to
@ -31,15 +31,15 @@ import org.apache.hadoop.metrics2.lib.MutableCounterLong;
@InterfaceAudience.Private
public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsRESTSource {
private MutableCounterLong request;
private MutableCounterLong sucGet;
private MutableCounterLong sucPut;
private MutableCounterLong sucDel;
private MutableCounterLong sucScan;
private MutableCounterLong fGet;
private MutableCounterLong fPut;
private MutableCounterLong fDel;
private MutableCounterLong fScan;
private MutableFastCounter request;
private MutableFastCounter sucGet;
private MutableFastCounter sucPut;
private MutableFastCounter sucDel;
private MutableFastCounter sucScan;
private MutableFastCounter fGet;
private MutableFastCounter fPut;
private MutableFastCounter fDel;
private MutableFastCounter fScan;
public MetricsRESTSourceImpl() {
this(METRICS_NAME, METRICS_DESCRIPTION, CONTEXT, JMX_CONTEXT);
@ -55,17 +55,17 @@ public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsREST
@Override
public void init() {
super.init();
request = getMetricsRegistry().getLongCounter(REQUEST_KEY, 0l);
request = getMetricsRegistry().getCounter(REQUEST_KEY, 0l);
sucGet = getMetricsRegistry().getLongCounter(SUCCESSFUL_GET_KEY, 0l);
sucPut = getMetricsRegistry().getLongCounter(SUCCESSFUL_PUT_KEY, 0l);
sucDel = getMetricsRegistry().getLongCounter(SUCCESSFUL_DELETE_KEY, 0l);
sucScan = getMetricsRegistry().getLongCounter(SUCCESSFUL_SCAN_KEY, 0L);
sucGet = getMetricsRegistry().getCounter(SUCCESSFUL_GET_KEY, 0l);
sucPut = getMetricsRegistry().getCounter(SUCCESSFUL_PUT_KEY, 0l);
sucDel = getMetricsRegistry().getCounter(SUCCESSFUL_DELETE_KEY, 0l);
sucScan = getMetricsRegistry().getCounter(SUCCESSFUL_SCAN_KEY, 0L);
fGet = getMetricsRegistry().getLongCounter(FAILED_GET_KEY, 0l);
fPut = getMetricsRegistry().getLongCounter(FAILED_PUT_KEY, 0l);
fDel = getMetricsRegistry().getLongCounter(FAILED_DELETE_KEY, 0l);
fScan = getMetricsRegistry().getLongCounter(FAILED_SCAN_KEY, 0l);
fGet = getMetricsRegistry().getCounter(FAILED_GET_KEY, 0l);
fPut = getMetricsRegistry().getCounter(FAILED_PUT_KEY, 0l);
fDel = getMetricsRegistry().getCounter(FAILED_DELETE_KEY, 0l);
fScan = getMetricsRegistry().getCounter(FAILED_SCAN_KEY, 0l);
}
@Override

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.thrift;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
@ -32,12 +33,12 @@ import org.apache.hadoop.metrics2.lib.MutableHistogram;
public class MetricsThriftServerSourceImpl extends BaseSourceImpl implements
MetricsThriftServerSource {
private MutableHistogram batchGetStat;
private MutableHistogram batchMutateStat;
private MutableHistogram queueTimeStat;
private MetricHistogram batchGetStat;
private MetricHistogram batchMutateStat;
private MetricHistogram queueTimeStat;
private MutableHistogram thriftCallStat;
private MutableHistogram thriftSlowCallStat;
private MetricHistogram thriftCallStat;
private MetricHistogram thriftSlowCallStat;
private MutableGaugeLong callQueueLenGauge;
@ -56,7 +57,7 @@ public class MetricsThriftServerSourceImpl extends BaseSourceImpl implements
queueTimeStat = getMetricsRegistry().newTimeHistogram(TIME_IN_QUEUE_KEY);
thriftCallStat = getMetricsRegistry().newTimeHistogram(THRIFT_CALL_KEY);
thriftSlowCallStat = getMetricsRegistry().newTimeHistogram(SLOW_THRIFT_CALL_KEY);
callQueueLenGauge = getMetricsRegistry().getLongGauge(CALL_QUEUE_LEN_KEY, 0);
callQueueLenGauge = getMetricsRegistry().getGauge(CALL_QUEUE_LEN_KEY, 0);
}

View File

@ -105,28 +105,6 @@ public class DynamicMetricsRegistry {
return tagsMap.get(name);
}
/**
* Create a mutable integer counter
* @param name of the metric
* @param desc metric description
* @param iVal initial value
* @return a new counter object
*/
public MutableCounterInt newCounter(String name, String desc, int iVal) {
return newCounter(new MetricsInfoImpl(name, desc), iVal);
}
/**
* Create a mutable integer counter
* @param info metadata of the metric
* @param iVal initial value
* @return a new counter object
*/
public MutableCounterInt newCounter(MetricsInfo info, int iVal) {
MutableCounterInt ret = new MutableCounterInt(info, iVal);
return addNewMetricIfAbsent(info.name(), ret, MutableCounterInt.class);
}
/**
* Create a mutable long integer counter
* @param name of the metric
@ -134,7 +112,7 @@ public class DynamicMetricsRegistry {
* @param iVal initial value
* @return a new counter object
*/
public MutableCounterLong newCounter(String name, String desc, long iVal) {
public MutableFastCounter newCounter(String name, String desc, long iVal) {
return newCounter(new MetricsInfoImpl(name, desc), iVal);
}
@ -144,30 +122,9 @@ public class DynamicMetricsRegistry {
* @param iVal initial value
* @return a new counter object
*/
public MutableCounterLong newCounter(MetricsInfo info, long iVal) {
MutableCounterLong ret = new MutableCounterLong(info, iVal);
return addNewMetricIfAbsent(info.name(), ret, MutableCounterLong.class);
}
/**
* Create a mutable integer gauge
* @param name of the metric
* @param desc metric description
* @param iVal initial value
* @return a new gauge object
*/
public MutableGaugeInt newGauge(String name, String desc, int iVal) {
return newGauge(new MetricsInfoImpl(name, desc), iVal);
}
/**
* Create a mutable integer gauge
* @param info metadata of the metric
* @param iVal initial value
* @return a new gauge object
*/
public MutableGaugeInt newGauge(MetricsInfo info, int iVal) {
MutableGaugeInt ret = new MutableGaugeInt(info, iVal);
return addNewMetricIfAbsent(info.name(), ret, MutableGaugeInt.class);
public MutableFastCounter newCounter(MetricsInfo info, long iVal) {
MutableFastCounter ret = new MutableFastCounter(info, iVal);
return addNewMetricIfAbsent(info.name(), ret, MutableFastCounter.class);
}
/**
@ -326,19 +283,6 @@ public class DynamicMetricsRegistry {
return addNewMetricIfAbsent(name, histo, MutableSizeHistogram.class);
}
/**
* Create a new MutableQuantile(A more accurate histogram).
* @param name The name of the histogram
* @return a new MutableQuantile
*/
public MetricMutableQuantiles newQuantile(String name) {
return newQuantile(name, "");
}
public MetricMutableQuantiles newQuantile(String name, String desc) {
MetricMutableQuantiles histo = new MetricMutableQuantiles(name, desc, "Ops", "", 60);
return addNewMetricIfAbsent(name, histo, MetricMutableQuantiles.class);
}
synchronized void add(String name, MutableMetric metric) {
addNewMetricIfAbsent(name, metric, MutableMetric.class);
@ -475,7 +419,7 @@ public class DynamicMetricsRegistry {
* @param gaugeName name of the gauge to create or get.
* @param potentialStartingValue value of the new gauge if we have to create it.
*/
public MutableGaugeLong getLongGauge(String gaugeName, long potentialStartingValue) {
public MutableGaugeLong getGauge(String gaugeName, long potentialStartingValue) {
//Try and get the guage.
MutableMetric metric = metricsMap.get(gaugeName);
@ -510,12 +454,12 @@ public class DynamicMetricsRegistry {
* @param counterName Name of the counter to get
* @param potentialStartingValue starting value if we have to create a new counter
*/
public MutableCounterLong getLongCounter(String counterName, long potentialStartingValue) {
//See getLongGauge for description on how this works.
public MutableFastCounter getCounter(String counterName, long potentialStartingValue) {
//See getGauge for description on how this works.
MutableMetric counter = metricsMap.get(counterName);
if (counter == null) {
MutableCounterLong newCounter =
new MutableCounterLong(new MetricsInfoImpl(counterName, ""), potentialStartingValue);
MutableFastCounter newCounter =
new MutableFastCounter(new MetricsInfoImpl(counterName, ""), potentialStartingValue);
counter = metricsMap.putIfAbsent(counterName, newCounter);
if (counter == null) {
return newCounter;
@ -523,16 +467,16 @@ public class DynamicMetricsRegistry {
}
if (!(counter instanceof MutableCounterLong)) {
if (!(counter instanceof MutableCounter)) {
throw new MetricsException("Metric already exists in registry for metric name: " +
counterName + " and not of type MetricMutableCounterLong");
counterName + " and not of type MutableCounter");
}
return (MutableCounterLong) counter;
return (MutableFastCounter) counter;
}
public MutableHistogram getHistogram(String histoName) {
//See getLongGauge for description on how this works.
//See getGauge for description on how this works.
MutableMetric histo = metricsMap.get(histoName);
if (histo == null) {
MutableHistogram newCounter =
@ -552,27 +496,6 @@ public class DynamicMetricsRegistry {
return (MutableHistogram) histo;
}
public MetricMutableQuantiles getQuantile(String histoName) {
//See getLongGauge for description on how this works.
MutableMetric histo = metricsMap.get(histoName);
if (histo == null) {
MetricMutableQuantiles newCounter =
new MetricMutableQuantiles(histoName, "", "Ops", "", 60);
histo = metricsMap.putIfAbsent(histoName, newCounter);
if (histo == null) {
return newCounter;
}
}
if (!(histo instanceof MetricMutableQuantiles)) {
throw new MetricsException("Metric already exists in registry for metric name: " +
histoName + " and not of type MutableHistogram");
}
return (MetricMutableQuantiles) histo;
}
private<T extends MutableMetric> T
addNewMetricIfAbsent(String name,
T ret,

View File

@ -1,154 +0,0 @@
/**
* 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.hadoop.metrics2.lib;
import static org.apache.hadoop.metrics2.lib.Interns.info;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.util.MetricQuantile;
import org.apache.hadoop.metrics2.util.MetricSampleQuantiles;
import com.google.common.annotations.VisibleForTesting;
/**
* Watches a stream of long values, maintaining online estimates of specific quantiles with provably
* low error bounds. This is particularly useful for accurate high-percentile (e.g. 95th, 99th)
* latency metrics.
*/
@InterfaceAudience.Private
public class MetricMutableQuantiles extends MutableMetric implements MetricHistogram {
static final MetricQuantile[] quantiles = {new MetricQuantile(0.50, 0.050),
new MetricQuantile(0.75, 0.025), new MetricQuantile(0.90, 0.010),
new MetricQuantile(0.95, 0.005), new MetricQuantile(0.99, 0.001)};
private final MetricsInfo numInfo;
private final MetricsInfo[] quantileInfos;
private final int interval;
private MetricSampleQuantiles estimator;
private long previousCount = 0;
private MetricsExecutor executor;
@VisibleForTesting
protected Map<MetricQuantile, Long> previousSnapshot = null;
/**
* Instantiates a new {@link MetricMutableQuantiles} for a metric that rolls itself over on the
* specified time interval.
*
* @param name of the metric
* @param description long-form textual description of the metric
* @param sampleName type of items in the stream (e.g., "Ops")
* @param valueName type of the values
* @param interval rollover interval (in seconds) of the estimator
*/
public MetricMutableQuantiles(String name, String description, String sampleName,
String valueName, int interval) {
String ucName = StringUtils.capitalize(name);
String usName = StringUtils.capitalize(sampleName);
String uvName = StringUtils.capitalize(valueName);
String desc = StringUtils.uncapitalize(description);
String lsName = StringUtils.uncapitalize(sampleName);
String lvName = StringUtils.uncapitalize(valueName);
numInfo = info(ucName + "Num" + usName, String.format(
"Number of %s for %s with %ds interval", lsName, desc, interval));
// Construct the MetricsInfos for the quantiles, converting to percentiles
quantileInfos = new MetricsInfo[quantiles.length];
String nameTemplate = "%s%dthPercentile%dsInterval%s";
String descTemplate = "%d percentile %s with %d second interval for %s";
for (int i = 0; i < quantiles.length; i++) {
int percentile = (int) (100 * quantiles[i].quantile);
quantileInfos[i] = info(String.format(nameTemplate, ucName, percentile, interval, uvName),
String.format(descTemplate, percentile, lvName, interval, desc));
}
estimator = new MetricSampleQuantiles(quantiles);
executor = new MetricsExecutorImpl();
this.interval = interval;
executor.getExecutor().scheduleAtFixedRate(new RolloverSample(this),
interval,
interval,
TimeUnit.SECONDS);
}
@Override
public synchronized void snapshot(MetricsRecordBuilder builder, boolean all) {
if (all || changed()) {
builder.addGauge(numInfo, previousCount);
for (int i = 0; i < quantiles.length; i++) {
long newValue = 0;
// If snapshot is null, we failed to update since the window was empty
if (previousSnapshot != null) {
newValue = previousSnapshot.get(quantiles[i]);
}
builder.addGauge(quantileInfos[i], newValue);
}
if (changed()) {
clearChanged();
}
}
}
public synchronized void add(long value) {
estimator.insert(value);
}
public int getInterval() {
return interval;
}
/** Runnable used to periodically roll over the internal {@link org.apache.hadoop.metrics2.util.MetricSampleQuantiles} every interval. */
private static class RolloverSample implements Runnable {
MetricMutableQuantiles parent;
public RolloverSample(MetricMutableQuantiles parent) {
this.parent = parent;
}
@Override
public void run() {
synchronized (parent) {
try {
parent.previousCount = parent.estimator.getCount();
parent.previousSnapshot = parent.estimator.snapshot();
} catch (IOException e) {
// Couldn't get a new snapshot because the window was empty
parent.previousCount = 0;
parent.previousSnapshot = null;
}
parent.estimator.clear();
}
parent.setChanged();
}
}
}

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.metrics2.MetricsExecutor;
/**
* Class to handle the ScheduledExecutorService{@link ScheduledExecutorService} used by
* MetricMutableQuantiles{@link MetricMutableQuantiles}, MetricsRegionAggregateSourceImpl, and
* MetricsRegionAggregateSourceImpl, and
* JmxCacheBuster
*/
@InterfaceAudience.Private

View File

@ -0,0 +1,60 @@
/**
* 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.hadoop.metrics2.lib;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
public class MutableFastCounter extends MutableCounter {
private final Counter counter;
protected MutableFastCounter(MetricsInfo info, long iVal) {
super(info);
counter = new Counter(iVal);
}
@Override
public void incr() {
counter.increment();
setChanged();
}
/**
* Increment the value by a delta
* @param delta of the increment
*/
public void incr(long delta) {
counter.add(delta);
setChanged();
}
@Override
public void snapshot(MetricsRecordBuilder builder, boolean all) {
if (all || changed()) {
builder.addCounter(info(), value());
clearChanged();
}
}
public long value() {
return counter.get();
}
}

View File

@ -18,124 +18,79 @@
package org.apache.hadoop.metrics2.lib;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.FastLongHistogram;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import com.codahale.metrics.ExponentiallyDecayingReservoir;
import com.codahale.metrics.Reservoir;
import com.codahale.metrics.Snapshot;
/**
* A histogram implementation that runs in constant space, and exports to hadoop2's metrics2 system.
*/
@InterfaceAudience.Private
public class MutableHistogram extends MutableMetric implements MetricHistogram {
private static final int DEFAULT_SAMPLE_SIZE = 2046;
// the bias towards sampling from more recent data.
// Per Cormode et al. an alpha of 0.015 strongly biases to the last 5 minutes
private static final double DEFAULT_ALPHA = 0.015;
// Double buffer the two FastLongHistograms.
// As they are reset they learn how the buckets should be spaced
// So keep two around and use them
protected final FastLongHistogram histogram;
protected final String name;
protected final String desc;
private final Reservoir reservoir;
private final AtomicLong min;
private final AtomicLong max;
private final AtomicLong sum;
private final AtomicLong count;
protected final Counter counter = new Counter(0);
public MutableHistogram(MetricsInfo info) {
this(info.name(), info.description());
}
public MutableHistogram(String name, String description) {
this(name, description, Integer.MAX_VALUE << 2);
}
protected MutableHistogram(String name, String description, long maxExpected) {
this.name = StringUtils.capitalize(name);
this.desc = StringUtils.uncapitalize(description);
reservoir = new ExponentiallyDecayingReservoir(DEFAULT_SAMPLE_SIZE, DEFAULT_ALPHA);
count = new AtomicLong();
min = new AtomicLong(Long.MAX_VALUE);
max = new AtomicLong(Long.MIN_VALUE);
sum = new AtomicLong();
this.histogram = new FastLongHistogram(FastLongHistogram.DEFAULT_NBINS, 1, maxExpected);
}
public void add(final long val) {
setChanged();
count.incrementAndGet();
reservoir.update(val);
setMax(val);
setMin(val);
sum.getAndAdd(val);
}
private void setMax(final long potentialMax) {
boolean done = false;
while (!done) {
final long currentMax = max.get();
done = currentMax >= potentialMax
|| max.compareAndSet(currentMax, potentialMax);
}
}
private void setMin(long potentialMin) {
boolean done = false;
while (!done) {
final long currentMin = min.get();
done = currentMin <= potentialMin
|| min.compareAndSet(currentMin, potentialMin);
}
}
public long getMax() {
if (count.get() > 0) {
return max.get();
}
return 0L;
}
public long getMin() {
if (count.get() > 0) {
return min.get();
}
return 0L;
}
public double getMean() {
long cCount = count.get();
if (cCount > 0) {
return sum.get() / (double) cCount;
}
return 0.0;
counter.increment();
histogram.add(val, 1);
}
@Override
public void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
if (all || changed()) {
clearChanged();
updateSnapshotMetrics(metricsRecordBuilder);
}
public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
// Get a reference to the old histogram.
FastLongHistogram histo = histogram.reset();
updateSnapshotMetrics(metricsRecordBuilder, histo);
}
public void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder) {
final Snapshot s = reservoir.getSnapshot();
metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc), count.get());
protected void updateSnapshotMetrics(MetricsRecordBuilder metricsRecordBuilder,
FastLongHistogram histo) {
metricsRecordBuilder.addCounter(Interns.info(name + NUM_OPS_METRIC_NAME, desc), counter.get());
metricsRecordBuilder.addGauge(Interns.info(name + MIN_METRIC_NAME, desc), histo.getMin());
metricsRecordBuilder.addGauge(Interns.info(name + MAX_METRIC_NAME, desc), histo.getMax());
metricsRecordBuilder.addGauge(Interns.info(name + MEAN_METRIC_NAME, desc), histo.getMean());
metricsRecordBuilder.addGauge(Interns.info(name + MIN_METRIC_NAME, desc), getMin());
metricsRecordBuilder.addGauge(Interns.info(name + MAX_METRIC_NAME, desc), getMax());
metricsRecordBuilder.addGauge(Interns.info(name + MEAN_METRIC_NAME, desc), getMean());
long[] percentiles = histo.getQuantiles();
metricsRecordBuilder.addGauge(Interns.info(name + MEDIAN_METRIC_NAME, desc), s.getMedian());
metricsRecordBuilder.addGauge(Interns.info(name + TWENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
percentiles[0]);
metricsRecordBuilder.addGauge(Interns.info(name + MEDIAN_METRIC_NAME, desc),
percentiles[1]);
metricsRecordBuilder.addGauge(Interns.info(name + SEVENTY_FIFTH_PERCENTILE_METRIC_NAME, desc),
s.get75thPercentile());
percentiles[2]);
metricsRecordBuilder.addGauge(Interns.info(name + NINETIETH_PERCENTILE_METRIC_NAME, desc),
s.getValue(0.90));
percentiles[3]);
metricsRecordBuilder.addGauge(Interns.info(name + NINETY_FIFTH_PERCENTILE_METRIC_NAME, desc),
s.get95thPercentile());
percentiles[4]);
metricsRecordBuilder.addGauge(Interns.info(name + NINETY_EIGHTH_PERCENTILE_METRIC_NAME, desc),
percentiles[5]);
metricsRecordBuilder.addGauge(Interns.info(name + NINETY_NINETH_PERCENTILE_METRIC_NAME, desc),
s.get99thPercentile());
percentiles[6]);
metricsRecordBuilder.addGauge(
Interns.info(name + NINETY_NINE_POINT_NINETH_PERCENTILE_METRIC_NAME, desc),
percentiles[7]);
}
}

View File

@ -18,24 +18,27 @@
package org.apache.hadoop.metrics2.lib;
import java.util.concurrent.atomic.AtomicLongArray;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.FastLongHistogram;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
/**
* Extended histogram implementation with metric range counters.
*/
@InterfaceAudience.Private
public abstract class MutableRangeHistogram extends MutableHistogram {
public abstract class MutableRangeHistogram extends MutableHistogram implements MetricHistogram {
public MutableRangeHistogram(MetricsInfo info) {
this(info.name(), info.description());
}
public MutableRangeHistogram(String name, String description) {
super(name, description);
this(name, description, Integer.MAX_VALUE << 2);
}
public MutableRangeHistogram(String name, String description, long expectedMax) {
super(name, description, expectedMax);
}
/**
@ -46,49 +49,39 @@ public abstract class MutableRangeHistogram extends MutableHistogram {
/**
* Returns the ranges to be counted
*/
public abstract long[] getRange();
public abstract long[] getRanges();
/**
* Returns the range counts
*/
public abstract AtomicLongArray getRangeVals();
@Override
public void add(final long val) {
super.add(val);
updateBand(val);
public synchronized void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
// Get a reference to the old histogram.
FastLongHistogram histo = histogram.reset();
updateSnapshotMetrics(metricsRecordBuilder, histo);
updateSnapshotRangeMetrics(metricsRecordBuilder, histo);
}
private void updateBand(final long val) {
int i;
for (i=0; i<getRange().length && val > getRange()[i]; i++);
getRangeVals().incrementAndGet(i);
}
public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder,
FastLongHistogram histogram) {
long priorRange = 0;
long cumNum = 0;
@Override
public void snapshot(MetricsRecordBuilder metricsRecordBuilder, boolean all) {
if (all || changed()) {
clearChanged();
updateSnapshotMetrics(metricsRecordBuilder);
updateSnapshotRangeMetrics(metricsRecordBuilder);
}
}
public void updateSnapshotRangeMetrics(MetricsRecordBuilder metricsRecordBuilder) {
long prior = 0;
for (int i = 0; i < getRange().length; i++) {
long val = getRangeVals().get(i);
if (val > 0) {
final long[] ranges = getRanges();
final String rangeType = getRangeType();
for (int i = 0; i < ranges.length - 1; i++) {
long val = histogram.getNumAtOrBelow(ranges[i]);
if (val - cumNum > 0) {
metricsRecordBuilder.addCounter(
Interns.info(name + "_" + getRangeType() + "_" + prior + "-" + getRange()[i], desc), val);
Interns.info(name + "_" + rangeType + "_" + priorRange + "-" + ranges[i], desc),
val - cumNum);
}
prior = getRange()[i];
priorRange = ranges[i];
cumNum = val;
}
long val = getRangeVals().get(getRange().length);
if (val > 0) {
long val = histogram.getCount();
if (val - cumNum > 0) {
metricsRecordBuilder.addCounter(
Interns.info(name + "_" + getRangeType() + "_" + getRange()[getRange().length - 1] + "-inf", desc),
getRangeVals().get(getRange().length));
Interns.info(name + "_" + rangeType + "_" + ranges[ranges.length - 1] + "-inf", desc),
val - cumNum);
}
}
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.metrics2.lib;
import java.util.concurrent.atomic.AtomicLongArray;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricsInfo;
@ -28,30 +26,29 @@ import org.apache.hadoop.metrics2.MetricsInfo;
*/
@InterfaceAudience.Private
public class MutableSizeHistogram extends MutableRangeHistogram {
private final String rangeType = "SizeRangeCount";
private final long[] ranges = {10,100,1000,10000,100000,1000000,10000000,100000000};
private final AtomicLongArray rangeVals = new AtomicLongArray(getRange().length+1);
private final static String RANGE_TYPE = "SizeRangeCount";
private final static long[] RANGES = {10,100,1000,10000,100000,1000000,10000000,100000000};
public MutableSizeHistogram(MetricsInfo info) {
this(info.name(), info.description());
}
public MutableSizeHistogram(String name, String description) {
super(name, description);
this(name, description, RANGES[RANGES.length-2]);
}
public MutableSizeHistogram(String name, String description, long expectedMax) {
super(name, description, expectedMax);
}
@Override
public String getRangeType() {
return rangeType;
return RANGE_TYPE;
}
@Override
public long[] getRange() {
return ranges;
public long[] getRanges() {
return RANGES;
}
@Override
public AtomicLongArray getRangeVals() {
return rangeVals;
}
}

View File

@ -18,8 +18,6 @@
package org.apache.hadoop.metrics2.lib;
import java.util.concurrent.atomic.AtomicLongArray;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricsInfo;
@ -28,31 +26,30 @@ import org.apache.hadoop.metrics2.MetricsInfo;
*/
@InterfaceAudience.Private
public class MutableTimeHistogram extends MutableRangeHistogram {
private final String rangeType = "TimeRangeCount";
private final long[] ranges =
private final static String RANGE_TYPE = "TimeRangeCount";
private final static long[] RANGES =
{ 1, 3, 10, 30, 100, 300, 1000, 3000, 10000, 30000, 60000, 120000, 300000, 600000 };
private final AtomicLongArray rangeVals = new AtomicLongArray(ranges.length+1);
public MutableTimeHistogram(MetricsInfo info) {
this(info.name(), info.description());
}
public MutableTimeHistogram(String name, String description) {
super(name, description);
this(name, description, RANGES[RANGES.length - 2]);
}
public MutableTimeHistogram(String name, String description, long expectedMax) {
super(name, description, expectedMax);
}
@Override
public String getRangeType() {
return rangeType;
return RANGE_TYPE;
}
@Override
public long[] getRange() {
return ranges;
public long[] getRanges() {
return RANGES;
}
@Override
public AtomicLongArray getRangeVals() {
return rangeVals;
}
}

View File

@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.metrics;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MetricsTests;
import org.apache.hadoop.metrics2.lib.MutableCounterLong;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.junit.BeforeClass;
import org.junit.Test;
@ -72,9 +73,9 @@ public class TestBaseSourceImpl {
@Test
public void testIncCounters() throws Exception {
bmsi.incCounters("testinccounter", 100);
assertEquals(100, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
assertEquals(100, ((MutableFastCounter) bmsi.metricsRegistry.get("testinccounter")).value());
bmsi.incCounters("testinccounter", 100);
assertEquals(200, ((MutableCounterLong) bmsi.metricsRegistry.get("testinccounter")).value());
assertEquals(200, ((MutableFastCounter) bmsi.metricsRegistry.get("testinccounter")).value());
}

View File

@ -178,7 +178,6 @@ org.apache.hadoop.util.StringUtils;
AgeSnapshot ageAtEvictionSnapshot = bc.getStats().getAgeAtEvictionSnapshot();
// Only show if non-zero mean and stddev as is the case in combinedblockcache
double mean = ageAtEvictionSnapshot.getMean();
double stddev = ageAtEvictionSnapshot.getStdDev();
</%java>
<tr>
<td>Evicted</td>
@ -197,13 +196,6 @@ org.apache.hadoop.util.StringUtils;
<td>Mean age of Blocks at eviction time (seconds)</td>
</tr>
</%if>
<%if stddev > 0 %>
<tr>
<td>StdDev</td>
<td><% String.format("%,d", (long)(ageAtEvictionSnapshot.getStdDev()/1000000)) %></td>
<td>Standard Deviation for age of Blocks at eviction time</td>
</tr>
</%if>
</%def>
<%def hits_tmpl>

View File

@ -37,7 +37,6 @@ org.apache.hadoop.hbase.io.hfile.bucket.BucketCache;
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator;
org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.Bucket;
org.apache.hadoop.util.StringUtils;
com.codahale.metrics.Snapshot;
</%import>
<%java>
BlockCache bc = cacheConfig == null ? null : cacheConfig.getBlockCache();

View File

@ -34,7 +34,6 @@ org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
org.apache.hadoop.hbase.util.DirectMemoryUtils;
org.apache.hadoop.util.StringUtils;
com.codahale.metrics.Snapshot;
java.lang.management.ManagementFactory;
</%import>
<div class="tabbable">

View File

@ -17,56 +17,54 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import org.apache.hadoop.hbase.util.FastLongHistogram;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.Snapshot;
/**
* Snapshot of block cache age in cache.
* This object is preferred because we can control how it is serialized out when JSON'ing.
*/
@JsonIgnoreProperties({"ageHistogram", "snapshot"})
public class AgeSnapshot {
private final Snapshot snapshot;
AgeSnapshot(final Histogram ageHistogram) {
this.snapshot = ageHistogram.getSnapshot();
private final FastLongHistogram ageHistogram;
private final long[] quantiles;
AgeSnapshot(final FastLongHistogram ageHistogram) {
this.ageHistogram = ageHistogram;
this.quantiles = ageHistogram.getQuantiles(new double[]{0.75, 0.95, 0.98, 0.99, 0.999});
}
public double get75thPercentile() {
return snapshot.get75thPercentile();
return quantiles[0];
}
public double get95thPercentile() {
return snapshot.get95thPercentile();
return quantiles[1];
}
public double get98thPercentile() {
return snapshot.get98thPercentile();
}
public double get999thPercentile() {
return snapshot.get999thPercentile();
return quantiles[2];
}
public double get99thPercentile() {
return snapshot.get99thPercentile();
return quantiles[3];
}
public double get999thPercentile() {
return quantiles[4];
}
public double getMean() {
return this.snapshot.getMean();
return this.ageHistogram.getMean();
}
public double getMax() {
return snapshot.getMax();
return this.ageHistogram.getMax();
}
public double getMin() {
return snapshot.getMin();
}
public double getStdDev() {
return snapshot.getStdDev();
return this.ageHistogram.getMin();
}
}

View File

@ -25,17 +25,13 @@ import java.util.concurrent.ConcurrentSkipListSet;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.FastLongHistogram;
import org.codehaus.jackson.JsonGenerationException;
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
import org.codehaus.jackson.map.JsonMappingException;
import org.codehaus.jackson.map.ObjectMapper;
import org.codehaus.jackson.map.SerializationConfig;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.MetricRegistry;
import com.codahale.metrics.Snapshot;
import static com.codahale.metrics.MetricRegistry.name;
/**
* Utilty for aggregating counts in CachedBlocks and toString/toJSON CachedBlocks and BlockCaches.
@ -43,11 +39,6 @@ import static com.codahale.metrics.MetricRegistry.name;
*/
@InterfaceAudience.Private
public class BlockCacheUtil {
/**
* Needed making histograms.
*/
private static final MetricRegistry METRICS = new MetricRegistry();
/**
* Needed generating JSON.
*/
@ -206,7 +197,7 @@ public class BlockCacheUtil {
*/
private NavigableMap<String, NavigableSet<CachedBlock>> cachedBlockByFile =
new ConcurrentSkipListMap<String, NavigableSet<CachedBlock>>();
Histogram age = METRICS.histogram(name(CachedBlocksByFile.class, "age"));
FastLongHistogram hist = new FastLongHistogram();
/**
* @param cb
@ -228,7 +219,7 @@ public class BlockCacheUtil {
this.dataSize += cb.getSize();
}
long age = this.now - cb.getCachedTime();
this.age.update(age);
this.hist.add(age, 1);
return false;
}
@ -271,18 +262,22 @@ public class BlockCacheUtil {
}
public AgeSnapshot getAgeInCacheSnapshot() {
return new AgeSnapshot(this.age);
return new AgeSnapshot(this.hist);
}
@Override
public String toString() {
Snapshot snapshot = age.getSnapshot();
AgeSnapshot snapshot = getAgeInCacheSnapshot();
return "count=" + count + ", dataBlockCount=" + dataBlockCount + ", size=" + size +
", dataSize=" + getDataSize() +
", mean age=" + snapshot.getMean() + ", stddev age=" + snapshot.getStdDev() +
", min age=" + snapshot.getMin() + ", max age=" + snapshot.getMax() +
", mean age=" + snapshot.getMean() +
", min age=" + snapshot.getMin() +
", max age=" + snapshot.getMax() +
", 75th percentile age=" + snapshot.get75thPercentile() +
", 95th percentile age=" + snapshot.get95thPercentile() +
", 99th percentile age=" + snapshot.get99thPercentile();
", 98th percentile age=" + snapshot.get98thPercentile() +
", 99th percentile age=" + snapshot.get99thPercentile() +
", 99.9th percentile age=" + snapshot.get99thPercentile();
}
}
}

View File

@ -22,20 +22,15 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import com.codahale.metrics.Histogram;
import com.codahale.metrics.MetricRegistry;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.FastLongHistogram;
import static com.codahale.metrics.MetricRegistry.name;
/**
* Class that implements cache metrics.
*/
@InterfaceAudience.Private
public class CacheStats {
/**
* Needed making histograms.
*/
private static final MetricRegistry METRICS = new MetricRegistry();
/** Sliding window statistics. The number of metric periods to include in
* sliding window hit ratio calculations.
@ -43,10 +38,10 @@ public class CacheStats {
static final int DEFAULT_WINDOW_PERIODS = 5;
/** The number of getBlock requests that were cache hits */
private final AtomicLong hitCount = new AtomicLong(0);
private final Counter hitCount = new Counter();
/** The number of getBlock requests that were cache hits from primary replica */
private final AtomicLong primaryHitCount = new AtomicLong(0);
private final Counter primaryHitCount = new Counter();
/**
* The number of getBlock requests that were cache hits, but only from
@ -54,27 +49,27 @@ public class CacheStats {
* attempt to read from the block cache even if they will not put new blocks
* into the block cache. See HBASE-2253 for more information.
*/
private final AtomicLong hitCachingCount = new AtomicLong(0);
private final Counter hitCachingCount = new Counter();
/** The number of getBlock requests that were cache misses */
private final AtomicLong missCount = new AtomicLong(0);
private final Counter missCount = new Counter();
/** The number of getBlock requests for primary replica that were cache misses */
private final AtomicLong primaryMissCount = new AtomicLong(0);
private final Counter primaryMissCount = new Counter();
/**
* The number of getBlock requests that were cache misses, but only from
* requests that were set to use the block cache.
*/
private final AtomicLong missCachingCount = new AtomicLong(0);
private final Counter missCachingCount = new Counter();
/** The number of times an eviction has occurred */
private final AtomicLong evictionCount = new AtomicLong(0);
private final Counter evictionCount = new Counter();
/** The total number of blocks that have been evicted */
private final AtomicLong evictedBlockCount = new AtomicLong(0);
private final Counter evictedBlockCount = new Counter();
/** The total number of blocks for primary replica that have been evicted */
private final AtomicLong primaryEvictedBlockCount = new AtomicLong(0);
private final Counter primaryEvictedBlockCount = new Counter();
/** The total number of blocks that were not inserted. */
private final AtomicLong failedInserts = new AtomicLong(0);
@ -102,7 +97,7 @@ public class CacheStats {
/**
* Keep running age at eviction time
*/
private Histogram ageAtEviction;
private FastLongHistogram ageAtEviction;
private long startTime = System.nanoTime();
public CacheStats(final String name) {
@ -115,7 +110,7 @@ public class CacheStats {
this.hitCachingCounts = initializeZeros(numPeriodsInWindow);
this.requestCounts = initializeZeros(numPeriodsInWindow);
this.requestCachingCounts = initializeZeros(numPeriodsInWindow);
this.ageAtEviction = METRICS.histogram(name(CacheStats.class, name + ".ageAtEviction"));
this.ageAtEviction = new FastLongHistogram();
}
@Override
@ -127,14 +122,13 @@ public class CacheStats {
", evictedBlockCount=" + getEvictedCount() +
", primaryMissCount=" + getPrimaryMissCount() +
", primaryHitCount=" + getPrimaryHitCount() +
", evictedAgeMean=" + snapshot.getMean() +
", evictedAgeStdDev=" + snapshot.getStdDev();
", evictedAgeMean=" + snapshot.getMean();
}
public void miss(boolean caching, boolean primary) {
missCount.incrementAndGet();
if (primary) primaryMissCount.incrementAndGet();
if (caching) missCachingCount.incrementAndGet();
missCount.increment();
if (primary) primaryMissCount.increment();
if (caching) missCachingCount.increment();
}
public void hit(boolean caching) {
@ -142,20 +136,20 @@ public class CacheStats {
}
public void hit(boolean caching, boolean primary) {
hitCount.incrementAndGet();
if (primary) primaryHitCount.incrementAndGet();
if (caching) hitCachingCount.incrementAndGet();
hitCount.increment();
if (primary) primaryHitCount.increment();
if (caching) hitCachingCount.increment();
}
public void evict() {
evictionCount.incrementAndGet();
evictionCount.increment();
}
public void evicted(final long t, boolean primary) {
if (t > this.startTime) this.ageAtEviction.update(t - this.startTime);
this.evictedBlockCount.incrementAndGet();
if (t > this.startTime) this.ageAtEviction.add(t - this.startTime,1);
this.evictedBlockCount.increment();
if (primary) {
primaryEvictedBlockCount.incrementAndGet();
primaryEvictedBlockCount.increment();
}
}

View File

@ -34,7 +34,6 @@ import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
@ -61,6 +60,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.io.Writable;
@ -179,17 +179,19 @@ public class HFile {
*/
public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
// For measuring number of checksum failures
static final AtomicLong checksumFailures = new AtomicLong();
static final Counter checksumFailures = new Counter();
// for test purpose
public static final AtomicLong dataBlockReadCnt = new AtomicLong(0);
public static final Counter dataBlockReadCnt = new Counter();
/**
* Number of checksum verification failures. It also
* clears the counter.
*/
public static final long getChecksumFailuresCount() {
return checksumFailures.getAndSet(0);
long count = checksumFailures.get();
checksumFailures.set(0);
return count;
}
/** API required to write an {@link HFile} */

View File

@ -1522,7 +1522,7 @@ public class HFileBlock implements Cacheable {
HFile.LOG.warn(msg);
throw new IOException(msg); // cannot happen case here
}
HFile.checksumFailures.incrementAndGet(); // update metrics
HFile.checksumFailures.increment(); // update metrics
// If we have a checksum failure, we fall back into a mode where
// the next few reads use HDFS level checksums. We aim to make the

View File

@ -1497,7 +1497,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
assert cachedBlock.isUnpacked() : "Packed block leak.";
if (cachedBlock.getBlockType().isData()) {
if (updateCacheMetrics) {
HFile.dataBlockReadCnt.incrementAndGet();
HFile.dataBlockReadCnt.increment();
}
// Validate encoding type for data blocks. We include encoding
// type in the cache key, and we expect it to match on a cache hit.
@ -1537,7 +1537,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable {
}
if (updateCacheMetrics && hfileBlock.getBlockType().isData()) {
HFile.dataBlockReadCnt.incrementAndGet();
HFile.dataBlockReadCnt.increment();
}
return unpacked;

View File

@ -18,10 +18,9 @@
*/
package org.apache.hadoop.hbase.io.hfile.bucket;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
@ -29,8 +28,8 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
*/
@InterfaceAudience.Private
public class BucketCacheStats extends CacheStats {
private final AtomicLong ioHitCount = new AtomicLong(0);
private final AtomicLong ioHitTime = new AtomicLong(0);
private final Counter ioHitCount = new Counter(0);
private final Counter ioHitTime = new Counter(0);
private final static int nanoTime = 1000000;
private long lastLogTime = EnvironmentEdgeManager.currentTime();
@ -45,8 +44,8 @@ public class BucketCacheStats extends CacheStats {
}
public void ioHit(long time) {
ioHitCount.incrementAndGet();
ioHitTime.addAndGet(time);
ioHitCount.increment();
ioHitTime.add(time);
}
public long getIOHitsPerSecond() {

View File

@ -24,7 +24,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
@ -37,6 +36,7 @@ import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile.Reader;
import org.apache.hadoop.hbase.util.Counter;
/**
* KeyValueScanner adaptor over the Reader. It also provides hooks into
@ -60,7 +60,7 @@ public class StoreFileScanner implements KeyValueScanner {
// if have encountered the next row. Only used for reversed scan
private boolean stopSkippingKVsIfNextRow = false;
private static AtomicLong seekCount;
private static Counter seekCount;
private ScanQueryMatcher matcher;
@ -164,7 +164,7 @@ public class StoreFileScanner implements KeyValueScanner {
}
public boolean seek(Cell key) throws IOException {
if (seekCount != null) seekCount.incrementAndGet();
if (seekCount != null) seekCount.increment();
try {
try {
@ -191,7 +191,7 @@ public class StoreFileScanner implements KeyValueScanner {
}
public boolean reseek(Cell key) throws IOException {
if (seekCount != null) seekCount.incrementAndGet();
if (seekCount != null) seekCount.increment();
try {
try {
@ -424,7 +424,7 @@ public class StoreFileScanner implements KeyValueScanner {
return seekCount.get();
}
static final void instrument() {
seekCount = new AtomicLong();
seekCount = new Counter();
}
@Override
@ -447,7 +447,7 @@ public class StoreFileScanner implements KeyValueScanner {
Cell key = originalKey;
do {
Cell seekKey = CellUtil.createFirstOnRow(key);
if (seekCount != null) seekCount.incrementAndGet();
if (seekCount != null) seekCount.increment();
if (!hfs.seekBefore(seekKey)) {
this.cur = null;
return false;
@ -455,7 +455,7 @@ public class StoreFileScanner implements KeyValueScanner {
Cell curCell = hfs.getCell();
Cell firstKeyOfPreviousRow = CellUtil.createFirstOnRow(curCell);
if (seekCount != null) seekCount.incrementAndGet();
if (seekCount != null) seekCount.increment();
if (!seekAtOrAfter(hfs, firstKeyOfPreviousRow)) {
this.cur = null;
return false;