HBASE-19724 Fixed Checkstyle errors in hbase-hadoop2-compat and enabled Checkstyle to fail on violations

This commit is contained in:
Jan Hentschel 2018-01-07 13:10:31 +01:00
parent 2fdd36fdcf
commit d5ea926321
50 changed files with 468 additions and 437 deletions

View File

@ -71,6 +71,13 @@ limitations under the License.
<groupId>net.revelc.code</groupId>
<artifactId>warbucks-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-checkstyle-plugin</artifactId>
<configuration>
<failOnViolation>true</failOnViolation>
</configuration>
</plugin>
</plugins>
<pluginManagement>
<plugins>

View File

@ -19,20 +19,17 @@
package org.apache.hadoop.hbase.ipc;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.ExceptionTrackingSourceImpl;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsHBaseServerSourceImpl extends ExceptionTrackingSourceImpl
implements MetricsHBaseServerSource {
private final MetricsHBaseServerWrapper wrapper;
private final MutableFastCounter authorizationSuccesses;
private final MutableFastCounter authorizationFailures;
@ -119,10 +116,14 @@ public class MetricsHBaseServerSourceImpl extends ExceptionTrackingSourceImpl
}
@Override
public void sentResponse(long count) { this.responseSize.add(count); }
public void sentResponse(long count) {
this.responseSize.add(count);
}
@Override
public void receivedRequest(long count) { this.requestSize.add(count); }
public void receivedRequest(long count) {
this.requestSize.add(count);
}
@Override
public void dequeuedCall(int qTime) {

View File

@ -20,14 +20,14 @@ package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapreduce.Cluster;
import org.apache.hadoop.mapreduce.JobSubmissionFiles;
/**
* Utility methods to interact with a job.
@ -46,8 +46,8 @@ public abstract class JobUtil {
*
* @param conf system configuration
* @return staging directory path
* @throws IOException
* @throws InterruptedException
* @throws IOException if the ownership on the staging directory is not as expected
* @throws InterruptedException if the thread getting the staging directory is interrupted
*/
public static Path getStagingDir(Configuration conf)
throws IOException, InterruptedException {

View File

@ -18,12 +18,12 @@
package org.apache.hadoop.hbase.master;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.OperationMetrics;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsAssignmentManagerSourceImpl
@ -53,12 +53,12 @@ public class MetricsAssignmentManagerSourceImpl
}
public void init() {
ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, RIT_COUNT_DESC, 0l);
ritGauge = metricsRegistry.newGauge(RIT_COUNT_NAME, RIT_COUNT_DESC, 0L);
ritCountOverThresholdGauge = metricsRegistry.newGauge(RIT_COUNT_OVER_THRESHOLD_NAME,
RIT_COUNT_OVER_THRESHOLD_DESC,0l);
ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, RIT_OLDEST_AGE_DESC, 0l);
RIT_COUNT_OVER_THRESHOLD_DESC,0L);
ritOldestAgeGauge = metricsRegistry.newGauge(RIT_OLDEST_AGE_NAME, RIT_OLDEST_AGE_DESC, 0L);
ritDurationHisto = metricsRegistry.newTimeHistogram(RIT_DURATION_NAME, RIT_DURATION_DESC);
operationCounter = metricsRegistry.getCounter(OPERATION_COUNT_NAME, 0l);
operationCounter = metricsRegistry.getCounter(OPERATION_COUNT_NAME, 0L);
/**
* NOTE: Please refer to HBASE-9774 and HBASE-14282. Based on these two issues, HBase is

View File

@ -18,9 +18,9 @@
package org.apache.hadoop.hbase.master;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsMasterFilesystemSourceImpl

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.hbase.master;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop2 implementation of MetricsMasterSource.

View File

@ -19,20 +19,21 @@ package org.apache.hadoop.hbase.master;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.yetus.audience.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.MutableGaugeLong;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Implementation of {@link MetricsMasterQuotaSource} which writes the values passed in via the
* interface to the metrics backend.
*/
@InterfaceAudience.Private
public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl implements MetricsMasterQuotaSource {
public class MetricsMasterQuotaSourceImpl extends BaseSourceImpl
implements MetricsMasterQuotaSource {
private final MetricsMasterWrapper wrapper;
private final MutableGaugeLong spaceQuotasGauge;
private final MutableGaugeLong tablesViolatingQuotasGauge;

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.master;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.hbase.metrics.OperationMetrics;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop2 implementation of MetricsMasterSource.
@ -61,9 +61,9 @@ public class MetricsMasterSourceImpl
@Override
public void init() {
super.init();
clusterRequestsCounter = metricsRegistry.newCounter(CLUSTER_REQUESTS_NAME, "", 0l);
clusterRequestsCounter = metricsRegistry.newCounter(CLUSTER_REQUESTS_NAME, "", 0L);
/**
/*
* NOTE: Please refer to HBASE-9774 and HBASE-14282. Based on these two issues, HBase is
* moving away from using Hadoop's metric2 to having independent HBase specific Metrics. Use
* {@link BaseSourceImpl#registry} to register the new metrics.
@ -92,8 +92,9 @@ public class MetricsMasterSourceImpl
MASTER_ACTIVE_TIME_DESC), masterWrapper.getActiveTime())
.addGauge(Interns.info(MASTER_START_TIME_NAME,
MASTER_START_TIME_DESC), masterWrapper.getStartTime())
.addGauge(Interns.info(MASTER_FINISHED_INITIALIZATION_TIME_NAME, MASTER_FINISHED_INITIALIZATION_TIME_DESC),
masterWrapper.getMasterInitializationTime())
.addGauge(Interns.info(MASTER_FINISHED_INITIALIZATION_TIME_NAME,
MASTER_FINISHED_INITIALIZATION_TIME_DESC),
masterWrapper.getMasterInitializationTime())
.addGauge(Interns.info(AVERAGE_LOAD_NAME, AVERAGE_LOAD_DESC),
masterWrapper.getAverageLoad())
.tag(Interns.info(LIVE_REGION_SERVERS_NAME, LIVE_REGION_SERVERS_DESC),

View File

@ -18,9 +18,9 @@
package org.apache.hadoop.hbase.master;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsSnapshotSourceImpl extends BaseSourceImpl implements MetricsSnapshotSource {

View File

@ -18,14 +18,13 @@
package org.apache.hadoop.hbase.master.balancer;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsBalancerSourceImpl extends BaseSourceImpl implements MetricsBalancerSource{
public class MetricsBalancerSourceImpl extends BaseSourceImpl implements MetricsBalancerSource {
private MetricHistogram blanceClusterHisto;
private MutableFastCounter miscCount;
@ -39,12 +38,10 @@ public class MetricsBalancerSourceImpl extends BaseSourceImpl implements Metrics
super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
}
@Override
public void init() {
blanceClusterHisto = metricsRegistry.newTimeHistogram(BALANCE_CLUSTER);
miscCount = metricsRegistry.newCounter(MISC_INVOATION_COUNT, "", 0L);
}
@Override
@ -54,6 +51,6 @@ public class MetricsBalancerSourceImpl extends BaseSourceImpl implements Metrics
@Override
public void incrMiscInvocations() {
miscCount.incr();
miscCount.incr();
}
}

View File

@ -22,10 +22,10 @@ import java.util.LinkedHashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
public class MetricsStochasticBalancerSourceImpl extends MetricsBalancerSourceImpl implements
@ -38,14 +38,14 @@ public class MetricsStochasticBalancerSourceImpl extends MetricsBalancerSourceIm
private int mruCap = calcMruCap(metricsSize);
private Map<String, Map<String, Double>> stochasticCosts =
new LinkedHashMap<String, Map<String, Double>>(mruCap, MRU_LOAD_FACTOR, true) {
private static final long serialVersionUID = 8204713453436906599L;
new LinkedHashMap<String, Map<String, Double>>(mruCap, MRU_LOAD_FACTOR, true) {
private static final long serialVersionUID = 8204713453436906599L;
@Override
protected boolean removeEldestEntry(Map.Entry<String, Map<String, Double>> eldest) {
return size() > mruCap;
}
};
@Override
protected boolean removeEldestEntry(Map.Entry<String, Map<String, Double>> eldest) {
return size() > mruCap;
}
};
private Map<String, String> costFunctionDescs = new ConcurrentHashMap<>();
/**
@ -98,7 +98,11 @@ public class MetricsStochasticBalancerSourceImpl extends MetricsBalancerSourceIm
String attrName = tableEntry.getKey() + TABLE_FUNCTION_SEP + costEntry.getKey();
Double cost = costEntry.getValue();
String functionDesc = costFunctionDescs.get(costEntry.getKey());
if (functionDesc == null) functionDesc = costEntry.getKey();
if (functionDesc == null) {
functionDesc = costEntry.getKey();
}
metricsRecordBuilder.addGauge(Interns.info(attrName, functionDesc), cost);
}
}
@ -106,5 +110,4 @@ public class MetricsStochasticBalancerSourceImpl extends MetricsBalancerSourceIm
}
metricsRegistry.snapshot(metricsRecordBuilder, all);
}
}

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.hbase.metrics;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.impl.GlobalMetricRegistriesAdapter;
import org.apache.hadoop.hbase.metrics.impl.HBaseMetrics2HadoopMetricsAdapter;
import org.apache.hadoop.hbase.regionserver.MetricsRegionServerSourceImpl;
@ -31,6 +30,7 @@ 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;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop 2 implementation of BaseSource (using metrics2 framework). It handles registration to
@ -46,7 +46,10 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
private boolean inited = false;
synchronized void init(String name) {
if (inited) return;
if (inited) {
return;
}
inited = true;
DefaultMetricsSystem.initialize(HBASE_METRICS_SYSTEM_NAME);
JvmMetrics.initSingleton(name, "");
@ -60,9 +63,10 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
/**
* @deprecated Use hbase-metrics/hbase-metrics-api module interfaces for new metrics.
* Defining BaseSources for new metric groups (WAL, RPC, etc) is not needed anymore, however,
* for existing BaseSource implemetnations, please use the field named "registry" which is a
* MetricRegistry instance together with the HBaseMetrics2HadoopMetricsAdapter.
* Defining BaseSources for new metric groups (WAL, RPC, etc) is not needed anymore,
* however, for existing {@link BaseSource} implementations, please use the field
* named "registry" which is a {@link MetricRegistry} instance together with the
* {@link HBaseMetrics2HadoopMetricsAdapter}.
*/
@Deprecated
protected final DynamicMetricsRegistry metricsRegistry;
@ -144,7 +148,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.getGauge(gaugeName, 0l);
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0L);
gaugeInt.incr(delta);
}
@ -155,7 +159,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.getGauge(gaugeName, 0l);
MutableGaugeLong gaugeInt = metricsRegistry.getGauge(gaugeName, 0L);
gaugeInt.decr(delta);
}
@ -166,7 +170,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
* @param delta the ammount to increment
*/
public void incCounters(String key, long delta) {
MutableFastCounter counter = metricsRegistry.getCounter(key, 0l);
MutableFastCounter counter = metricsRegistry.getCounter(key, 0L);
counter.incr(delta);
}
@ -180,7 +184,7 @@ public class BaseSourceImpl implements BaseSource, MetricsSource {
/**
* Remove a named gauge.
*
* @param key
* @param key the key of the gauge to remove
*/
public void removeMetric(String key) {
metricsRegistry.removeMetric(key);

View File

@ -18,21 +18,18 @@
package org.apache.hadoop.hbase.metrics;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheBuilder;
import org.apache.hbase.thirdparty.com.google.common.cache.CacheLoader;
import org.apache.hbase.thirdparty.com.google.common.cache.LoadingCache;
/**
* Helpers to create interned metrics info

View File

@ -20,8 +20,8 @@ package org.apache.hadoop.hbase.metrics;
import javax.management.ObjectName;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.metrics2.util.MBeans;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop2 metrics2 implementation of an object that registers MBeans.

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.metrics;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
import org.apache.hbase.thirdparty.com.google.common.base.Objects;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricsInfo;
/**
* Making implementing metric info a little easier
*/

View File

@ -18,11 +18,11 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop2 implementation of MetricsHeapMemoryManagerSource. Implements BaseSource through

View File

@ -23,15 +23,15 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.impl.JmxCacheBuster;
import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class MetricsRegionAggregateSourceImpl extends BaseSourceImpl

View File

@ -17,10 +17,10 @@
*/
package org.apache.hadoop.hbase.regionserver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.MetricsIOSource;
import org.apache.hadoop.hbase.io.MetricsIOSourceImpl;
import org.apache.hadoop.hbase.io.MetricsIOWrapper;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Factory to create MetricsRegionServerSource when given a MetricsRegionServerWrapper
@ -65,7 +65,8 @@ public class MetricsRegionServerSourceFactoryImpl implements MetricsRegionServer
}
@Override
public synchronized MetricsRegionServerSource createServer(MetricsRegionServerWrapper regionServerWrapper) {
public synchronized MetricsRegionServerSource createServer(
MetricsRegionServerWrapper regionServerWrapper) {
return new MetricsRegionServerSourceImpl(regionServerWrapper);
}

View File

@ -18,13 +18,13 @@
package org.apache.hadoop.hbase.regionserver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop2 implementation of MetricsRegionServerSource.
@ -220,7 +220,7 @@ public class MetricsRegionServerSourceImpl
@Override
public void incrSlowPut() {
slowPut.incr();
slowPut.incr();
}
@Override
@ -329,209 +329,136 @@ public class MetricsRegionServerSourceImpl
*/
@Override
public void getMetrics(MetricsCollector metricsCollector, boolean all) {
MetricsRecordBuilder mrb = metricsCollector.addRecord(metricsName);
// rsWrap can be null because this function is called inside of init.
if (rsWrap != null) {
mrb.addGauge(Interns.info(REGION_COUNT, REGION_COUNT_DESC), rsWrap.getNumOnlineRegions())
.addGauge(Interns.info(STORE_COUNT, STORE_COUNT_DESC), rsWrap.getNumStores())
.addGauge(Interns.info(WALFILE_COUNT, WALFILE_COUNT_DESC), rsWrap.getNumWALFiles())
.addGauge(Interns.info(WALFILE_SIZE, WALFILE_SIZE_DESC), rsWrap.getWALFileSize())
.addGauge(Interns.info(STOREFILE_COUNT, STOREFILE_COUNT_DESC), rsWrap.getNumStoreFiles())
.addGauge(Interns.info(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC), rsWrap.getMemStoreSize())
.addGauge(Interns.info(STOREFILE_SIZE, STOREFILE_SIZE_DESC), rsWrap.getStoreFileSize())
.addGauge(Interns.info(MAX_STORE_FILE_AGE, MAX_STORE_FILE_AGE_DESC),
rsWrap.getMaxStoreFileAge())
.addGauge(Interns.info(MIN_STORE_FILE_AGE, MIN_STORE_FILE_AGE_DESC),
rsWrap.getMinStoreFileAge())
.addGauge(Interns.info(AVG_STORE_FILE_AGE, AVG_STORE_FILE_AGE_DESC),
rsWrap.getAvgStoreFileAge())
.addGauge(Interns.info(NUM_REFERENCE_FILES, NUM_REFERENCE_FILES_DESC),
rsWrap.getNumReferenceFiles())
.addGauge(Interns.info(RS_START_TIME_NAME, RS_START_TIME_DESC),
rsWrap.getStartCode())
.addGauge(Interns.info(AVERAGE_REGION_SIZE, AVERAGE_REGION_SIZE_DESC), rsWrap.getAverageRegionSize())
.addCounter(Interns.info(TOTAL_REQUEST_COUNT, TOTAL_REQUEST_COUNT_DESC),
rsWrap.getTotalRequestCount())
.addCounter(Interns.info(TOTAL_ROW_ACTION_REQUEST_COUNT, TOTAL_ROW_ACTION_REQUEST_COUNT_DESC),
rsWrap.getTotalRowActionRequestCount())
.addCounter(Interns.info(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC),
rsWrap.getReadRequestsCount())
.addCounter(Interns.info(FILTERED_READ_REQUEST_COUNT, FILTERED_READ_REQUEST_COUNT_DESC),
rsWrap.getFilteredReadRequestsCount())
.addCounter(Interns.info(WRITE_REQUEST_COUNT, WRITE_REQUEST_COUNT_DESC),
rsWrap.getWriteRequestsCount())
.addCounter(Interns.info(RPC_GET_REQUEST_COUNT, RPC_GET_REQUEST_COUNT_DESC),
rsWrap.getRpcGetRequestsCount())
.addCounter(Interns.info(RPC_SCAN_REQUEST_COUNT, RPC_SCAN_REQUEST_COUNT_DESC),
rsWrap.getRpcScanRequestsCount())
.addCounter(Interns.info(RPC_MULTI_REQUEST_COUNT, RPC_MULTI_REQUEST_COUNT_DESC),
rsWrap.getRpcMultiRequestsCount())
.addCounter(Interns.info(RPC_MUTATE_REQUEST_COUNT, RPC_MUTATE_REQUEST_COUNT_DESC),
rsWrap.getRpcMutateRequestsCount())
.addCounter(Interns.info(CHECK_MUTATE_FAILED_COUNT, CHECK_MUTATE_FAILED_COUNT_DESC),
rsWrap.getCheckAndMutateChecksFailed())
.addCounter(Interns.info(CHECK_MUTATE_PASSED_COUNT, CHECK_MUTATE_PASSED_COUNT_DESC),
rsWrap.getCheckAndMutateChecksPassed())
.addGauge(Interns.info(STOREFILE_INDEX_SIZE, STOREFILE_INDEX_SIZE_DESC),
rsWrap.getStoreFileIndexSize())
.addGauge(Interns.info(STATIC_INDEX_SIZE, STATIC_INDEX_SIZE_DESC),
rsWrap.getTotalStaticIndexSize())
.addGauge(Interns.info(STATIC_BLOOM_SIZE, STATIC_BLOOM_SIZE_DESC),
rsWrap.getTotalStaticBloomSize())
.addGauge(
Interns.info(NUMBER_OF_MUTATIONS_WITHOUT_WAL, NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC),
rsWrap.getNumMutationsWithoutWAL())
.addGauge(Interns.info(DATA_SIZE_WITHOUT_WAL, DATA_SIZE_WITHOUT_WAL_DESC),
rsWrap.getDataInMemoryWithoutWAL())
.addGauge(Interns.info(PERCENT_FILES_LOCAL, PERCENT_FILES_LOCAL_DESC),
rsWrap.getPercentFileLocal())
.addGauge(Interns.info(PERCENT_FILES_LOCAL_SECONDARY_REGIONS,
PERCENT_FILES_LOCAL_SECONDARY_REGIONS_DESC),
rsWrap.getPercentFileLocalSecondaryRegions())
.addGauge(Interns.info(SPLIT_QUEUE_LENGTH, SPLIT_QUEUE_LENGTH_DESC),
rsWrap.getSplitQueueSize())
.addGauge(Interns.info(COMPACTION_QUEUE_LENGTH, COMPACTION_QUEUE_LENGTH_DESC),
rsWrap.getCompactionQueueSize())
.addGauge(Interns.info(SMALL_COMPACTION_QUEUE_LENGTH, SMALL_COMPACTION_QUEUE_LENGTH_DESC),
rsWrap.getSmallCompactionQueueSize())
.addGauge(Interns.info(LARGE_COMPACTION_QUEUE_LENGTH, LARGE_COMPACTION_QUEUE_LENGTH_DESC),
rsWrap.getLargeCompactionQueueSize())
.addGauge(Interns.info(FLUSH_QUEUE_LENGTH, FLUSH_QUEUE_LENGTH_DESC),
rsWrap.getFlushQueueSize())
.addGauge(Interns.info(BLOCK_CACHE_FREE_SIZE, BLOCK_CACHE_FREE_DESC),
rsWrap.getBlockCacheFreeSize())
.addGauge(Interns.info(BLOCK_CACHE_COUNT, BLOCK_CACHE_COUNT_DESC),
rsWrap.getBlockCacheCount())
.addGauge(Interns.info(BLOCK_CACHE_SIZE, BLOCK_CACHE_SIZE_DESC),
rsWrap.getBlockCacheSize())
.addCounter(Interns.info(BLOCK_CACHE_HIT_COUNT, BLOCK_CACHE_HIT_COUNT_DESC),
rsWrap.getBlockCacheHitCount())
.addCounter(Interns.info(BLOCK_CACHE_PRIMARY_HIT_COUNT,
BLOCK_CACHE_PRIMARY_HIT_COUNT_DESC), rsWrap.getBlockCachePrimaryHitCount())
.addCounter(Interns.info(BLOCK_CACHE_MISS_COUNT, BLOCK_COUNT_MISS_COUNT_DESC),
rsWrap.getBlockCacheMissCount())
.addCounter(Interns.info(BLOCK_CACHE_PRIMARY_MISS_COUNT,
BLOCK_COUNT_PRIMARY_MISS_COUNT_DESC), rsWrap.getBlockCachePrimaryMissCount())
.addCounter(Interns.info(BLOCK_CACHE_EVICTION_COUNT, BLOCK_CACHE_EVICTION_COUNT_DESC),
rsWrap.getBlockCacheEvictedCount())
.addCounter(Interns.info(BLOCK_CACHE_PRIMARY_EVICTION_COUNT,
BLOCK_CACHE_PRIMARY_EVICTION_COUNT_DESC), rsWrap.getBlockCachePrimaryEvictedCount())
.addGauge(Interns.info(BLOCK_CACHE_HIT_PERCENT, BLOCK_CACHE_HIT_PERCENT_DESC),
rsWrap.getBlockCacheHitPercent())
.addGauge(Interns.info(BLOCK_CACHE_EXPRESS_HIT_PERCENT,
BLOCK_CACHE_EXPRESS_HIT_PERCENT_DESC), rsWrap.getBlockCacheHitCachingPercent())
.addCounter(Interns.info(BLOCK_CACHE_FAILED_INSERTION_COUNT,
BLOCK_CACHE_FAILED_INSERTION_COUNT_DESC),rsWrap.getBlockCacheFailedInsertions())
.addCounter(Interns.info(BLOCK_CACHE_DATA_MISS_COUNT, ""), rsWrap.getDataMissCount())
.addCounter(Interns.info(BLOCK_CACHE_LEAF_INDEX_MISS_COUNT, ""),
rsWrap.getLeafIndexMissCount())
.addCounter(Interns.info(BLOCK_CACHE_BLOOM_CHUNK_MISS_COUNT, ""),
rsWrap.getBloomChunkMissCount())
.addCounter(Interns.info(BLOCK_CACHE_META_MISS_COUNT, ""), rsWrap.getMetaMissCount())
.addCounter(Interns.info(BLOCK_CACHE_ROOT_INDEX_MISS_COUNT, ""),
rsWrap.getRootIndexMissCount())
.addCounter(Interns.info(BLOCK_CACHE_INTERMEDIATE_INDEX_MISS_COUNT, ""),
rsWrap.getIntermediateIndexMissCount())
.addCounter(Interns.info(BLOCK_CACHE_FILE_INFO_MISS_COUNT, ""),
rsWrap.getFileInfoMissCount())
.addCounter(Interns.info(BLOCK_CACHE_GENERAL_BLOOM_META_MISS_COUNT, ""),
rsWrap.getGeneralBloomMetaMissCount())
.addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_MISS_COUNT, ""),
rsWrap.getDeleteFamilyBloomMissCount())
.addCounter(Interns.info(BLOCK_CACHE_TRAILER_MISS_COUNT, ""),
rsWrap.getTrailerMissCount())
.addCounter(Interns.info(BLOCK_CACHE_DATA_HIT_COUNT, ""), rsWrap.getDataHitCount())
.addCounter(Interns.info(BLOCK_CACHE_LEAF_INDEX_HIT_COUNT, ""),
rsWrap.getLeafIndexHitCount())
.addCounter(Interns.info(BLOCK_CACHE_BLOOM_CHUNK_HIT_COUNT, ""),
rsWrap.getBloomChunkHitCount())
.addCounter(Interns.info(BLOCK_CACHE_META_HIT_COUNT, ""), rsWrap.getMetaHitCount())
.addCounter(Interns.info(BLOCK_CACHE_ROOT_INDEX_HIT_COUNT, ""),
rsWrap.getRootIndexHitCount())
.addCounter(Interns.info(BLOCK_CACHE_INTERMEDIATE_INDEX_HIT_COUNT, ""),
rsWrap.getIntermediateIndexHitCount())
.addCounter(Interns.info(BLOCK_CACHE_FILE_INFO_HIT_COUNT, ""),
rsWrap.getFileInfoHitCount())
.addCounter(Interns.info(BLOCK_CACHE_GENERAL_BLOOM_META_HIT_COUNT, ""),
rsWrap.getGeneralBloomMetaHitCount())
.addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_HIT_COUNT, ""),
rsWrap.getDeleteFamilyBloomHitCount())
.addCounter(Interns.info(BLOCK_CACHE_TRAILER_HIT_COUNT, ""), rsWrap.getTrailerHitCount())
.addGauge(Interns.info(L1_CACHE_HIT_COUNT, L1_CACHE_HIT_COUNT_DESC),
rsWrap.getL1CacheHitCount())
.addGauge(Interns.info(L1_CACHE_MISS_COUNT, L1_CACHE_MISS_COUNT_DESC),
rsWrap.getL1CacheMissCount())
.addGauge(Interns.info(L1_CACHE_HIT_RATIO, L1_CACHE_HIT_RATIO_DESC),
rsWrap.getL1CacheHitRatio())
.addGauge(Interns.info(L1_CACHE_MISS_RATIO, L1_CACHE_MISS_RATIO_DESC),
rsWrap.getL1CacheMissRatio())
.addGauge(Interns.info(L2_CACHE_HIT_COUNT, L2_CACHE_HIT_COUNT_DESC),
rsWrap.getL2CacheHitCount())
.addGauge(Interns.info(L2_CACHE_MISS_COUNT, L2_CACHE_MISS_COUNT_DESC),
rsWrap.getL2CacheMissCount())
.addGauge(Interns.info(L2_CACHE_HIT_RATIO, L2_CACHE_HIT_RATIO_DESC),
rsWrap.getL2CacheHitRatio())
.addGauge(Interns.info(L2_CACHE_MISS_RATIO, L2_CACHE_MISS_RATIO_DESC),
rsWrap.getL2CacheMissRatio())
.addCounter(Interns.info(UPDATES_BLOCKED_TIME, UPDATES_BLOCKED_DESC),
rsWrap.getUpdatesBlockedTime())
.addCounter(Interns.info(FLUSHED_CELLS, FLUSHED_CELLS_DESC),
rsWrap.getFlushedCellsCount())
.addCounter(Interns.info(COMPACTED_CELLS, COMPACTED_CELLS_DESC),
rsWrap.getCompactedCellsCount())
.addCounter(Interns.info(MAJOR_COMPACTED_CELLS, MAJOR_COMPACTED_CELLS_DESC),
rsWrap.getMajorCompactedCellsCount())
.addCounter(Interns.info(FLUSHED_CELLS_SIZE, FLUSHED_CELLS_SIZE_DESC),
rsWrap.getFlushedCellsSize())
.addCounter(Interns.info(COMPACTED_CELLS_SIZE, COMPACTED_CELLS_SIZE_DESC),
rsWrap.getCompactedCellsSize())
.addCounter(Interns.info(MAJOR_COMPACTED_CELLS_SIZE, MAJOR_COMPACTED_CELLS_SIZE_DESC),
rsWrap.getMajorCompactedCellsSize())
.addCounter(
Interns.info(CELLS_COUNT_COMPACTED_FROM_MOB, CELLS_COUNT_COMPACTED_FROM_MOB_DESC),
rsWrap.getCellsCountCompactedFromMob())
.addCounter(Interns.info(CELLS_COUNT_COMPACTED_TO_MOB, CELLS_COUNT_COMPACTED_TO_MOB_DESC),
rsWrap.getCellsCountCompactedToMob())
.addCounter(
Interns.info(CELLS_SIZE_COMPACTED_FROM_MOB, CELLS_SIZE_COMPACTED_FROM_MOB_DESC),
rsWrap.getCellsSizeCompactedFromMob())
.addCounter(Interns.info(CELLS_SIZE_COMPACTED_TO_MOB, CELLS_SIZE_COMPACTED_TO_MOB_DESC),
rsWrap.getCellsSizeCompactedToMob())
.addCounter(Interns.info(MOB_FLUSH_COUNT, MOB_FLUSH_COUNT_DESC),
rsWrap.getMobFlushCount())
.addCounter(Interns.info(MOB_FLUSHED_CELLS_COUNT, MOB_FLUSHED_CELLS_COUNT_DESC),
rsWrap.getMobFlushedCellsCount())
.addCounter(Interns.info(MOB_FLUSHED_CELLS_SIZE, MOB_FLUSHED_CELLS_SIZE_DESC),
rsWrap.getMobFlushedCellsSize())
.addCounter(Interns.info(MOB_SCAN_CELLS_COUNT, MOB_SCAN_CELLS_COUNT_DESC),
rsWrap.getMobScanCellsCount())
.addCounter(Interns.info(MOB_SCAN_CELLS_SIZE, MOB_SCAN_CELLS_SIZE_DESC),
rsWrap.getMobScanCellsSize())
.addGauge(Interns.info(MOB_FILE_CACHE_COUNT, MOB_FILE_CACHE_COUNT_DESC),
rsWrap.getMobFileCacheCount())
.addCounter(Interns.info(MOB_FILE_CACHE_ACCESS_COUNT, MOB_FILE_CACHE_ACCESS_COUNT_DESC),
rsWrap.getMobFileCacheAccessCount())
.addCounter(Interns.info(MOB_FILE_CACHE_MISS_COUNT, MOB_FILE_CACHE_MISS_COUNT_DESC),
rsWrap.getMobFileCacheMissCount())
.addCounter(
Interns.info(MOB_FILE_CACHE_EVICTED_COUNT, MOB_FILE_CACHE_EVICTED_COUNT_DESC),
rsWrap.getMobFileCacheEvictedCount())
.addGauge(Interns.info(MOB_FILE_CACHE_HIT_PERCENT, MOB_FILE_CACHE_HIT_PERCENT_DESC),
rsWrap.getMobFileCacheHitPercent())
.addCounter(Interns.info(HEDGED_READS, HEDGED_READS_DESC), rsWrap.getHedgedReadOps())
.addCounter(Interns.info(HEDGED_READ_WINS, HEDGED_READ_WINS_DESC),
rsWrap.getHedgedReadWins())
.addCounter(Interns.info(BLOCKED_REQUESTS_COUNT, BLOCKED_REQUESTS_COUNT_DESC),
rsWrap.getBlockedRequestsCount())
.tag(Interns.info(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC),
rsWrap.getZookeeperQuorum())
.tag(Interns.info(SERVER_NAME_NAME, SERVER_NAME_DESC), rsWrap.getServerName())
.tag(Interns.info(CLUSTER_ID_NAME, CLUSTER_ID_DESC), rsWrap.getClusterId());
addGaugesToMetricsRecordBuilder(mrb)
.addCounter(Interns.info(TOTAL_REQUEST_COUNT, TOTAL_REQUEST_COUNT_DESC),
rsWrap.getTotalRequestCount())
.addCounter(Interns.info(TOTAL_ROW_ACTION_REQUEST_COUNT,
TOTAL_ROW_ACTION_REQUEST_COUNT_DESC), rsWrap.getTotalRowActionRequestCount())
.addCounter(Interns.info(READ_REQUEST_COUNT, READ_REQUEST_COUNT_DESC),
rsWrap.getReadRequestsCount())
.addCounter(Interns.info(FILTERED_READ_REQUEST_COUNT,
FILTERED_READ_REQUEST_COUNT_DESC), rsWrap.getFilteredReadRequestsCount())
.addCounter(Interns.info(WRITE_REQUEST_COUNT, WRITE_REQUEST_COUNT_DESC),
rsWrap.getWriteRequestsCount())
.addCounter(Interns.info(RPC_GET_REQUEST_COUNT, RPC_GET_REQUEST_COUNT_DESC),
rsWrap.getRpcGetRequestsCount())
.addCounter(Interns.info(RPC_SCAN_REQUEST_COUNT, RPC_SCAN_REQUEST_COUNT_DESC),
rsWrap.getRpcScanRequestsCount())
.addCounter(Interns.info(RPC_MULTI_REQUEST_COUNT, RPC_MULTI_REQUEST_COUNT_DESC),
rsWrap.getRpcMultiRequestsCount())
.addCounter(Interns.info(RPC_MUTATE_REQUEST_COUNT, RPC_MUTATE_REQUEST_COUNT_DESC),
rsWrap.getRpcMutateRequestsCount())
.addCounter(Interns.info(CHECK_MUTATE_FAILED_COUNT, CHECK_MUTATE_FAILED_COUNT_DESC),
rsWrap.getCheckAndMutateChecksFailed())
.addCounter(Interns.info(CHECK_MUTATE_PASSED_COUNT, CHECK_MUTATE_PASSED_COUNT_DESC),
rsWrap.getCheckAndMutateChecksPassed())
.addCounter(Interns.info(BLOCK_CACHE_HIT_COUNT, BLOCK_CACHE_HIT_COUNT_DESC),
rsWrap.getBlockCacheHitCount())
.addCounter(Interns.info(BLOCK_CACHE_PRIMARY_HIT_COUNT,
BLOCK_CACHE_PRIMARY_HIT_COUNT_DESC), rsWrap.getBlockCachePrimaryHitCount())
.addCounter(Interns.info(BLOCK_CACHE_MISS_COUNT, BLOCK_COUNT_MISS_COUNT_DESC),
rsWrap.getBlockCacheMissCount())
.addCounter(Interns.info(BLOCK_CACHE_PRIMARY_MISS_COUNT,
BLOCK_COUNT_PRIMARY_MISS_COUNT_DESC), rsWrap.getBlockCachePrimaryMissCount())
.addCounter(Interns.info(BLOCK_CACHE_EVICTION_COUNT, BLOCK_CACHE_EVICTION_COUNT_DESC),
rsWrap.getBlockCacheEvictedCount())
.addCounter(Interns.info(BLOCK_CACHE_PRIMARY_EVICTION_COUNT,
BLOCK_CACHE_PRIMARY_EVICTION_COUNT_DESC),
rsWrap.getBlockCachePrimaryEvictedCount())
.addCounter(Interns.info(BLOCK_CACHE_FAILED_INSERTION_COUNT,
BLOCK_CACHE_FAILED_INSERTION_COUNT_DESC),
rsWrap.getBlockCacheFailedInsertions())
.addCounter(Interns.info(BLOCK_CACHE_DATA_MISS_COUNT, ""),
rsWrap.getDataMissCount())
.addCounter(Interns.info(BLOCK_CACHE_LEAF_INDEX_MISS_COUNT, ""),
rsWrap.getLeafIndexMissCount())
.addCounter(Interns.info(BLOCK_CACHE_BLOOM_CHUNK_MISS_COUNT, ""),
rsWrap.getBloomChunkMissCount())
.addCounter(Interns.info(BLOCK_CACHE_META_MISS_COUNT, ""),
rsWrap.getMetaMissCount())
.addCounter(Interns.info(BLOCK_CACHE_ROOT_INDEX_MISS_COUNT, ""),
rsWrap.getRootIndexMissCount())
.addCounter(Interns.info(BLOCK_CACHE_INTERMEDIATE_INDEX_MISS_COUNT, ""),
rsWrap.getIntermediateIndexMissCount())
.addCounter(Interns.info(BLOCK_CACHE_FILE_INFO_MISS_COUNT, ""),
rsWrap.getFileInfoMissCount())
.addCounter(Interns.info(BLOCK_CACHE_GENERAL_BLOOM_META_MISS_COUNT, ""),
rsWrap.getGeneralBloomMetaMissCount())
.addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_MISS_COUNT, ""),
rsWrap.getDeleteFamilyBloomMissCount())
.addCounter(Interns.info(BLOCK_CACHE_TRAILER_MISS_COUNT, ""),
rsWrap.getTrailerMissCount())
.addCounter(Interns.info(BLOCK_CACHE_DATA_HIT_COUNT, ""),
rsWrap.getDataHitCount())
.addCounter(Interns.info(BLOCK_CACHE_LEAF_INDEX_HIT_COUNT, ""),
rsWrap.getLeafIndexHitCount())
.addCounter(Interns.info(BLOCK_CACHE_BLOOM_CHUNK_HIT_COUNT, ""),
rsWrap.getBloomChunkHitCount())
.addCounter(Interns.info(BLOCK_CACHE_META_HIT_COUNT, ""),
rsWrap.getMetaHitCount())
.addCounter(Interns.info(BLOCK_CACHE_ROOT_INDEX_HIT_COUNT, ""),
rsWrap.getRootIndexHitCount())
.addCounter(Interns.info(BLOCK_CACHE_INTERMEDIATE_INDEX_HIT_COUNT, ""),
rsWrap.getIntermediateIndexHitCount())
.addCounter(Interns.info(BLOCK_CACHE_FILE_INFO_HIT_COUNT, ""),
rsWrap.getFileInfoHitCount())
.addCounter(Interns.info(BLOCK_CACHE_GENERAL_BLOOM_META_HIT_COUNT, ""),
rsWrap.getGeneralBloomMetaHitCount())
.addCounter(Interns.info(BLOCK_CACHE_DELETE_FAMILY_BLOOM_HIT_COUNT, ""),
rsWrap.getDeleteFamilyBloomHitCount())
.addCounter(Interns.info(BLOCK_CACHE_TRAILER_HIT_COUNT, ""),
rsWrap.getTrailerHitCount())
.addCounter(Interns.info(UPDATES_BLOCKED_TIME, UPDATES_BLOCKED_DESC),
rsWrap.getUpdatesBlockedTime())
.addCounter(Interns.info(FLUSHED_CELLS, FLUSHED_CELLS_DESC),
rsWrap.getFlushedCellsCount())
.addCounter(Interns.info(COMPACTED_CELLS, COMPACTED_CELLS_DESC),
rsWrap.getCompactedCellsCount())
.addCounter(Interns.info(MAJOR_COMPACTED_CELLS, MAJOR_COMPACTED_CELLS_DESC),
rsWrap.getMajorCompactedCellsCount())
.addCounter(Interns.info(FLUSHED_CELLS_SIZE, FLUSHED_CELLS_SIZE_DESC),
rsWrap.getFlushedCellsSize())
.addCounter(Interns.info(COMPACTED_CELLS_SIZE, COMPACTED_CELLS_SIZE_DESC),
rsWrap.getCompactedCellsSize())
.addCounter(Interns.info(MAJOR_COMPACTED_CELLS_SIZE, MAJOR_COMPACTED_CELLS_SIZE_DESC),
rsWrap.getMajorCompactedCellsSize())
.addCounter(Interns.info(CELLS_COUNT_COMPACTED_FROM_MOB,
CELLS_COUNT_COMPACTED_FROM_MOB_DESC), rsWrap.getCellsCountCompactedFromMob())
.addCounter(Interns.info(CELLS_COUNT_COMPACTED_TO_MOB,
CELLS_COUNT_COMPACTED_TO_MOB_DESC), rsWrap.getCellsCountCompactedToMob())
.addCounter(Interns.info(CELLS_SIZE_COMPACTED_FROM_MOB,
CELLS_SIZE_COMPACTED_FROM_MOB_DESC), rsWrap.getCellsSizeCompactedFromMob())
.addCounter(Interns.info(CELLS_SIZE_COMPACTED_TO_MOB,
CELLS_SIZE_COMPACTED_TO_MOB_DESC), rsWrap.getCellsSizeCompactedToMob())
.addCounter(Interns.info(MOB_FLUSH_COUNT, MOB_FLUSH_COUNT_DESC),
rsWrap.getMobFlushCount())
.addCounter(Interns.info(MOB_FLUSHED_CELLS_COUNT, MOB_FLUSHED_CELLS_COUNT_DESC),
rsWrap.getMobFlushedCellsCount())
.addCounter(Interns.info(MOB_FLUSHED_CELLS_SIZE, MOB_FLUSHED_CELLS_SIZE_DESC),
rsWrap.getMobFlushedCellsSize())
.addCounter(Interns.info(MOB_SCAN_CELLS_COUNT, MOB_SCAN_CELLS_COUNT_DESC),
rsWrap.getMobScanCellsCount())
.addCounter(Interns.info(MOB_SCAN_CELLS_SIZE, MOB_SCAN_CELLS_SIZE_DESC),
rsWrap.getMobScanCellsSize())
.addCounter(Interns.info(MOB_FILE_CACHE_ACCESS_COUNT,
MOB_FILE_CACHE_ACCESS_COUNT_DESC), rsWrap.getMobFileCacheAccessCount())
.addCounter(Interns.info(MOB_FILE_CACHE_MISS_COUNT, MOB_FILE_CACHE_MISS_COUNT_DESC),
rsWrap.getMobFileCacheMissCount())
.addCounter(Interns.info(MOB_FILE_CACHE_EVICTED_COUNT,
MOB_FILE_CACHE_EVICTED_COUNT_DESC), rsWrap.getMobFileCacheEvictedCount())
.addCounter(Interns.info(HEDGED_READS, HEDGED_READS_DESC), rsWrap.getHedgedReadOps())
.addCounter(Interns.info(HEDGED_READ_WINS, HEDGED_READ_WINS_DESC),
rsWrap.getHedgedReadWins())
.addCounter(Interns.info(BLOCKED_REQUESTS_COUNT, BLOCKED_REQUESTS_COUNT_DESC),
rsWrap.getBlockedRequestsCount())
.tag(Interns.info(ZOOKEEPER_QUORUM_NAME, ZOOKEEPER_QUORUM_DESC),
rsWrap.getZookeeperQuorum())
.tag(Interns.info(SERVER_NAME_NAME, SERVER_NAME_DESC), rsWrap.getServerName())
.tag(Interns.info(CLUSTER_ID_NAME, CLUSTER_ID_DESC), rsWrap.getClusterId());
}
metricsRegistry.snapshot(mrb, all);
@ -543,6 +470,83 @@ public class MetricsRegionServerSourceImpl
}
}
private MetricsRecordBuilder addGaugesToMetricsRecordBuilder(MetricsRecordBuilder mrb) {
return mrb.addGauge(Interns.info(REGION_COUNT, REGION_COUNT_DESC), rsWrap.getNumOnlineRegions())
.addGauge(Interns.info(STORE_COUNT, STORE_COUNT_DESC), rsWrap.getNumStores())
.addGauge(Interns.info(WALFILE_COUNT, WALFILE_COUNT_DESC), rsWrap.getNumWALFiles())
.addGauge(Interns.info(WALFILE_SIZE, WALFILE_SIZE_DESC), rsWrap.getWALFileSize())
.addGauge(Interns.info(STOREFILE_COUNT, STOREFILE_COUNT_DESC),
rsWrap.getNumStoreFiles())
.addGauge(Interns.info(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC), rsWrap.getMemStoreSize())
.addGauge(Interns.info(STOREFILE_SIZE, STOREFILE_SIZE_DESC), rsWrap.getStoreFileSize())
.addGauge(Interns.info(MAX_STORE_FILE_AGE, MAX_STORE_FILE_AGE_DESC),
rsWrap.getMaxStoreFileAge())
.addGauge(Interns.info(MIN_STORE_FILE_AGE, MIN_STORE_FILE_AGE_DESC),
rsWrap.getMinStoreFileAge())
.addGauge(Interns.info(AVG_STORE_FILE_AGE, AVG_STORE_FILE_AGE_DESC),
rsWrap.getAvgStoreFileAge())
.addGauge(Interns.info(NUM_REFERENCE_FILES, NUM_REFERENCE_FILES_DESC),
rsWrap.getNumReferenceFiles())
.addGauge(Interns.info(RS_START_TIME_NAME, RS_START_TIME_DESC), rsWrap.getStartCode())
.addGauge(Interns.info(AVERAGE_REGION_SIZE, AVERAGE_REGION_SIZE_DESC),
rsWrap.getAverageRegionSize())
.addGauge(Interns.info(STOREFILE_INDEX_SIZE, STOREFILE_INDEX_SIZE_DESC),
rsWrap.getStoreFileIndexSize())
.addGauge(Interns.info(STATIC_INDEX_SIZE, STATIC_INDEX_SIZE_DESC),
rsWrap.getTotalStaticIndexSize())
.addGauge(Interns.info(STATIC_BLOOM_SIZE, STATIC_BLOOM_SIZE_DESC),
rsWrap.getTotalStaticBloomSize())
.addGauge(Interns.info(NUMBER_OF_MUTATIONS_WITHOUT_WAL,
NUMBER_OF_MUTATIONS_WITHOUT_WAL_DESC), rsWrap.getNumMutationsWithoutWAL())
.addGauge(Interns.info(DATA_SIZE_WITHOUT_WAL, DATA_SIZE_WITHOUT_WAL_DESC),
rsWrap.getDataInMemoryWithoutWAL())
.addGauge(Interns.info(PERCENT_FILES_LOCAL, PERCENT_FILES_LOCAL_DESC),
rsWrap.getPercentFileLocal())
.addGauge(Interns.info(PERCENT_FILES_LOCAL_SECONDARY_REGIONS,
PERCENT_FILES_LOCAL_SECONDARY_REGIONS_DESC),
rsWrap.getPercentFileLocalSecondaryRegions())
.addGauge(Interns.info(SPLIT_QUEUE_LENGTH, SPLIT_QUEUE_LENGTH_DESC),
rsWrap.getSplitQueueSize())
.addGauge(Interns.info(COMPACTION_QUEUE_LENGTH, COMPACTION_QUEUE_LENGTH_DESC),
rsWrap.getCompactionQueueSize())
.addGauge(Interns.info(SMALL_COMPACTION_QUEUE_LENGTH,
SMALL_COMPACTION_QUEUE_LENGTH_DESC), rsWrap.getSmallCompactionQueueSize())
.addGauge(Interns.info(LARGE_COMPACTION_QUEUE_LENGTH,
LARGE_COMPACTION_QUEUE_LENGTH_DESC), rsWrap.getLargeCompactionQueueSize())
.addGauge(Interns.info(FLUSH_QUEUE_LENGTH, FLUSH_QUEUE_LENGTH_DESC),
rsWrap.getFlushQueueSize())
.addGauge(Interns.info(BLOCK_CACHE_FREE_SIZE, BLOCK_CACHE_FREE_DESC),
rsWrap.getBlockCacheFreeSize())
.addGauge(Interns.info(BLOCK_CACHE_COUNT, BLOCK_CACHE_COUNT_DESC),
rsWrap.getBlockCacheCount())
.addGauge(Interns.info(BLOCK_CACHE_SIZE, BLOCK_CACHE_SIZE_DESC),
rsWrap.getBlockCacheSize())
.addGauge(Interns.info(BLOCK_CACHE_HIT_PERCENT, BLOCK_CACHE_HIT_PERCENT_DESC),
rsWrap.getBlockCacheHitPercent())
.addGauge(Interns.info(BLOCK_CACHE_EXPRESS_HIT_PERCENT,
BLOCK_CACHE_EXPRESS_HIT_PERCENT_DESC), rsWrap.getBlockCacheHitCachingPercent())
.addGauge(Interns.info(L1_CACHE_HIT_COUNT, L1_CACHE_HIT_COUNT_DESC),
rsWrap.getL1CacheHitCount())
.addGauge(Interns.info(L1_CACHE_MISS_COUNT, L1_CACHE_MISS_COUNT_DESC),
rsWrap.getL1CacheMissCount())
.addGauge(Interns.info(L1_CACHE_HIT_RATIO, L1_CACHE_HIT_RATIO_DESC),
rsWrap.getL1CacheHitRatio())
.addGauge(Interns.info(L1_CACHE_MISS_RATIO, L1_CACHE_MISS_RATIO_DESC),
rsWrap.getL1CacheMissRatio())
.addGauge(Interns.info(L2_CACHE_HIT_COUNT, L2_CACHE_HIT_COUNT_DESC),
rsWrap.getL2CacheHitCount())
.addGauge(Interns.info(L2_CACHE_MISS_COUNT, L2_CACHE_MISS_COUNT_DESC),
rsWrap.getL2CacheMissCount())
.addGauge(Interns.info(L2_CACHE_HIT_RATIO, L2_CACHE_HIT_RATIO_DESC),
rsWrap.getL2CacheHitRatio())
.addGauge(Interns.info(L2_CACHE_MISS_RATIO, L2_CACHE_MISS_RATIO_DESC),
rsWrap.getL2CacheMissRatio())
.addGauge(Interns.info(MOB_FILE_CACHE_COUNT, MOB_FILE_CACHE_COUNT_DESC),
rsWrap.getMobFileCacheCount())
.addGauge(Interns.info(MOB_FILE_CACHE_HIT_PERCENT, MOB_FILE_CACHE_HIT_PERCENT_DESC),
rsWrap.getMobFileCacheHitPercent());
}
@Override
public void incInfoThresholdExceeded(int count) {
infoPauseThresholdExceeded.incr(count);

View File

@ -20,13 +20,13 @@ package org.apache.hadoop.hbase.regionserver;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class MetricsRegionSourceImpl implements MetricsRegionSource {

View File

@ -20,17 +20,17 @@ package org.apache.hadoop.hbase.regionserver;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class MetricsTableAggregateSourceImpl extends BaseSourceImpl
implements MetricsTableAggregateSource {
implements MetricsTableAggregateSource {
private static final Logger LOG = LoggerFactory.getLogger(MetricsTableAggregateSourceImpl.class);
private ConcurrentHashMap<String, MetricsTableSource> tableSources = new ConcurrentHashMap<>();

View File

@ -21,12 +21,12 @@ package org.apache.hadoop.hbase.regionserver;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.hbase.TableName;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.lib.DynamicMetricsRegistry;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@InterfaceAudience.Private
public class MetricsTableSourceImpl implements MetricsTableSource {
@ -156,9 +156,15 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
return (o instanceof MetricsTableSourceImpl && compareTo((MetricsTableSourceImpl) o) == 0);
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return (compareTo((MetricsTableSourceImpl) o) == 0);
}
public MetricsTableWrapperAggregate getTableWrapper() {

View File

@ -18,17 +18,16 @@
package org.apache.hadoop.hbase.regionserver.wal;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Class that transitions metrics from MetricsWAL into the metrics subsystem.
*
* Implements BaseSource through BaseSourceImpl, following the pattern.
* @see org.apache.hadoop.hbase.regionserver.wal.MetricsWAL
* @see org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource
*/
@InterfaceAudience.Private
public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSource {
@ -55,15 +54,15 @@ public class MetricsWALSourceImpl extends BaseSourceImpl implements MetricsWALSo
//Create and store the metrics that will be used.
appendTimeHisto = this.getMetricsRegistry().newTimeHistogram(APPEND_TIME, APPEND_TIME_DESC);
appendSizeHisto = this.getMetricsRegistry().newSizeHistogram(APPEND_SIZE, APPEND_SIZE_DESC);
appendCount = this.getMetricsRegistry().newCounter(APPEND_COUNT, APPEND_COUNT_DESC, 0l);
appendCount = this.getMetricsRegistry().newCounter(APPEND_COUNT, APPEND_COUNT_DESC, 0L);
slowAppendCount =
this.getMetricsRegistry().newCounter(SLOW_APPEND_COUNT, SLOW_APPEND_COUNT_DESC, 0l);
this.getMetricsRegistry().newCounter(SLOW_APPEND_COUNT, SLOW_APPEND_COUNT_DESC, 0L);
syncTimeHisto = this.getMetricsRegistry().newTimeHistogram(SYNC_TIME, SYNC_TIME_DESC);
logRollRequested =
this.getMetricsRegistry().newCounter(ROLL_REQUESTED, ROLL_REQUESTED_DESC, 0L);
lowReplicationLogRollRequested = this.getMetricsRegistry()
.newCounter(LOW_REPLICA_ROLL_REQUESTED, LOW_REPLICA_ROLL_REQUESTED_DESC, 0L);
writtenBytes = this.getMetricsRegistry().newCounter(WRITTEN_BYTES, WRITTEN_BYTES_DESC, 0l);
writtenBytes = this.getMetricsRegistry().newCounter(WRITTEN_BYTES, WRITTEN_BYTES_DESC, 0L);
}
@Override

View File

@ -36,6 +36,10 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
private final MutableFastCounter shippedBatchesCounter;
private final MutableFastCounter shippedOpsCounter;
private final MutableFastCounter shippedBytesCounter;
/**
* @deprecated since 1.3.0. Use {@link #shippedBytesCounter} instead.
*/
@Deprecated
private final MutableFastCounter shippedKBsCounter;
private final MutableFastCounter logReadInBytesCounter;
@ -75,13 +79,16 @@ public class MetricsReplicationGlobalSourceSource implements MetricsReplicationS
sizeOfHFileRefsQueueGauge =
rms.getMetricsRegistry().getGauge(SOURCE_SIZE_OF_HFILE_REFS_QUEUE, 0L);
unknownFileLengthForClosedWAL = rms.getMetricsRegistry().getCounter(SOURCE_CLOSED_LOGS_WITH_UNKNOWN_LENGTH, 0L);
unknownFileLengthForClosedWAL = rms.getMetricsRegistry()
.getCounter(SOURCE_CLOSED_LOGS_WITH_UNKNOWN_LENGTH, 0L);
uncleanlyClosedWAL = rms.getMetricsRegistry().getCounter(SOURCE_UNCLEANLY_CLOSED_LOGS, 0L);
uncleanlyClosedSkippedBytes = rms.getMetricsRegistry().getCounter(SOURCE_UNCLEANLY_CLOSED_IGNORED_IN_BYTES, 0L);
uncleanlyClosedSkippedBytes = rms.getMetricsRegistry()
.getCounter(SOURCE_UNCLEANLY_CLOSED_IGNORED_IN_BYTES, 0L);
restartWALReading = rms.getMetricsRegistry().getCounter(SOURCE_RESTARTED_LOG_READING, 0L);
repeatedFileBytes = rms.getMetricsRegistry().getCounter(SOURCE_REPEATED_LOG_FILE_BYTES, 0L);
completedWAL = rms.getMetricsRegistry().getCounter(SOURCE_COMPLETED_LOGS, 0L);
completedRecoveryQueue = rms.getMetricsRegistry().getCounter(SOURCE_COMPLETED_RECOVERY_QUEUES, 0L);
completedRecoveryQueue = rms.getMetricsRegistry()
.getCounter(SOURCE_COMPLETED_RECOVERY_QUEUES, 0L);
}
@Override public void setLastShippedAge(long age) {

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.replication.regionserver;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop2 implementation of MetricsReplicationSource. This provides access to metrics gauges and

View File

@ -35,6 +35,9 @@ public class MetricsReplicationSourceSourceImpl implements MetricsReplicationSou
private final String shippedOpsKey;
private String keyPrefix;
/**
* @deprecated since 1.3.0. Use {@link #shippedBytesKey} instead.
*/
@Deprecated
private final String shippedKBsKey;
private final String shippedBytesKey;

View File

@ -18,10 +18,10 @@
package org.apache.hadoop.hbase.rest;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop Two implementation of a metrics2 source that will export metrics from the Rest server to
@ -31,7 +31,6 @@ import org.apache.hadoop.metrics2.lib.MutableFastCounter;
*/
@InterfaceAudience.Private
public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsRESTSource {
private MutableFastCounter request;
private MutableFastCounter sucGet;
private MutableFastCounter sucPut;
@ -74,20 +73,20 @@ public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsREST
@Override
public void init() {
super.init();
request = getMetricsRegistry().getCounter(REQUEST_KEY, 0l);
request = getMetricsRegistry().getCounter(REQUEST_KEY, 0L);
sucGet = getMetricsRegistry().getCounter(SUCCESSFUL_GET_KEY, 0l);
sucPut = getMetricsRegistry().getCounter(SUCCESSFUL_PUT_KEY, 0l);
sucDel = getMetricsRegistry().getCounter(SUCCESSFUL_DELETE_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);
sucAppend = getMetricsRegistry().getCounter(SUCCESSFUL_APPEND_KEY, 0l);
sucAppend = getMetricsRegistry().getCounter(SUCCESSFUL_APPEND_KEY, 0L);
sucIncrement = getMetricsRegistry().getCounter(SUCCESSFUL_INCREMENT_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);
fAppend = getMetricsRegistry().getCounter(FAILED_APPEND_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);
fAppend = getMetricsRegistry().getCounter(FAILED_APPEND_KEY, 0L);
fIncrement = getMetricsRegistry().getCounter(FAILED_INCREMENT_KEY, 0L);
}
@ -133,7 +132,7 @@ public class MetricsRESTSourceImpl extends BaseSourceImpl implements MetricsREST
@Override
public void incrementFailedScanRequests(int inc) {
fScan.incr(inc);
fScan.incr(inc);
}
@Override

View File

@ -18,16 +18,15 @@
package org.apache.hadoop.hbase.thrift;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.hbase.metrics.ExceptionTrackingSourceImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.lib.MutableFastCounter;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Hadoop 2 version of MetricsThriftServerSource{@link org.apache.hadoop.hbase.thrift.MetricsThriftServerSource}
* Hadoop 2 version of {@link org.apache.hadoop.hbase.thrift.MetricsThriftServerSource}
*
* Implements BaseSource through BaseSourceImpl, following the pattern
*/

View File

@ -18,11 +18,11 @@
package org.apache.hadoop.hbase.zookeeper;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
import org.apache.hadoop.metrics2.MetricsCollector;
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
import org.apache.hadoop.metrics2.lib.MutableHistogram;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Class that transitions metrics from MetricsZooKeeper into the metrics subsystem.
@ -51,25 +51,38 @@ public class MetricsZooKeeperSourceImpl extends BaseSourceImpl implements Metric
this(METRICS_NAME, METRICS_DESCRIPTION, METRICS_CONTEXT, METRICS_JMX_CONTEXT);
}
public MetricsZooKeeperSourceImpl(String metricsName, String metricsDescription, String metricsContext,
String metricsJmxContext) {
public MetricsZooKeeperSourceImpl(String metricsName, String metricsDescription,
String metricsContext, String metricsJmxContext) {
super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
//Create and store the metrics that will be used.
authFailedFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_AUTHFAILED, EXCEPTION_AUTHFAILED_DESC, 0L);
connectionLossFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_CONNECTIONLOSS, EXCEPTION_CONNECTIONLOSS_DESC, 0L);
dataInconsistencyFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_DATAINCONSISTENCY, EXCEPTION_DATAINCONSISTENCY_DESC, 0L);
invalidACLFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_INVALIDACL, EXCEPTION_INVALIDACL_DESC, 0L);
noAuthFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_NOAUTH, EXCEPTION_NOAUTH_DESC, 0L);
operationTimeOutFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_OPERATIONTIMEOUT, EXCEPTION_OPERATIONTIMEOUT_DESC, 0L);
runtimeInconsistencyFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_RUNTIMEINCONSISTENCY, EXCEPTION_RUNTIMEINCONSISTENCY_DESC, 0L);
sessionExpiredFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_SESSIONEXPIRED, EXCEPTION_SESSIONEXPIRED_DESC, 0L);
systemErrorFailedOpCount = this.getMetricsRegistry().newGauge(EXCEPTION_SYSTEMERROR, EXCEPTION_SYSTEMERROR_DESC, 0L);
totalFailedZKCalls = this.getMetricsRegistry().newGauge(TOTAL_FAILED_ZK_CALLS, TOTAL_FAILED_ZK_CALLS_DESC, 0L);
authFailedFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_AUTHFAILED, EXCEPTION_AUTHFAILED_DESC, 0L);
connectionLossFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_CONNECTIONLOSS, EXCEPTION_CONNECTIONLOSS_DESC, 0L);
dataInconsistencyFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_DATAINCONSISTENCY, EXCEPTION_DATAINCONSISTENCY_DESC, 0L);
invalidACLFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_INVALIDACL, EXCEPTION_INVALIDACL_DESC, 0L);
noAuthFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_NOAUTH, EXCEPTION_NOAUTH_DESC, 0L);
operationTimeOutFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_OPERATIONTIMEOUT, EXCEPTION_OPERATIONTIMEOUT_DESC, 0L);
runtimeInconsistencyFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_RUNTIMEINCONSISTENCY, EXCEPTION_RUNTIMEINCONSISTENCY_DESC, 0L);
sessionExpiredFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_SESSIONEXPIRED, EXCEPTION_SESSIONEXPIRED_DESC, 0L);
systemErrorFailedOpCount = this.getMetricsRegistry().newGauge(
EXCEPTION_SYSTEMERROR, EXCEPTION_SYSTEMERROR_DESC, 0L);
totalFailedZKCalls = this.getMetricsRegistry().newGauge(
TOTAL_FAILED_ZK_CALLS, TOTAL_FAILED_ZK_CALLS_DESC, 0L);
readOpLatency = this.getMetricsRegistry().newHistogram(READ_OPERATION_LATENCY_NAME, READ_OPERATION_LATENCY_DESC);
writeOpLatency = this.getMetricsRegistry().newHistogram(WRITE_OPERATION_LATENCY_NAME, WRITE_OPERATION_LATENCY_DESC);
syncOpLatency = this.getMetricsRegistry().newHistogram(SYNC_OPERATION_LATENCY_NAME, SYNC_OPERATION_LATENCY_DESC);
readOpLatency = this.getMetricsRegistry().newHistogram(
READ_OPERATION_LATENCY_NAME, READ_OPERATION_LATENCY_DESC);
writeOpLatency = this.getMetricsRegistry().newHistogram(
WRITE_OPERATION_LATENCY_NAME, WRITE_OPERATION_LATENCY_DESC);
syncOpLatency = this.getMetricsRegistry().newHistogram(
SYNC_OPERATION_LATENCY_NAME, SYNC_OPERATION_LATENCY_DESC);
}
public void getMetrics(MetricsCollector metricsCollector, boolean all) {

View File

@ -22,13 +22,13 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.MetricsExecutorImpl;
import org.apache.hadoop.util.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
@ -41,7 +41,7 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
* are package private.
*/
@InterfaceAudience.Private
public class JmxCacheBuster {
public final class JmxCacheBuster {
private static final Logger LOG = LoggerFactory.getLogger(JmxCacheBuster.class);
private static AtomicReference<ScheduledFuture> fut = new AtomicReference<>(null);
private static MetricsExecutor executor = new MetricsExecutorImpl();

View File

@ -92,8 +92,8 @@ public class DefaultMetricsSystemHelper {
}
} catch (Exception ex) {
if (LOG.isTraceEnabled()) {
LOG.trace("Received exception while trying to access Hadoop Metrics classes via reflection.",
ex);
LOG.trace("Received exception while trying to access Hadoop Metrics classes via " +
"reflection.", ex);
}
}
}

View File

@ -21,15 +21,15 @@ package org.apache.hadoop.metrics2.lib;
import java.util.Collection;
import java.util.concurrent.ConcurrentMap;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.metrics2.MetricsException;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.impl.MsInfo;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.MoreObjects;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
@ -56,14 +56,14 @@ public class DynamicMetricsRegistry {
private final MetricsInfo metricsInfo;
private final DefaultMetricsSystemHelper helper = new DefaultMetricsSystemHelper();
private final static String[] histogramSuffixes = new String[]{
"_num_ops",
"_min",
"_max",
"_median",
"_75th_percentile",
"_90th_percentile",
"_95th_percentile",
"_99th_percentile"};
"_num_ops",
"_min",
"_max",
"_median",
"_75th_percentile",
"_90th_percentile",
"_95th_percentile",
"_99th_percentile"};
/**
* Construct the registry with a record name
@ -215,7 +215,10 @@ public class DynamicMetricsRegistry {
if (returnExisting) {
MutableMetric rate = metricsMap.get(name);
if (rate != null) {
if (rate instanceof MutableRate) return (MutableRate) rate;
if (rate instanceof MutableRate) {
return (MutableRate) rate;
}
throw new MetricsException("Unexpected metrics type "+ rate.getClass()
+" for "+ name);
}
@ -230,7 +233,7 @@ public class DynamicMetricsRegistry {
* @return A new MutableHistogram
*/
public MutableHistogram newHistogram(String name) {
return newHistogram(name, "");
return newHistogram(name, "");
}
/**
@ -250,7 +253,7 @@ public class DynamicMetricsRegistry {
* @return A new MutableTimeHistogram
*/
public MutableTimeHistogram newTimeHistogram(String name) {
return newTimeHistogram(name, "");
return newTimeHistogram(name, "");
}
/**
@ -270,7 +273,7 @@ public class DynamicMetricsRegistry {
* @return A new MutableSizeHistogram
*/
public MutableSizeHistogram newSizeHistogram(String name) {
return newSizeHistogram(name, "");
return newSizeHistogram(name, "");
}
/**
@ -497,10 +500,8 @@ public class DynamicMetricsRegistry {
return (MutableHistogram) histo;
}
private<T extends MutableMetric> T
addNewMetricIfAbsent(String name,
T ret,
Class<T> metricClass) {
private<T extends MutableMetric> T addNewMetricIfAbsent(String name, T ret,
Class<T> metricClass) {
//If the value we get back is null then the put was successful and we will
// return that. Otherwise metric should contain the thing that was in
// before the put could be completed.

View File

@ -23,8 +23,8 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Class to handle the ScheduledExecutorService{@link ScheduledExecutorService} used by

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.metrics2.lib;
import org.apache.commons.lang3.StringUtils;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.Histogram;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.hbase.metrics.Snapshot;
@ -27,6 +26,7 @@ import org.apache.hadoop.hbase.metrics.impl.HistogramImpl;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.yetus.audience.InterfaceAudience;
/**
* A histogram implementation that runs in constant space, and exports to hadoop2's metrics2 system.

View File

@ -18,12 +18,13 @@
package org.apache.hadoop.metrics2.lib;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.hbase.metrics.Snapshot;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Extended histogram implementation with metric range counters.
*/

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.metrics2.lib;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Extended histogram implementation with counters for metric size ranges.

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.metrics2.lib;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.yetus.audience.InterfaceAudience;
/**
* Extended histogram implementation with counters for metric time ranges.
@ -28,7 +28,7 @@ import org.apache.hadoop.metrics2.MetricsInfo;
public class MutableTimeHistogram extends MutableRangeHistogram {
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 };
{ 1, 3, 10, 30, 100, 300, 1000, 3000, 10000, 30000, 60000, 120000, 300000, 600000 };
public MutableTimeHistogram(MetricsInfo info) {
this(info.name(), info.description());

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -15,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.metrics2.util;
import org.apache.yetus.audience.InterfaceAudience;
@ -26,8 +25,14 @@ import org.apache.yetus.audience.InterfaceAudience;
*/
@InterfaceAudience.Private
public class MetricQuantile {
public final double quantile;
public final double error;
/**
* The quantile to be watched by a {@link MetricSampleQuantiles}.
*/
final double quantile;
/**
* The error bounds for the {@link #quantile}.
*/
final double error;
public MetricQuantile(double quantile, double error) {
this.quantile = quantile;

View File

@ -69,7 +69,7 @@ public class MetricSampleQuantiles {
/**
* Array of Quantiles that we care about, along with desired error.
*/
private final MetricQuantile quantiles[];
private final MetricQuantile[] quantiles;
public MetricSampleQuantiles(MetricQuantile[] quantiles) {
this.quantiles = Arrays.copyOf(quantiles, quantiles.length);
@ -107,7 +107,7 @@ public class MetricSampleQuantiles {
/**
* Add a new value from the stream.
*
* @param v
* @param v the value to insert
*/
synchronized public void insert(long v) {
buffer[bufferCount] = v;
@ -280,7 +280,7 @@ public class MetricSampleQuantiles {
/**
* Value of the sampled item (e.g. a measured latency value)
*/
public final long value;
private final long value;
/**
* Difference between the lowest possible rank of the previous item, and
@ -288,13 +288,13 @@ public class MetricSampleQuantiles {
*
* The sum of the g of all previous items yields this item's lower bound.
*/
public int g;
private int g;
/**
* Difference between the item's greatest possible rank and lowest possible
* rank.
*/
public final int delta;
private final int delta;
public SampleItem(long value, int lowerDelta, int delta) {
this.value = value;

View File

@ -26,12 +26,11 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
* Compatibility shim layer implementation for Hadoop-2.
*/
public class HadoopShimsImpl implements HadoopShims {
/**
* Returns a TaskAttemptContext instance created from the given parameters.
* @param job an instance of o.a.h.mapreduce.Job
* @param taskId an identifier for the task attempt id. Should be parsable by
* TaskAttemptId.forName()
* {@link TaskAttemptID#forName(String)}
* @return a concrete TaskAttemptContext instance of o.a.h.mapreduce.TaskAttemptContext
*/
@Override

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -33,18 +33,17 @@ import org.junit.experimental.categories.Category;
*/
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsMasterSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsMasterSourceImpl.class);
@Test
public void testGetInstance() throws Exception {
public void testGetInstance() {
MetricsMasterSourceFactory metricsMasterSourceFactory = CompatibilitySingletonFactory
.getInstance(MetricsMasterSourceFactory.class);
MetricsMasterSource masterSource = metricsMasterSourceFactory.create(null);
assertTrue(masterSource instanceof MetricsMasterSourceImpl);
assertSame(metricsMasterSourceFactory, CompatibilitySingletonFactory.getInstance(MetricsMasterSourceFactory.class));
assertSame(metricsMasterSourceFactory, CompatibilitySingletonFactory.getInstance(
MetricsMasterSourceFactory.class));
}
}

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -28,18 +28,14 @@ import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test for MetricsRegionServerSourceImpl
*/
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsRegionServerSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsRegionServerSourceImpl.class);
@Test
public void testGetInstance() throws Exception {
public void testGetInstance() {
MetricsRegionServerSourceFactory metricsRegionServerSourceFactory =
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class);
MetricsRegionServerSource serverSource =
@ -51,7 +47,7 @@ public class TestMetricsRegionServerSourceImpl {
@Test(expected = RuntimeException.class)
public void testNoGetRegionServerMetricsSourceImpl() throws Exception {
public void testNoGetRegionServerMetricsSourceImpl() {
// This should throw an exception because MetricsRegionServerSourceImpl should only
// be created by a factory.
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceImpl.class);

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -19,7 +19,6 @@ package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
@ -31,15 +30,15 @@ import org.junit.experimental.categories.Category;
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsRegionSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsRegionSourceImpl.class);
@SuppressWarnings("SelfComparison")
@Test
public void testCompareToHashCodeEquals() throws Exception {
MetricsRegionServerSourceFactory fact = CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class);
public void testCompareToHashCodeEquals() {
MetricsRegionServerSourceFactory fact = CompatibilitySingletonFactory.getInstance(
MetricsRegionServerSourceFactory.class);
MetricsRegionSource one = fact.createRegion(new RegionWrapperStub("TEST"));
MetricsRegionSource oneClone = fact.createRegion(new RegionWrapperStub("TEST"));
@ -49,15 +48,14 @@ public class TestMetricsRegionSourceImpl {
assertEquals(one.hashCode(), oneClone.hashCode());
assertNotEquals(one, two);
assertTrue( one.compareTo(two) != 0);
assertTrue( two.compareTo(one) != 0);
assertTrue( two.compareTo(one) != one.compareTo(two));
assertTrue( two.compareTo(two) == 0);
assertNotEquals(0, one.compareTo(two));
assertNotEquals(0, two.compareTo(one));
assertNotEquals(one.compareTo(two), two.compareTo(one));
assertEquals(0, two.compareTo(two));
}
@Test(expected = RuntimeException.class)
public void testNoGetRegionServerMetricsSourceImpl() throws Exception {
public void testNoGetRegionServerMetricsSourceImpl() {
// This should throw an exception because MetricsRegionSourceImpl should only
// be created by a factory.
CompatibilitySingletonFactory.getInstance(MetricsRegionSource.class);
@ -67,7 +65,7 @@ public class TestMetricsRegionSourceImpl {
private String regionName;
public RegionWrapperStub(String regionName) {
RegionWrapperStub(String regionName) {
this.regionName = regionName;
}

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -34,7 +34,6 @@ import org.junit.experimental.categories.Category;
*/
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsTableSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsTableSourceImpl.class);
@ -46,7 +45,8 @@ public class TestMetricsTableSourceImpl {
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class);
MetricsTableSource one = metricsFact.createTable("ONETABLE", new TableWrapperStub("ONETABLE"));
MetricsTableSource oneClone = metricsFact.createTable("ONETABLE", new TableWrapperStub("ONETABLE"));
MetricsTableSource oneClone = metricsFact.createTable("ONETABLE",
new TableWrapperStub("ONETABLE"));
MetricsTableSource two = metricsFact.createTable("TWOTABLE", new TableWrapperStub("TWOTABLE"));
assertEquals(0, one.compareTo(oneClone));
@ -60,14 +60,14 @@ public class TestMetricsTableSourceImpl {
}
@Test(expected = RuntimeException.class)
public void testNoGetTableMetricsSourceImpl() throws Exception {
public void testNoGetTableMetricsSourceImpl() {
// This should throw an exception because MetricsTableSourceImpl should only
// be created by a factory.
CompatibilitySingletonFactory.getInstance(MetricsTableSourceImpl.class);
}
@Test
public void testGetTableMetrics() throws Exception{
public void testGetTableMetrics() {
MetricsTableSource oneTbl =
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
.createTable("ONETABLE", new TableWrapperStub("ONETABLE"));
@ -75,7 +75,6 @@ public class TestMetricsTableSourceImpl {
}
static class TableWrapperStub implements MetricsTableWrapperAggregate {
private String tableName;
public TableWrapperStub(String tableName) {

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -17,7 +17,7 @@
*/
package org.apache.hadoop.hbase.replication.regionserver;
import static org.junit.Assert.*;
import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
@ -29,17 +29,14 @@ import org.junit.experimental.categories.Category;
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsReplicationSourceFactoryImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsReplicationSourceFactoryImpl.class);
@Test
public void testGetInstance() throws Exception {
public void testGetInstance() {
MetricsReplicationSourceFactory rms = CompatibilitySingletonFactory
.getInstance(MetricsReplicationSourceFactory.class);
assertTrue(rms instanceof MetricsReplicationSourceFactoryImpl);
}
}

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -21,8 +21,6 @@ import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSource;
import org.apache.hadoop.hbase.replication.regionserver.MetricsReplicationSourceImpl;
import org.apache.hadoop.hbase.testclassification.MetricsTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.ClassRule;
@ -30,9 +28,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({MetricsTests.class, SmallTests.class})
/** Test for MetricsReplicationSourceImpl */
public class TestMetricsReplicationSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsReplicationSourceImpl.class);

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -22,8 +22,6 @@ import static org.junit.Assert.assertTrue;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.rest.MetricsRESTSource;
import org.apache.hadoop.hbase.rest.MetricsRESTSourceImpl;
import org.apache.hadoop.hbase.testclassification.MetricsTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.ClassRule;
@ -31,19 +29,18 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test for hadoop 2's version of MetricsRESTSource
* Test for hadoop 2's version of {@link MetricsRESTSource}.
*/
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsRESTSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsRESTSourceImpl.class);
@Test
public void ensureCompatRegistered() throws Exception {
public void ensureCompatRegistered() {
assertNotNull(CompatibilitySingletonFactory.getInstance(MetricsRESTSource.class));
assertTrue(CompatibilitySingletonFactory.getInstance(MetricsRESTSource.class) instanceof MetricsRESTSourceImpl);
assertTrue(CompatibilitySingletonFactory.getInstance(MetricsRESTSource.class)
instanceof MetricsRESTSourceImpl);
}
}

View File

@ -18,6 +18,14 @@
package org.apache.hadoop.hbase.test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import org.apache.hadoop.hbase.metrics.BaseSource;
import org.apache.hadoop.metrics2.AbstractMetric;
import org.apache.hadoop.metrics2.MetricsCollector;
@ -27,12 +35,6 @@ import org.apache.hadoop.metrics2.MetricsSource;
import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import static org.junit.Assert.*;
/**
* A helper class that will allow tests to get into hadoop2's metrics2 values.
*/

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -25,30 +25,29 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.testclassification.MetricsTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactory;
import org.apache.hadoop.hbase.thrift.MetricsThriftServerSourceFactoryImpl;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test for hadoop 2's version of MetricsThriftServerSourceFactory
* Test for hadoop 2's version of MetricsThriftServerSourceFactory.
*/
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsThriftServerSourceFactoryImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsThriftServerSourceFactoryImpl.class);
@Test
public void testCompatabilityRegistered() throws Exception {
assertNotNull(CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class));
assertTrue(CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class) instanceof MetricsThriftServerSourceFactoryImpl);
public void testCompatabilityRegistered() {
assertNotNull(CompatibilitySingletonFactory.getInstance(
MetricsThriftServerSourceFactory.class));
assertTrue(CompatibilitySingletonFactory.getInstance(MetricsThriftServerSourceFactory.class)
instanceof MetricsThriftServerSourceFactoryImpl);
}
@Test
public void testCreateThriftOneSource() throws Exception {
public void testCreateThriftOneSource() {
//Make sure that the factory gives back a singleton.
assertSame(new MetricsThriftServerSourceFactoryImpl().createThriftOneSource(),
new MetricsThriftServerSourceFactoryImpl().createThriftOneSource());
@ -56,7 +55,7 @@ public class TestMetricsThriftServerSourceFactoryImpl {
}
@Test
public void testCreateThriftTwoSource() throws Exception {
public void testCreateThriftTwoSource() {
//Make sure that the factory gives back a singleton.
assertSame(new MetricsThriftServerSourceFactoryImpl().createThriftTwoSource(),
new MetricsThriftServerSourceFactoryImpl().createThriftTwoSource());

View File

@ -1,4 +1,4 @@
/**
/*
* 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
@ -30,14 +30,14 @@ import org.junit.experimental.categories.Category;
@Category({MetricsTests.class, SmallTests.class})
public class TestMetricsZooKeeperSourceImpl {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestMetricsZooKeeperSourceImpl.class);
@Test
public void testGetInstance() throws Exception {
MetricsZooKeeperSource zkSource = CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class);
public void testGetInstance() {
MetricsZooKeeperSource zkSource =
CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class);
assertTrue(zkSource instanceof MetricsZooKeeperSourceImpl);
assertSame(zkSource, CompatibilitySingletonFactory.getInstance(MetricsZooKeeperSource.class));
}