HBASE-15728 Add Per-Table metrics back
Signed-off-by: Andrew Purtell <apurtell@apache.org>
This commit is contained in:
parent
fcd883b5dd
commit
e903ae5d5e
|
@ -254,6 +254,9 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo
|
|||
"action at the RPC Server (Sums 'readRequestsCount' and 'writeRequestsCount'); counts" +
|
||||
"once per access whether a Put of 1M rows or a Get that returns 1M Results";
|
||||
String READ_REQUEST_COUNT = "readRequestCount";
|
||||
String FILTERED_READ_REQUEST_COUNT = "filteredReadRequestCount";
|
||||
String FILTERED_READ_REQUEST_COUNT_DESC =
|
||||
"Number of read requests this region server has answered.";
|
||||
String READ_REQUEST_COUNT_DESC =
|
||||
"Number of read requests with non-empty Results that this RegionServer has answered.";
|
||||
String READ_REQUEST_RATE_PER_SECOND = "readRequestRatePerSecond";
|
||||
|
@ -262,9 +265,6 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo
|
|||
String CP_REQUEST_COUNT = "cpRequestCount";
|
||||
String CP_REQUEST_COUNT_DESC =
|
||||
"Number of coprocessor service requests this region server has answered.";
|
||||
String FILTERED_READ_REQUEST_COUNT = "filteredReadRequestCount";
|
||||
String FILTERED_READ_REQUEST_COUNT_DESC =
|
||||
"Number of filtered read requests this RegionServer has answered.";
|
||||
String WRITE_REQUEST_COUNT = "writeRequestCount";
|
||||
String WRITE_REQUEST_COUNT_DESC =
|
||||
"Number of mutation requests this RegionServer has answered.";
|
||||
|
|
|
@ -107,6 +107,8 @@ public interface MetricsRegionWrapper {
|
|||
*/
|
||||
long getWriteRequestCount();
|
||||
|
||||
long getTotalRequestCount();
|
||||
|
||||
long getNumFilesCompacted();
|
||||
|
||||
long getNumBytesCompacted();
|
||||
|
|
|
@ -52,18 +52,16 @@ public interface MetricsTableAggregateSource extends BaseSource {
|
|||
String NUMBER_OF_TABLES_DESC = "Number of tables in the metrics system";
|
||||
|
||||
/**
|
||||
* Register a MetricsTableSource as being open.
|
||||
*
|
||||
* Returns MetricsTableSource registered for the table. Creates one if not defined.
|
||||
* @param table The table name
|
||||
* @param source the source for the table being opened.
|
||||
*/
|
||||
void register(String table, MetricsTableSource source);
|
||||
MetricsTableSource getOrCreateTableSource(String table, MetricsTableWrapperAggregate wrapper);
|
||||
|
||||
/**
|
||||
* Remove a table's source. This is called when regions of a table are closed.
|
||||
*
|
||||
* @param table The table name
|
||||
*/
|
||||
void deregister(String table);
|
||||
void deleteTableSource(String table);
|
||||
|
||||
}
|
||||
|
|
|
@ -20,25 +20,15 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import org.apache.yetus.audience.InterfaceAudience;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* This interface will be implemented to allow region server to push table metrics into
|
||||
* MetricsRegionAggregateSource that will in turn push data to the Hadoop metrics system.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public interface MetricsTableSource extends Comparable<MetricsTableSource> {
|
||||
public interface MetricsTableSource extends Comparable<MetricsTableSource>, Closeable {
|
||||
|
||||
String READ_REQUEST_COUNT = "readRequestCount";
|
||||
String READ_REQUEST_COUNT_DESC = "Number of read requests";
|
||||
String CP_REQUEST_COUNT = "cpRequestCount";
|
||||
String CP_REQUEST_COUNT_DESC = "Number of coprocessor service requests";
|
||||
String WRITE_REQUEST_COUNT = "writeRequestCount";
|
||||
String WRITE_REQUEST_COUNT_DESC = "Number of write requests";
|
||||
String TOTAL_REQUEST_COUNT = "totalRequestCount";
|
||||
String TOTAL_REQUEST_COUNT_DESC = "Number of total requests";
|
||||
String MEMSTORE_SIZE = "memstoreSize";
|
||||
String MEMSTORE_SIZE_DESC = "The size of memory stores";
|
||||
String STORE_FILE_SIZE = "storeFileSize";
|
||||
String STORE_FILE_SIZE_DESC = "The size of store files size";
|
||||
String TABLE_SIZE = "tableSize";
|
||||
String TABLE_SIZE_DESC = "Total size of the table in the region server";
|
||||
|
||||
|
@ -47,11 +37,83 @@ public interface MetricsTableSource extends Comparable<MetricsTableSource> {
|
|||
/**
|
||||
* Close the table's metrics as all the region are closing.
|
||||
*/
|
||||
@Override
|
||||
void close();
|
||||
|
||||
void registerMetrics();
|
||||
|
||||
/**
|
||||
* Get the aggregate source to which this reports.
|
||||
*/
|
||||
MetricsTableAggregateSource getAggregateSource();
|
||||
|
||||
/**
|
||||
* Update the split transaction time histogram
|
||||
* @param t time it took, in milliseconds
|
||||
*/
|
||||
void updateSplitTime(long t);
|
||||
|
||||
/**
|
||||
* Increment number of a requested splits
|
||||
*/
|
||||
void incrSplitRequest();
|
||||
|
||||
/**
|
||||
* Increment number of successful splits
|
||||
*/
|
||||
void incrSplitSuccess();
|
||||
|
||||
/**
|
||||
* Update the flush time histogram
|
||||
* @param t time it took, in milliseconds
|
||||
*/
|
||||
void updateFlushTime(long t);
|
||||
|
||||
/**
|
||||
* Update the flush memstore size histogram
|
||||
* @param bytes the number of bytes in the memstore
|
||||
*/
|
||||
void updateFlushMemstoreSize(long bytes);
|
||||
|
||||
/**
|
||||
* Update the flush output file size histogram
|
||||
* @param bytes the number of bytes in the output file
|
||||
*/
|
||||
void updateFlushOutputSize(long bytes);
|
||||
|
||||
/**
|
||||
* Update the compaction time histogram, both major and minor
|
||||
* @param isMajor whether compaction is a major compaction
|
||||
* @param t time it took, in milliseconds
|
||||
*/
|
||||
void updateCompactionTime(boolean isMajor, long t);
|
||||
|
||||
/**
|
||||
* Update the compaction input number of files histogram
|
||||
* @param isMajor whether compaction is a major compaction
|
||||
* @param c number of files
|
||||
*/
|
||||
void updateCompactionInputFileCount(boolean isMajor, long c);
|
||||
|
||||
/**
|
||||
* Update the compaction total input file size histogram
|
||||
* @param isMajor whether compaction is a major compaction
|
||||
* @param bytes the number of bytes of the compaction input file
|
||||
*/
|
||||
void updateCompactionInputSize(boolean isMajor, long bytes);
|
||||
|
||||
/**
|
||||
* Update the compaction output number of files histogram
|
||||
* @param isMajor whether compaction is a major compaction
|
||||
* @param c number of files
|
||||
*/
|
||||
void updateCompactionOutputFileCount(boolean isMajor, long c);
|
||||
|
||||
/**
|
||||
* Update the compaction total output file size
|
||||
* @param isMajor whether compaction is a major compaction
|
||||
* @param bytes the number of bytes of the compaction input file
|
||||
*/
|
||||
void updateCompactionOutputSize(boolean isMajor, long bytes);
|
||||
|
||||
}
|
||||
|
|
|
@ -30,7 +30,7 @@ public interface MetricsTableWrapperAggregate {
|
|||
/**
|
||||
* Get the number of read requests that have been issued against this table
|
||||
*/
|
||||
long getReadRequestsCount(String table);
|
||||
long getReadRequestCount(String table);
|
||||
|
||||
/**
|
||||
* Get the number of CoprocessorService requests that have been issued against this table
|
||||
|
@ -38,27 +38,75 @@ public interface MetricsTableWrapperAggregate {
|
|||
long getCpRequestsCount(String table);
|
||||
|
||||
/**
|
||||
* Get the number of write requests that have been issued against this table
|
||||
* Get the total number of filtered read requests that have been issued against this table
|
||||
*/
|
||||
long getWriteRequestsCount(String table);
|
||||
long getFilteredReadRequestCount(String table);
|
||||
/**
|
||||
* Get the number of write requests that have been issued for this table
|
||||
*/
|
||||
long getWriteRequestCount(String table);
|
||||
|
||||
/**
|
||||
* Get the total number of requests that have been issued against this table
|
||||
* Get the total number of requests that have been issued for this table
|
||||
*/
|
||||
long getTotalRequestsCount(String table);
|
||||
|
||||
/**
|
||||
* Get the memory store size against this table
|
||||
*/
|
||||
long getMemStoresSize(String table);
|
||||
long getMemStoreSize(String table);
|
||||
|
||||
/**
|
||||
* Get the store file size against this table
|
||||
*/
|
||||
long getStoreFilesSize(String table);
|
||||
long getStoreFileSize(String table);
|
||||
|
||||
/**
|
||||
* Get the table region size against this table
|
||||
*/
|
||||
long getTableSize(String table);
|
||||
|
||||
|
||||
/**
|
||||
* Get the average region size for this table
|
||||
*/
|
||||
long getAvgRegionSize(String table);
|
||||
|
||||
/**
|
||||
* Get the number of regions hosted on for this table
|
||||
*/
|
||||
long getNumRegions(String table);
|
||||
|
||||
/**
|
||||
* Get the number of stores hosted on for this table
|
||||
*/
|
||||
long getNumStores(String table);
|
||||
|
||||
/**
|
||||
* Get the number of store files hosted for this table
|
||||
*/
|
||||
long getNumStoreFiles(String table);
|
||||
|
||||
/**
|
||||
* @return Max age of store files for this table
|
||||
*/
|
||||
long getMaxStoreFileAge(String table);
|
||||
|
||||
/**
|
||||
* @return Min age of store files for this table
|
||||
*/
|
||||
long getMinStoreFileAge(String table);
|
||||
|
||||
/**
|
||||
* @return Average age of store files for this table
|
||||
*/
|
||||
long getAvgStoreFileAge(String table);
|
||||
|
||||
/**
|
||||
* @return Number of reference files for this table
|
||||
*/
|
||||
long getNumReferenceFiles(String table);
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -54,9 +54,10 @@ public class MetricsRegionServerSourceImpl
|
|||
private final MutableFastCounter slowGet;
|
||||
private final MutableFastCounter slowIncrement;
|
||||
private final MutableFastCounter slowAppend;
|
||||
|
||||
// split related metrics
|
||||
private final MutableFastCounter splitRequest;
|
||||
private final MutableFastCounter splitSuccess;
|
||||
|
||||
private final MetricHistogram splitTimeHisto;
|
||||
|
||||
// flush related metrics
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver;
|
|||
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.metrics.BaseSourceImpl;
|
||||
import org.apache.hadoop.hbase.metrics.Interns;
|
||||
import org.apache.hadoop.metrics2.MetricsCollector;
|
||||
|
@ -46,24 +47,48 @@ public class MetricsTableAggregateSourceImpl extends BaseSourceImpl
|
|||
super(metricsName, metricsDescription, metricsContext, metricsJmxContext);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void register(String table, MetricsTableSource source) {
|
||||
tableSources.put(table, source);
|
||||
private void register(MetricsTableSource source) {
|
||||
synchronized (this) {
|
||||
source.registerMetrics();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void deregister(String table) {
|
||||
public void deleteTableSource(String table) {
|
||||
try {
|
||||
tableSources.remove(table);
|
||||
synchronized (this) {
|
||||
MetricsTableSource source = tableSources.remove(table);
|
||||
if (source != null) {
|
||||
source.close();
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// Ignored. If this errors out it means that someone is double
|
||||
// closing the region source and the region is already nulled out.
|
||||
LOG.info(
|
||||
"Error trying to remove " + table + " from " + this.getClass().getSimpleName(),
|
||||
e);
|
||||
// closing the user source and the user metrics is already nulled out.
|
||||
LOG.info("Error trying to remove " + table + " from " + getClass().getSimpleName(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public MetricsTableSource getOrCreateTableSource(String table,
|
||||
MetricsTableWrapperAggregate wrapper) {
|
||||
MetricsTableSource source = tableSources.get(table);
|
||||
if (source != null) {
|
||||
return source;
|
||||
}
|
||||
source = CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
|
||||
.createTable(table, wrapper);
|
||||
MetricsTableSource prev = tableSources.putIfAbsent(table, source);
|
||||
|
||||
if (prev != null) {
|
||||
return prev;
|
||||
} else {
|
||||
// register the new metrics now
|
||||
register(source);
|
||||
}
|
||||
return source;
|
||||
}
|
||||
|
||||
/**
|
||||
* Yes this is a get function that doesn't return anything. Thanks Hadoop for breaking all
|
||||
* expectations of java programmers. Instead of returning anything Hadoop metrics expects
|
||||
|
|
|
@ -23,11 +23,57 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.metrics.Interns;
|
||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||
import org.apache.hadoop.metrics2.MetricHistogram;
|
||||
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;
|
||||
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_INPUT_BYTES;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_INPUT_BYTES_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_OUTPUT_BYTES;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTED_OUTPUT_BYTES_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_FILE_COUNT;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_FILE_COUNT_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_INPUT_SIZE_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_FILE_COUNT;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_FILE_COUNT_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_OUTPUT_SIZE_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_TIME;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.COMPACTION_TIME_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_MEMSTORE_BYTES;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_MEMSTORE_BYTES_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_OUTPUT_BYTES;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSHED_OUTPUT_BYTES_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_MEMSTORE_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_MEMSTORE_SIZE_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_OUTPUT_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_OUTPUT_SIZE_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_TIME;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.FLUSH_TIME_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_INPUT_BYTES;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_INPUT_BYTES_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_OUTPUT_BYTES;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTED_OUTPUT_BYTES_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_FILE_COUNT;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_FILE_COUNT_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_INPUT_SIZE_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_FILE_COUNT;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_FILE_COUNT_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_SIZE;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_OUTPUT_SIZE_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_TIME;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.MAJOR_COMPACTION_TIME_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_KEY;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_REQUEST_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_REQUEST_KEY;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_SUCCESS_DESC;
|
||||
import static org.apache.hadoop.hbase.regionserver.MetricsRegionServerSource.SPLIT_SUCCESS_KEY;
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public class MetricsTableSourceImpl implements MetricsTableSource {
|
||||
|
||||
|
@ -46,12 +92,41 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
|||
private final TableName tableName;
|
||||
private final int hashCode;
|
||||
|
||||
// split related metrics
|
||||
private MutableFastCounter splitRequest;
|
||||
private MutableFastCounter splitSuccess;
|
||||
private MetricHistogram splitTimeHisto;
|
||||
|
||||
// flush related metrics
|
||||
private MetricHistogram flushTimeHisto;
|
||||
private MetricHistogram flushMemstoreSizeHisto;
|
||||
private MetricHistogram flushOutputSizeHisto;
|
||||
private MutableFastCounter flushedMemstoreBytes;
|
||||
private MutableFastCounter flushedOutputBytes;
|
||||
|
||||
// compaction related metrics
|
||||
private MetricHistogram compactionTimeHisto;
|
||||
private MetricHistogram compactionInputFileCountHisto;
|
||||
private MetricHistogram compactionInputSizeHisto;
|
||||
private MetricHistogram compactionOutputFileCountHisto;
|
||||
private MetricHistogram compactionOutputSizeHisto;
|
||||
private MutableFastCounter compactedInputBytes;
|
||||
private MutableFastCounter compactedOutputBytes;
|
||||
|
||||
private MetricHistogram majorCompactionTimeHisto;
|
||||
private MetricHistogram majorCompactionInputFileCountHisto;
|
||||
private MetricHistogram majorCompactionInputSizeHisto;
|
||||
private MetricHistogram majorCompactionOutputFileCountHisto;
|
||||
private MetricHistogram majorCompactionOutputSizeHisto;
|
||||
private MutableFastCounter majorCompactedInputBytes;
|
||||
private MutableFastCounter majorCompactedOutputBytes;
|
||||
|
||||
public MetricsTableSourceImpl(String tblName,
|
||||
MetricsTableAggregateSourceImpl aggregate, MetricsTableWrapperAggregate tblWrapperAgg) {
|
||||
LOG.debug("Creating new MetricsTableSourceImpl for table ");
|
||||
this.tableName = TableName.valueOf(tblName);
|
||||
this.agg = aggregate;
|
||||
agg.register(tblName, this);
|
||||
|
||||
this.tableWrapperAgg = tblWrapperAgg;
|
||||
this.registry = agg.getMetricsRegistry();
|
||||
this.tableNamePrefix = "Namespace_" + this.tableName.getNamespaceAsString() +
|
||||
|
@ -59,6 +134,79 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
|||
this.hashCode = this.tableName.hashCode();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void registerMetrics() {
|
||||
flushTimeHisto = registry.newTimeHistogram(tableNamePrefix + FLUSH_TIME, FLUSH_TIME_DESC);
|
||||
flushMemstoreSizeHisto =
|
||||
registry.newSizeHistogram(tableNamePrefix + FLUSH_MEMSTORE_SIZE, FLUSH_MEMSTORE_SIZE_DESC);
|
||||
flushOutputSizeHisto =
|
||||
registry.newSizeHistogram(tableNamePrefix + FLUSH_OUTPUT_SIZE, FLUSH_OUTPUT_SIZE_DESC);
|
||||
flushedOutputBytes =
|
||||
registry.newCounter(tableNamePrefix + FLUSHED_OUTPUT_BYTES, FLUSHED_OUTPUT_BYTES_DESC, 0L);
|
||||
flushedMemstoreBytes = registry.newCounter(tableNamePrefix + FLUSHED_MEMSTORE_BYTES,
|
||||
FLUSHED_MEMSTORE_BYTES_DESC, 0L);
|
||||
|
||||
compactionTimeHisto =
|
||||
registry.newTimeHistogram(tableNamePrefix + COMPACTION_TIME, COMPACTION_TIME_DESC);
|
||||
compactionInputFileCountHisto = registry.newHistogram(
|
||||
tableNamePrefix + COMPACTION_INPUT_FILE_COUNT, COMPACTION_INPUT_FILE_COUNT_DESC);
|
||||
compactionInputSizeHisto = registry.newSizeHistogram(tableNamePrefix + COMPACTION_INPUT_SIZE,
|
||||
COMPACTION_INPUT_SIZE_DESC);
|
||||
compactionOutputFileCountHisto = registry.newHistogram(
|
||||
tableNamePrefix + COMPACTION_OUTPUT_FILE_COUNT, COMPACTION_OUTPUT_FILE_COUNT_DESC);
|
||||
compactionOutputSizeHisto = registry.newSizeHistogram(tableNamePrefix + COMPACTION_OUTPUT_SIZE,
|
||||
COMPACTION_OUTPUT_SIZE_DESC);
|
||||
compactedInputBytes = registry.newCounter(tableNamePrefix + COMPACTED_INPUT_BYTES,
|
||||
COMPACTED_INPUT_BYTES_DESC, 0L);
|
||||
compactedOutputBytes = registry.newCounter(tableNamePrefix + COMPACTED_OUTPUT_BYTES,
|
||||
COMPACTED_OUTPUT_BYTES_DESC, 0L);
|
||||
|
||||
majorCompactionTimeHisto = registry.newTimeHistogram(tableNamePrefix + MAJOR_COMPACTION_TIME,
|
||||
MAJOR_COMPACTION_TIME_DESC);
|
||||
majorCompactionInputFileCountHisto = registry.newHistogram(
|
||||
tableNamePrefix + MAJOR_COMPACTION_INPUT_FILE_COUNT, MAJOR_COMPACTION_INPUT_FILE_COUNT_DESC);
|
||||
majorCompactionInputSizeHisto = registry.newSizeHistogram(
|
||||
tableNamePrefix + MAJOR_COMPACTION_INPUT_SIZE, MAJOR_COMPACTION_INPUT_SIZE_DESC);
|
||||
majorCompactionOutputFileCountHisto =
|
||||
registry.newHistogram(tableNamePrefix + MAJOR_COMPACTION_OUTPUT_FILE_COUNT,
|
||||
MAJOR_COMPACTION_OUTPUT_FILE_COUNT_DESC);
|
||||
majorCompactionOutputSizeHisto = registry.newSizeHistogram(
|
||||
tableNamePrefix + MAJOR_COMPACTION_OUTPUT_SIZE, MAJOR_COMPACTION_OUTPUT_SIZE_DESC);
|
||||
majorCompactedInputBytes = registry.newCounter(tableNamePrefix + MAJOR_COMPACTED_INPUT_BYTES,
|
||||
MAJOR_COMPACTED_INPUT_BYTES_DESC, 0L);
|
||||
majorCompactedOutputBytes = registry.newCounter(tableNamePrefix + MAJOR_COMPACTED_OUTPUT_BYTES,
|
||||
MAJOR_COMPACTED_OUTPUT_BYTES_DESC, 0L);
|
||||
|
||||
splitTimeHisto = registry.newTimeHistogram(tableNamePrefix + SPLIT_KEY);
|
||||
splitRequest = registry.newCounter(tableNamePrefix + SPLIT_REQUEST_KEY, SPLIT_REQUEST_DESC, 0L);
|
||||
splitSuccess = registry.newCounter(tableNamePrefix + SPLIT_SUCCESS_KEY, SPLIT_SUCCESS_DESC, 0L);
|
||||
}
|
||||
|
||||
private void deregisterMetrics() {
|
||||
registry.removeHistogramMetrics(tableNamePrefix + FLUSH_TIME);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + FLUSH_MEMSTORE_SIZE);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + FLUSH_OUTPUT_SIZE);
|
||||
registry.removeMetric(tableNamePrefix + FLUSHED_OUTPUT_BYTES);
|
||||
registry.removeMetric(tableNamePrefix + FLUSHED_MEMSTORE_BYTES);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_TIME);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_INPUT_FILE_COUNT);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_INPUT_SIZE);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_OUTPUT_FILE_COUNT);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + COMPACTION_OUTPUT_SIZE);
|
||||
registry.removeMetric(tableNamePrefix + COMPACTED_INPUT_BYTES);
|
||||
registry.removeMetric(tableNamePrefix + COMPACTED_OUTPUT_BYTES);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_TIME);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_INPUT_FILE_COUNT);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_INPUT_SIZE);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_OUTPUT_FILE_COUNT);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + MAJOR_COMPACTION_OUTPUT_SIZE);
|
||||
registry.removeMetric(tableNamePrefix + MAJOR_COMPACTED_INPUT_BYTES);
|
||||
registry.removeMetric(tableNamePrefix + MAJOR_COMPACTED_OUTPUT_BYTES);
|
||||
registry.removeHistogramMetrics(tableNamePrefix + SPLIT_KEY);
|
||||
registry.removeMetric(tableNamePrefix + SPLIT_REQUEST_KEY);
|
||||
registry.removeMetric(tableNamePrefix + SPLIT_SUCCESS_KEY);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
boolean wasClosed = closed.getAndSet(true);
|
||||
|
@ -70,7 +218,7 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
|||
|
||||
// Before removing the metrics remove this table from the aggregate table bean.
|
||||
// This should mean that it's unlikely that snapshot and close happen at the same time.
|
||||
agg.deregister(tableName.getNameAsString());
|
||||
agg.deleteTableSource(tableName.getNameAsString());
|
||||
|
||||
// While it's un-likely that snapshot and close happen at the same time it's still possible.
|
||||
// So grab the lock to ensure that all calls to snapshot are done before we remove the metrics
|
||||
|
@ -78,6 +226,7 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
|||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Removing table Metrics for table ");
|
||||
}
|
||||
deregisterMetrics();
|
||||
tableWrapperAgg = null;
|
||||
}
|
||||
}
|
||||
|
@ -122,27 +271,55 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
|||
}
|
||||
|
||||
if (this.tableWrapperAgg != null) {
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsTableSource.READ_REQUEST_COUNT,
|
||||
MetricsTableSource.READ_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getReadRequestsCount(tableName.getNameAsString()));
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsTableSource.CP_REQUEST_COUNT,
|
||||
MetricsTableSource.CP_REQUEST_COUNT_DESC),
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsRegionServerSource.CP_REQUEST_COUNT,
|
||||
MetricsRegionServerSource.CP_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getCpRequestsCount(tableName.getNameAsString()));
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsTableSource.WRITE_REQUEST_COUNT,
|
||||
MetricsTableSource.WRITE_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getWriteRequestsCount(tableName.getNameAsString()));
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsTableSource.TOTAL_REQUEST_COUNT,
|
||||
MetricsTableSource.TOTAL_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getTotalRequestsCount(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsTableSource.MEMSTORE_SIZE,
|
||||
MetricsTableSource.MEMSTORE_SIZE_DESC),
|
||||
tableWrapperAgg.getMemStoresSize(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsTableSource.STORE_FILE_SIZE,
|
||||
MetricsTableSource.STORE_FILE_SIZE_DESC),
|
||||
tableWrapperAgg.getStoreFilesSize(tableName.getNameAsString()));
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsRegionServerSource.READ_REQUEST_COUNT,
|
||||
MetricsRegionServerSource.READ_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getReadRequestCount(tableName.getNameAsString()));
|
||||
mrb.addCounter(
|
||||
Interns.info(tableNamePrefix + MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT,
|
||||
MetricsRegionServerSource.FILTERED_READ_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getFilteredReadRequestCount(tableName.getNameAsString()));
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsRegionServerSource.WRITE_REQUEST_COUNT,
|
||||
MetricsRegionServerSource.WRITE_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getWriteRequestCount(tableName.getNameAsString()));
|
||||
mrb.addCounter(Interns.info(tableNamePrefix + MetricsRegionServerSource.TOTAL_REQUEST_COUNT,
|
||||
MetricsRegionServerSource.TOTAL_REQUEST_COUNT_DESC),
|
||||
tableWrapperAgg.getTotalRequestsCount(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
|
||||
MetricsRegionServerSource.MEMSTORE_SIZE_DESC),
|
||||
tableWrapperAgg.getMemStoreSize(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.STOREFILE_COUNT,
|
||||
MetricsRegionServerSource.STOREFILE_COUNT_DESC),
|
||||
tableWrapperAgg.getNumStoreFiles(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
|
||||
MetricsRegionServerSource.STOREFILE_SIZE_DESC),
|
||||
tableWrapperAgg.getStoreFileSize(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsTableSource.TABLE_SIZE,
|
||||
MetricsTableSource.TABLE_SIZE_DESC),
|
||||
tableWrapperAgg.getTableSize(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.AVERAGE_REGION_SIZE,
|
||||
MetricsRegionServerSource.AVERAGE_REGION_SIZE_DESC),
|
||||
tableWrapperAgg.getAvgRegionSize(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.REGION_COUNT,
|
||||
MetricsRegionServerSource.REGION_COUNT_DESC),
|
||||
tableWrapperAgg.getNumRegions(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.STORE_COUNT,
|
||||
MetricsRegionServerSource.STORE_COUNT_DESC),
|
||||
tableWrapperAgg.getNumStores(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.MAX_STORE_FILE_AGE,
|
||||
MetricsRegionServerSource.MAX_STORE_FILE_AGE_DESC),
|
||||
tableWrapperAgg.getMaxStoreFileAge(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.MIN_STORE_FILE_AGE,
|
||||
MetricsRegionServerSource.MIN_STORE_FILE_AGE_DESC),
|
||||
tableWrapperAgg.getMinStoreFileAge(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.AVG_STORE_FILE_AGE,
|
||||
MetricsRegionServerSource.AVG_STORE_FILE_AGE_DESC),
|
||||
tableWrapperAgg.getAvgStoreFileAge(tableName.getNameAsString()));
|
||||
mrb.addGauge(Interns.info(tableNamePrefix + MetricsRegionServerSource.NUM_REFERENCE_FILES,
|
||||
MetricsRegionServerSource.NUM_REFERENCE_FILES_DESC),
|
||||
tableWrapperAgg.getNumReferenceFiles(tableName.getNameAsString()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -177,4 +354,80 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
|||
public String getTableNamePrefix() {
|
||||
return tableNamePrefix;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrSplitRequest() {
|
||||
splitRequest.incr();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void incrSplitSuccess() {
|
||||
splitSuccess.incr();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateSplitTime(long t) {
|
||||
splitTimeHisto.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateFlushTime(long t) {
|
||||
flushTimeHisto.add(t);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateFlushMemstoreSize(long bytes) {
|
||||
flushMemstoreSizeHisto.add(bytes);
|
||||
flushedMemstoreBytes.incr(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateFlushOutputSize(long bytes) {
|
||||
flushOutputSizeHisto.add(bytes);
|
||||
flushedOutputBytes.incr(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateCompactionTime(boolean isMajor, long t) {
|
||||
compactionTimeHisto.add(t);
|
||||
if (isMajor) {
|
||||
majorCompactionTimeHisto.add(t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateCompactionInputFileCount(boolean isMajor, long c) {
|
||||
compactionInputFileCountHisto.add(c);
|
||||
if (isMajor) {
|
||||
majorCompactionInputFileCountHisto.add(c);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateCompactionInputSize(boolean isMajor, long bytes) {
|
||||
compactionInputSizeHisto.add(bytes);
|
||||
compactedInputBytes.incr(bytes);
|
||||
if (isMajor) {
|
||||
majorCompactionInputSizeHisto.add(bytes);
|
||||
majorCompactedInputBytes.incr(bytes);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateCompactionOutputFileCount(boolean isMajor, long c) {
|
||||
compactionOutputFileCountHisto.add(c);
|
||||
if (isMajor) {
|
||||
majorCompactionOutputFileCountHisto.add(c);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateCompactionOutputSize(boolean isMajor, long bytes) {
|
||||
compactionOutputSizeHisto.add(bytes);
|
||||
compactedOutputBytes.incr(bytes);
|
||||
if (isMajor) {
|
||||
majorCompactionOutputSizeHisto.add(bytes);
|
||||
majorCompactedOutputBytes.incr(bytes);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,17 +27,12 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long getReadRequestsCount(String table) {
|
||||
public long getReadRequestCount(String table) {
|
||||
return 10;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCpRequestsCount(String table) {
|
||||
return 15;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWriteRequestsCount(String table) {
|
||||
public long getWriteRequestCount(String table) {
|
||||
return 20;
|
||||
}
|
||||
|
||||
|
@ -47,12 +42,17 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long getMemStoresSize(String table) {
|
||||
public long getFilteredReadRequestCount(String table) {
|
||||
return 40;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMemStoreSize(String table) {
|
||||
return 1000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStoreFilesSize(String table) {
|
||||
public long getStoreFileSize(String table) {
|
||||
return 2000;
|
||||
}
|
||||
|
||||
|
@ -64,4 +64,47 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
|
|||
public String getTableName() {
|
||||
return tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumRegions(String table) {
|
||||
return 11;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumStores(String table) {
|
||||
return 22;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumStoreFiles(String table) {
|
||||
return 33;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxStoreFileAge(String table) {
|
||||
return 44;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMinStoreFileAge(String table) {
|
||||
return 55;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAvgStoreFileAge(String table) {
|
||||
return 66;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumReferenceFiles(String table) {
|
||||
return 77;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAvgRegionSize(String table) {
|
||||
return 88;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCpRequestsCount(String table) {return 99;}
|
||||
}
|
|
@ -201,5 +201,10 @@ public class TestMetricsRegionSourceImpl {
|
|||
public long getMaxFlushQueueSize() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTotalRequestCount() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,10 +44,13 @@ public class TestMetricsTableSourceImpl {
|
|||
MetricsRegionServerSourceFactory metricsFact =
|
||||
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class);
|
||||
|
||||
MetricsTableSource one = metricsFact.createTable("ONETABLE", new TableWrapperStub("ONETABLE"));
|
||||
MetricsTableSource oneClone = metricsFact.createTable("ONETABLE",
|
||||
new TableWrapperStub("ONETABLE"));
|
||||
MetricsTableSource two = metricsFact.createTable("TWOTABLE", new TableWrapperStub("TWOTABLE"));
|
||||
MetricsTableSource one = metricsFact.createTable(
|
||||
"ONETABLE", new MetricsTableWrapperStub("ONETABLE"));
|
||||
MetricsTableSource oneClone = metricsFact.createTable(
|
||||
"ONETABLE",
|
||||
new MetricsTableWrapperStub("ONETABLE"));
|
||||
MetricsTableSource two = metricsFact.createTable(
|
||||
"TWOTABLE", new MetricsTableWrapperStub("TWOTABLE"));
|
||||
|
||||
assertEquals(0, one.compareTo(oneClone));
|
||||
assertEquals(one.hashCode(), oneClone.hashCode());
|
||||
|
@ -70,54 +73,8 @@ public class TestMetricsTableSourceImpl {
|
|||
public void testGetTableMetrics() {
|
||||
MetricsTableSource oneTbl =
|
||||
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
|
||||
.createTable("ONETABLE", new TableWrapperStub("ONETABLE"));
|
||||
.createTable("ONETABLE", new MetricsTableWrapperStub("ONETABLE"));
|
||||
assertEquals("ONETABLE", oneTbl.getTableName());
|
||||
}
|
||||
|
||||
static class TableWrapperStub implements MetricsTableWrapperAggregate {
|
||||
private String tableName;
|
||||
|
||||
public TableWrapperStub(String tableName) {
|
||||
this.tableName = tableName;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getReadRequestsCount(String table) {
|
||||
return 10;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getCpRequestsCount(String table) {
|
||||
return 15;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWriteRequestsCount(String table) {
|
||||
return 20;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTotalRequestsCount(String table) {
|
||||
return 30;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMemStoresSize(String table) {
|
||||
return 1000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStoreFilesSize(String table) {
|
||||
return 2000;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTableSize(String table) {
|
||||
return 3000;
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return tableName;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -2823,7 +2823,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
status.setStatus(msg);
|
||||
|
||||
if (rsServices != null && rsServices.getMetrics() != null) {
|
||||
rsServices.getMetrics().updateFlush(time - startTime,
|
||||
rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(),
|
||||
time - startTime,
|
||||
mss.getDataSize(), flushedOutputFileSize);
|
||||
}
|
||||
|
||||
|
|
|
@ -1524,9 +1524,9 @@ public class HRegionServer extends HasThread implements
|
|||
// This call sets up an initialized replication and WAL. Later we start it up.
|
||||
setupWALAndReplication();
|
||||
// Init in here rather than in constructor after thread name has been set
|
||||
this.metricsRegionServer = new MetricsRegionServer(
|
||||
new MetricsRegionServerWrapperImpl(this), conf);
|
||||
this.metricsTable = new MetricsTable(new MetricsTableWrapperAggregateImpl(this));
|
||||
this.metricsRegionServer = new MetricsRegionServer(
|
||||
new MetricsRegionServerWrapperImpl(this), conf, metricsTable);
|
||||
// Now that we have a metrics source, start the pause monitor
|
||||
this.pauseMonitor = new JvmPauseMonitor(conf, getMetrics().getMetricsSource());
|
||||
pauseMonitor.start();
|
||||
|
|
|
@ -1430,9 +1430,11 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat
|
|||
long now = EnvironmentEdgeManager.currentTime();
|
||||
if (region.getRegionServerServices() != null
|
||||
&& region.getRegionServerServices().getMetrics() != null) {
|
||||
region.getRegionServerServices().getMetrics().updateCompaction(cr.isMajor(),
|
||||
now - compactionStartTime, cr.getFiles().size(), newFiles.size(), cr.getSize(),
|
||||
outputBytes);
|
||||
region.getRegionServerServices().getMetrics().updateCompaction(
|
||||
region.getTableDescriptor().getTableName().getNameAsString(),
|
||||
cr.isMajor(), now - compactionStartTime, cr.getFiles().size(),
|
||||
newFiles.size(), cr.getSize(), outputBytes);
|
||||
|
||||
}
|
||||
|
||||
logCompactionEndMessage(cr, sfs, now, compactionStartTime);
|
||||
|
|
|
@ -46,16 +46,19 @@ public class MetricsRegionServer {
|
|||
private MetricsRegionServerSource serverSource;
|
||||
private MetricsRegionServerWrapper regionServerWrapper;
|
||||
private RegionServerTableMetrics tableMetrics;
|
||||
private final MetricsTable metricsTable;
|
||||
private MetricsRegionServerQuotaSource quotaSource;
|
||||
|
||||
private MetricRegistry metricRegistry;
|
||||
private Timer bulkLoadTimer;
|
||||
|
||||
public MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper, Configuration conf) {
|
||||
public MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper, Configuration conf,
|
||||
MetricsTable metricsTable) {
|
||||
this(regionServerWrapper,
|
||||
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
|
||||
.createServer(regionServerWrapper),
|
||||
createTableMetrics(conf));
|
||||
createTableMetrics(conf),
|
||||
metricsTable);
|
||||
|
||||
// Create hbase-metrics module based metrics. The registry should already be registered by the
|
||||
// MetricsRegionServerSource
|
||||
|
@ -69,10 +72,12 @@ public class MetricsRegionServer {
|
|||
|
||||
MetricsRegionServer(MetricsRegionServerWrapper regionServerWrapper,
|
||||
MetricsRegionServerSource serverSource,
|
||||
RegionServerTableMetrics tableMetrics) {
|
||||
RegionServerTableMetrics tableMetrics,
|
||||
MetricsTable metricsTable) {
|
||||
this.regionServerWrapper = regionServerWrapper;
|
||||
this.serverSource = serverSource;
|
||||
this.tableMetrics = tableMetrics;
|
||||
this.metricsTable = metricsTable;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -196,19 +201,34 @@ public class MetricsRegionServer {
|
|||
serverSource.incrSplitSuccess();
|
||||
}
|
||||
|
||||
public void updateFlush(long t, long memstoreSize, long fileSize) {
|
||||
public void updateFlush(String table, long t, long memstoreSize, long fileSize) {
|
||||
serverSource.updateFlushTime(t);
|
||||
serverSource.updateFlushMemStoreSize(memstoreSize);
|
||||
serverSource.updateFlushOutputSize(fileSize);
|
||||
|
||||
if (table != null) {
|
||||
metricsTable.updateFlushTime(table, memstoreSize);
|
||||
metricsTable.updateFlushMemstoreSize(table, memstoreSize);
|
||||
metricsTable.updateFlushOutputSize(table, fileSize);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public void updateCompaction(boolean isMajor, long t, int inputFileCount, int outputFileCount,
|
||||
public void updateCompaction(String table, boolean isMajor, long t, int inputFileCount, int outputFileCount,
|
||||
long inputBytes, long outputBytes) {
|
||||
serverSource.updateCompactionTime(isMajor, t);
|
||||
serverSource.updateCompactionInputFileCount(isMajor, inputFileCount);
|
||||
serverSource.updateCompactionOutputFileCount(isMajor, outputFileCount);
|
||||
serverSource.updateCompactionInputSize(isMajor, inputBytes);
|
||||
serverSource.updateCompactionOutputSize(isMajor, outputBytes);
|
||||
|
||||
if (table != null) {
|
||||
metricsTable.updateCompactionTime(table, isMajor, t);
|
||||
metricsTable.updateCompactionInputFileCount(table, isMajor, inputFileCount);
|
||||
metricsTable.updateCompactionOutputFileCount(table, isMajor, outputFileCount);
|
||||
metricsTable.updateCompactionInputSize(table, isMajor, inputBytes);
|
||||
metricsTable.updateCompactionOutputSize(table, isMajor, outputBytes);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateBulkLoad(long millis) {
|
||||
|
|
|
@ -166,6 +166,11 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
|||
return now - lastMajorCompactionTs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTotalRequestCount() {
|
||||
return getReadRequestCount() + getWriteRequestCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumCompactionsFailed() {
|
||||
return this.region.compactionsFailed.sum();
|
||||
|
|
|
@ -24,19 +24,67 @@ import org.apache.yetus.audience.InterfaceAudience;
|
|||
@InterfaceAudience.Private
|
||||
public class MetricsTable {
|
||||
private final MetricsTableAggregateSource tableSourceAgg;
|
||||
private MetricsTableWrapperAggregate tableWrapperAgg;
|
||||
private MetricsTableWrapperAggregate wrapper;
|
||||
|
||||
public MetricsTable(final MetricsTableWrapperAggregate wrapper) {
|
||||
tableSourceAgg = CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
|
||||
.getTableAggregate();
|
||||
this.tableWrapperAgg = wrapper;
|
||||
this.wrapper = wrapper;
|
||||
}
|
||||
|
||||
public MetricsTableWrapperAggregate getTableWrapperAgg() {
|
||||
return tableWrapperAgg;
|
||||
return wrapper;
|
||||
}
|
||||
|
||||
public MetricsTableAggregateSource getTableSourceAgg() {
|
||||
return tableSourceAgg;
|
||||
}
|
||||
|
||||
public void incrSplitRequest(String table) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).incrSplitRequest();
|
||||
}
|
||||
|
||||
public void incrSplitSuccess(String table) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).incrSplitSuccess();
|
||||
}
|
||||
|
||||
public void updateSplitTime(String table, long t) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).updateSplitTime(t);
|
||||
}
|
||||
|
||||
public void updateFlushTime(String table, long t) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).updateFlushTime(t);
|
||||
}
|
||||
|
||||
public void updateFlushMemstoreSize(String table, long bytes) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).updateFlushMemstoreSize(bytes);
|
||||
}
|
||||
|
||||
public void updateFlushOutputSize(String table, long bytes) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).updateFlushOutputSize(bytes);
|
||||
}
|
||||
|
||||
public void updateCompactionTime(String table, boolean isMajor, long t) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper).updateCompactionTime(isMajor, t);
|
||||
}
|
||||
|
||||
public void updateCompactionInputFileCount(String table, boolean isMajor, long c) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper)
|
||||
.updateCompactionInputFileCount(isMajor, c);
|
||||
}
|
||||
|
||||
public void updateCompactionInputSize(String table, boolean isMajor, long bytes) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper)
|
||||
.updateCompactionInputSize(isMajor, bytes);
|
||||
}
|
||||
|
||||
public void updateCompactionOutputFileCount(String table, boolean isMajor, long c) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper)
|
||||
.updateCompactionOutputFileCount(isMajor, c);
|
||||
}
|
||||
|
||||
public void updateCompactionOutputSize(String table, boolean isMajor, long bytes) {
|
||||
tableSourceAgg.getOrCreateTableSource(table, wrapper)
|
||||
.updateCompactionOutputSize(isMajor, bytes);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,7 +43,8 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
private Runnable runnable;
|
||||
private long period;
|
||||
private ScheduledFuture<?> tableMetricsUpdateTask;
|
||||
private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap = new ConcurrentHashMap<>();
|
||||
private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap
|
||||
= new ConcurrentHashMap<>();
|
||||
|
||||
public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
|
||||
this.regionServer = regionServer;
|
||||
|
@ -51,8 +52,8 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
HConstants.DEFAULT_REGIONSERVER_METRICS_PERIOD) + 1000;
|
||||
this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
|
||||
this.runnable = new TableMetricsWrapperRunnable();
|
||||
this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, period, this.period,
|
||||
TimeUnit.MILLISECONDS);
|
||||
this.tableMetricsUpdateTask = this.executor.scheduleWithFixedDelay(this.runnable, period,
|
||||
this.period, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
|
||||
public class TableMetricsWrapperRunnable implements Runnable {
|
||||
|
@ -62,35 +63,43 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
Map<TableName, MetricsTableValues> localMetricsTableMap = new HashMap<>();
|
||||
|
||||
for (Region r : regionServer.getOnlineRegionsLocalContext()) {
|
||||
TableName tbl= r.getTableDescriptor().getTableName();
|
||||
MetricsTableValues metricsTable = localMetricsTableMap.get(tbl);
|
||||
if (metricsTable == null) {
|
||||
metricsTable = new MetricsTableValues();
|
||||
localMetricsTableMap.put(tbl, metricsTable);
|
||||
TableName tbl = r.getTableDescriptor().getTableName();
|
||||
MetricsTableValues mt = localMetricsTableMap.get(tbl);
|
||||
if (mt == null) {
|
||||
mt = new MetricsTableValues();
|
||||
localMetricsTableMap.put(tbl, mt);
|
||||
}
|
||||
long tempStorefilesSize = 0;
|
||||
for (Store store : r.getStores()) {
|
||||
tempStorefilesSize += store.getStorefilesSize();
|
||||
if (r.getStores() != null) {
|
||||
for (Store store : r.getStores()) {
|
||||
mt.storeFileCount += store.getStorefilesCount();
|
||||
mt.memstoreSize += (store.getMemStoreSize().getDataSize() +
|
||||
store.getMemStoreSize().getHeapSize() + store.getMemStoreSize().getOffHeapSize());
|
||||
mt.storeFileSize += store.getStorefilesSize();
|
||||
mt.referenceFileCount += store.getNumReferenceFiles();
|
||||
|
||||
mt.maxStoreFileAge = Math.max(mt.maxStoreFileAge, store.getMaxStoreFileAge().getAsLong());
|
||||
mt.minStoreFileAge = Math.min(mt.minStoreFileAge, store.getMinStoreFileAge().getAsLong());
|
||||
mt.totalStoreFileAge = (long)store.getAvgStoreFileAge().getAsDouble() *
|
||||
store.getStorefilesCount();
|
||||
mt.storeCount += 1;
|
||||
}
|
||||
mt.regionCount += 1;
|
||||
|
||||
mt.readRequestCount += r.getReadRequestsCount();
|
||||
mt.filteredReadRequestCount += getFilteredReadRequestCount(tbl.getNameAsString());
|
||||
mt.writeRequestCount += r.getWriteRequestsCount();
|
||||
|
||||
}
|
||||
metricsTable.setMemStoresSize(metricsTable.getMemStoresSize() + r.getMemStoreDataSize());
|
||||
metricsTable.setStoreFilesSize(metricsTable.getStoreFilesSize() + tempStorefilesSize);
|
||||
metricsTable.setTableSize(metricsTable.getMemStoresSize() + metricsTable.getStoreFilesSize());
|
||||
metricsTable.setReadRequestsCount(metricsTable.getReadRequestsCount() + r.getReadRequestsCount());
|
||||
metricsTable.setCpRequestsCount(metricsTable.getCpRequestsCount() + r.getCpRequestsCount());
|
||||
metricsTable.setWriteRequestsCount(metricsTable.getWriteRequestsCount() + r.getWriteRequestsCount());
|
||||
metricsTable.setTotalRequestsCount(metricsTable.getReadRequestsCount()
|
||||
+ metricsTable.getWriteRequestsCount() + metricsTable.getCpRequestsCount());
|
||||
}
|
||||
|
||||
for(Map.Entry<TableName, MetricsTableValues> entry : localMetricsTableMap.entrySet()) {
|
||||
for (Map.Entry<TableName, MetricsTableValues> entry : localMetricsTableMap.entrySet()) {
|
||||
TableName tbl = entry.getKey();
|
||||
if (metricsTableMap.get(tbl) == null) {
|
||||
MetricsTableSource tableSource = CompatibilitySingletonFactory
|
||||
.getInstance(MetricsRegionServerSourceFactory.class).createTable(tbl.getNameAsString(),
|
||||
MetricsTableWrapperAggregateImpl.this);
|
||||
// this will add the Wrapper to the list of TableMetrics
|
||||
CompatibilitySingletonFactory
|
||||
.getInstance(MetricsRegionServerSourceFactory.class).getTableAggregate()
|
||||
.register(tbl.getNameAsString(), tableSource);
|
||||
.getInstance(MetricsRegionServerSourceFactory.class)
|
||||
.getTableAggregate()
|
||||
.getOrCreateTableSource(tbl.getNameAsString(), MetricsTableWrapperAggregateImpl.this);
|
||||
}
|
||||
metricsTableMap.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
|
@ -99,7 +108,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
MetricsTableAggregateSource agg = CompatibilitySingletonFactory
|
||||
.getInstance(MetricsRegionServerSourceFactory.class).getTableAggregate();
|
||||
for (TableName table : existingTableNames) {
|
||||
agg.deregister(table.getNameAsString());
|
||||
agg.deleteTableSource(table.getNameAsString());
|
||||
if (metricsTableMap.get(table) != null) {
|
||||
metricsTableMap.remove(table);
|
||||
}
|
||||
|
@ -108,12 +117,12 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
}
|
||||
|
||||
@Override
|
||||
public long getReadRequestsCount(String table) {
|
||||
public long getReadRequestCount(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getReadRequestsCount();
|
||||
return metricsTable.readRequestCount;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -123,17 +132,25 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getCpRequestsCount();
|
||||
return metricsTable.cpRequestCount;
|
||||
}
|
||||
}
|
||||
|
||||
public long getFilteredReadRequestCount(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.filteredReadRequestCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getWriteRequestsCount(String table) {
|
||||
public long getWriteRequestCount(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getWriteRequestsCount();
|
||||
return metricsTable.writeRequestCount;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -143,27 +160,27 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getTotalRequestsCount();
|
||||
return metricsTable.readRequestCount + metricsTable.writeRequestCount;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMemStoresSize(String table) {
|
||||
public long getMemStoreSize(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getMemStoresSize();
|
||||
return metricsTable.memstoreSize;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getStoreFilesSize(String table) {
|
||||
public long getStoreFileSize(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getStoreFilesSize();
|
||||
return metricsTable.storeFileSize;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -173,80 +190,113 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
|||
if (metricsTable == null) {
|
||||
return 0;
|
||||
} else {
|
||||
return metricsTable.getTableSize();
|
||||
return metricsTable.memstoreSize + metricsTable.storeFileSize;
|
||||
}
|
||||
}
|
||||
|
||||
public long getNumRegions(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.regionCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumStores(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.storeCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumStoreFiles(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.storeFileCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMaxStoreFileAge(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.maxStoreFileAge;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getMinStoreFileAge(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.minStoreFileAge == Long.MAX_VALUE ? 0 : metricsTable.minStoreFileAge;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAvgStoreFileAge(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
return metricsTable.storeFileCount == 0
|
||||
? 0
|
||||
: (metricsTable.totalStoreFileAge / metricsTable.storeFileCount);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumReferenceFiles(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.referenceFileCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAvgRegionSize(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.regionCount == 0
|
||||
? 0
|
||||
: (metricsTable.memstoreSize + metricsTable.storeFileSize) / metricsTable.regionCount;
|
||||
}
|
||||
|
||||
public long getCpRequestCount(String table) {
|
||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||
if (metricsTable == null) {
|
||||
return 0;
|
||||
}
|
||||
return metricsTable.cpRequestCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
tableMetricsUpdateTask.cancel(true);
|
||||
}
|
||||
|
||||
private static class MetricsTableValues {
|
||||
|
||||
private long totalRequestsCount;
|
||||
private long readRequestsCount;
|
||||
private long cpRequestsCount;
|
||||
private long writeRequestsCount;
|
||||
private long memstoresSize;
|
||||
private long storeFilesSize;
|
||||
private long tableSize;
|
||||
|
||||
public long getTotalRequestsCount() {
|
||||
return totalRequestsCount;
|
||||
}
|
||||
|
||||
public void setTotalRequestsCount(long totalRequestsCount) {
|
||||
this.totalRequestsCount = totalRequestsCount;
|
||||
}
|
||||
|
||||
public long getReadRequestsCount() {
|
||||
return readRequestsCount;
|
||||
}
|
||||
|
||||
public void setReadRequestsCount(long readRequestsCount) {
|
||||
this.readRequestsCount = readRequestsCount;
|
||||
}
|
||||
|
||||
public long getCpRequestsCount() {
|
||||
return cpRequestsCount;
|
||||
}
|
||||
|
||||
public void setCpRequestsCount(long cpRequestsCount) {
|
||||
this.cpRequestsCount = cpRequestsCount;
|
||||
}
|
||||
|
||||
public long getWriteRequestsCount() {
|
||||
return writeRequestsCount;
|
||||
}
|
||||
|
||||
public void setWriteRequestsCount(long writeRequestsCount) {
|
||||
this.writeRequestsCount = writeRequestsCount;
|
||||
}
|
||||
|
||||
public long getMemStoresSize() {
|
||||
return memstoresSize;
|
||||
}
|
||||
|
||||
public void setMemStoresSize(long memstoresSize) {
|
||||
this.memstoresSize = memstoresSize;
|
||||
}
|
||||
|
||||
public long getStoreFilesSize() {
|
||||
return storeFilesSize;
|
||||
}
|
||||
|
||||
public void setStoreFilesSize(long storeFilesSize) {
|
||||
this.storeFilesSize = storeFilesSize;
|
||||
}
|
||||
|
||||
public long getTableSize() {
|
||||
return tableSize;
|
||||
}
|
||||
|
||||
public void setTableSize(long tableSize) {
|
||||
this.tableSize = tableSize;
|
||||
}
|
||||
long readRequestCount;
|
||||
long filteredReadRequestCount;
|
||||
long writeRequestCount;
|
||||
long memstoreSize;
|
||||
long regionCount;
|
||||
long storeCount;
|
||||
long storeFileCount;
|
||||
long storeFileSize;
|
||||
long maxStoreFileAge;
|
||||
long minStoreFileAge = Long.MAX_VALUE;
|
||||
long totalStoreFileAge;
|
||||
long referenceFileCount;
|
||||
long cpRequestCount;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -167,4 +167,9 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
|
|||
public long getMaxFlushQueueSize() {
|
||||
return 6;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getTotalRequestCount() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -57,7 +57,7 @@ public class TestMetricsRegionServer {
|
|||
@Before
|
||||
public void setUp() {
|
||||
wrapper = new MetricsRegionServerWrapperStub();
|
||||
rsm = new MetricsRegionServer(wrapper, new Configuration(false));
|
||||
rsm = new MetricsRegionServer(wrapper, new Configuration(false), null);
|
||||
serverSource = rsm.getMetricsSource();
|
||||
}
|
||||
|
||||
|
@ -185,14 +185,14 @@ public class TestMetricsRegionServer {
|
|||
|
||||
@Test
|
||||
public void testFlush() {
|
||||
rsm.updateFlush(1, 2, 3);
|
||||
rsm.updateFlush(null, 1, 2, 3);
|
||||
HELPER.assertCounter("flushTime_num_ops", 1, serverSource);
|
||||
HELPER.assertCounter("flushMemstoreSize_num_ops", 1, serverSource);
|
||||
HELPER.assertCounter("flushOutputSize_num_ops", 1, serverSource);
|
||||
HELPER.assertCounter("flushedMemstoreBytes", 2, serverSource);
|
||||
HELPER.assertCounter("flushedOutputBytes", 3, serverSource);
|
||||
|
||||
rsm.updateFlush(10, 20, 30);
|
||||
rsm.updateFlush(null, 10, 20, 30);
|
||||
HELPER.assertCounter("flushTimeNumOps", 2, serverSource);
|
||||
HELPER.assertCounter("flushMemstoreSize_num_ops", 2, serverSource);
|
||||
HELPER.assertCounter("flushOutputSize_num_ops", 2, serverSource);
|
||||
|
@ -202,7 +202,7 @@ public class TestMetricsRegionServer {
|
|||
|
||||
@Test
|
||||
public void testCompaction() {
|
||||
rsm.updateCompaction(false, 1, 2, 3, 4, 5);
|
||||
rsm.updateCompaction(null, false, 1, 2, 3, 4, 5);
|
||||
HELPER.assertCounter("compactionTime_num_ops", 1, serverSource);
|
||||
HELPER.assertCounter("compactionInputFileCount_num_ops", 1, serverSource);
|
||||
HELPER.assertCounter("compactionInputSize_num_ops", 1, serverSource);
|
||||
|
@ -210,7 +210,7 @@ public class TestMetricsRegionServer {
|
|||
HELPER.assertCounter("compactedInputBytes", 4, serverSource);
|
||||
HELPER.assertCounter("compactedoutputBytes", 5, serverSource);
|
||||
|
||||
rsm.updateCompaction(false, 10, 20, 30, 40, 50);
|
||||
rsm.updateCompaction(null, false, 10, 20, 30, 40, 50);
|
||||
HELPER.assertCounter("compactionTime_num_ops", 2, serverSource);
|
||||
HELPER.assertCounter("compactionInputFileCount_num_ops", 2, serverSource);
|
||||
HELPER.assertCounter("compactionInputSize_num_ops", 2, serverSource);
|
||||
|
@ -219,7 +219,7 @@ public class TestMetricsRegionServer {
|
|||
HELPER.assertCounter("compactedoutputBytes", 55, serverSource);
|
||||
|
||||
// do major compaction
|
||||
rsm.updateCompaction(true, 100, 200, 300, 400, 500);
|
||||
rsm.updateCompaction(null, true, 100, 200, 300, 400, 500);
|
||||
|
||||
HELPER.assertCounter("compactionTime_num_ops", 3, serverSource);
|
||||
HELPER.assertCounter("compactionInputFileCount_num_ops", 3, serverSource);
|
||||
|
|
|
@ -18,12 +18,14 @@
|
|||
package org.apache.hadoop.hbase.regionserver;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.CompatibilityFactory;
|
||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||
import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
@ -33,27 +35,108 @@ public class TestMetricsTableAggregate {
|
|||
|
||||
@ClassRule
|
||||
public static final HBaseClassTestRule CLASS_RULE =
|
||||
HBaseClassTestRule.forClass(TestMetricsTableAggregate.class);
|
||||
HBaseClassTestRule.forClass(TestMetricsTableAggregate.class);
|
||||
|
||||
public static MetricsAssertHelper HELPER =
|
||||
CompatibilityFactory.getInstance(MetricsAssertHelper.class);
|
||||
CompatibilityFactory.getInstance(MetricsAssertHelper.class);
|
||||
|
||||
private String tableName = "testTableMetrics";
|
||||
private String pre = "Namespace_default_table_" + tableName + "_metric_";
|
||||
|
||||
private MetricsTableWrapperStub tableWrapper;
|
||||
private MetricsTable mt;
|
||||
private MetricsRegionServerWrapper rsWrapper;
|
||||
private MetricsRegionServer rsm;
|
||||
private MetricsTableAggregateSource agg;
|
||||
|
||||
@BeforeClass
|
||||
public static void classSetUp() {
|
||||
HELPER.init();
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() {
|
||||
tableWrapper = new MetricsTableWrapperStub(tableName);
|
||||
mt = new MetricsTable(tableWrapper);
|
||||
rsWrapper = new MetricsRegionServerWrapperStub();
|
||||
Configuration conf = new Configuration();
|
||||
rsm = new MetricsRegionServer(rsWrapper, conf, mt);
|
||||
agg = mt.getTableSourceAgg();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTableWrapperAggregateMetrics() throws IOException {
|
||||
String tableName = "testTableMetrics";
|
||||
MetricsTableWrapperStub tableWrapper = new MetricsTableWrapperStub(tableName);
|
||||
CompatibilitySingletonFactory.getInstance(MetricsRegionServerSourceFactory.class)
|
||||
.createTable(tableName, tableWrapper);
|
||||
MetricsTableAggregateSource agg = CompatibilitySingletonFactory
|
||||
.getInstance(MetricsRegionServerSourceFactory.class).getTableAggregate();
|
||||
|
||||
HELPER.assertCounter("Namespace_default_table_testTableMetrics_metric_readRequestCount", 10, agg);
|
||||
HELPER.assertCounter("Namespace_default_table_testTableMetrics_metric_cpRequestCount", 15, agg);
|
||||
HELPER.assertCounter("Namespace_default_table_testTableMetrics_metric_writeRequestCount", 20, agg);
|
||||
HELPER.assertCounter("Namespace_default_table_testTableMetrics_metric_totalRequestCount", 30, agg);
|
||||
|
||||
HELPER.assertGauge("Namespace_default_table_testTableMetrics_metric_memstoreSize", 1000, agg);
|
||||
HELPER.assertGauge("Namespace_default_table_testTableMetrics_metric_storeFileSize", 2000, agg);
|
||||
HELPER.assertGauge("Namespace_default_table_testTableMetrics_metric_tableSize", 3000, agg);
|
||||
public void testRequestMetrics() throws IOException {
|
||||
HELPER.assertCounter(pre + "readRequestCount", 10, agg);
|
||||
HELPER.assertCounter(pre + "writeRequestCount", 20, agg);
|
||||
HELPER.assertCounter(pre + "totalRequestCount", 30, agg);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRegionAndStoreMetrics() throws IOException {
|
||||
HELPER.assertGauge(pre + "memstoreSize", 1000, agg);
|
||||
HELPER.assertGauge(pre + "storeFileSize", 2000, agg);
|
||||
HELPER.assertGauge(pre + "tableSize", 3000, agg);
|
||||
|
||||
HELPER.assertGauge(pre + "regionCount", 11, agg);
|
||||
HELPER.assertGauge(pre + "storeCount", 22, agg);
|
||||
HELPER.assertGauge(pre + "storeFileCount", 33, agg);
|
||||
HELPER.assertGauge(pre + "maxStoreFileAge", 44, agg);
|
||||
HELPER.assertGauge(pre + "minStoreFileAge", 55, agg);
|
||||
HELPER.assertGauge(pre + "avgStoreFileAge", 66, agg);
|
||||
HELPER.assertGauge(pre + "numReferenceFiles", 77, agg);
|
||||
HELPER.assertGauge(pre + "averageRegionSize", 88, agg);
|
||||
}
|
||||
|
||||
public void testFlush() {
|
||||
rsm.updateFlush(tableName, 1, 2, 3);
|
||||
HELPER.assertCounter(pre + "flushTime_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "flushMemstoreSize_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "flushOutputSize_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "flushedMemstoreBytes", 2, agg);
|
||||
HELPER.assertCounter(pre + "flushedOutputBytes", 3, agg);
|
||||
|
||||
rsm.updateFlush(tableName, 10, 20, 30);
|
||||
HELPER.assertCounter(pre + "flushTime_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "flushMemstoreSize_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "flushOutputSize_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "flushedMemstoreBytes", 22, agg);
|
||||
HELPER.assertCounter(pre + "flushedOutputBytes", 33, agg);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompaction() {
|
||||
rsm.updateCompaction(tableName, false, 1, 2, 3, 4, 5);
|
||||
HELPER.assertCounter(pre + "compactionTime_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "compactionInputFileCount_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "compactionInputSize_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "compactionOutputFileCount_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "compactedInputBytes", 4, agg);
|
||||
HELPER.assertCounter(pre + "compactedoutputBytes", 5, agg);
|
||||
|
||||
rsm.updateCompaction(tableName, false, 10, 20, 30, 40, 50);
|
||||
HELPER.assertCounter(pre + "compactionTime_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "compactionInputFileCount_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "compactionInputSize_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "compactionOutputFileCount_num_ops", 2, agg);
|
||||
HELPER.assertCounter(pre + "compactedInputBytes", 44, agg);
|
||||
HELPER.assertCounter(pre + "compactedoutputBytes", 55, agg);
|
||||
|
||||
// do major compaction
|
||||
rsm.updateCompaction(tableName, true, 100, 200, 300, 400, 500);
|
||||
|
||||
HELPER.assertCounter(pre + "compactionTime_num_ops", 3, agg);
|
||||
HELPER.assertCounter(pre + "compactionInputFileCount_num_ops", 3, agg);
|
||||
HELPER.assertCounter(pre + "compactionInputSize_num_ops", 3, agg);
|
||||
HELPER.assertCounter(pre + "compactionOutputFileCount_num_ops", 3, agg);
|
||||
HELPER.assertCounter(pre + "compactedInputBytes", 444, agg);
|
||||
HELPER.assertCounter(pre + "compactedoutputBytes", 555, agg);
|
||||
|
||||
HELPER.assertCounter(pre + "majorCompactionTime_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "majorCompactionInputFileCount_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "majorCompactionInputSize_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "majorCompactionOutputFileCount_num_ops", 1, agg);
|
||||
HELPER.assertCounter(pre + "majorCompactedInputBytes", 400, agg);
|
||||
HELPER.assertCounter(pre + "majorCompactedoutputBytes", 500, agg);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue