HBASE-27483 Expose table and region storefiles accessed days and size to the metrics
This commit is contained in:
parent
5cea8112fd
commit
c6097e18dd
|
@ -56,6 +56,8 @@ public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
|
||||||
String ROW_READS_ONLY_ON_MEMSTORE_DESC = "Row reads happening completely out of memstore";
|
String ROW_READS_ONLY_ON_MEMSTORE_DESC = "Row reads happening completely out of memstore";
|
||||||
String MIXED_ROW_READS = "mixedRowReadsCount";
|
String MIXED_ROW_READS = "mixedRowReadsCount";
|
||||||
String MIXED_ROW_READS_ON_STORE_DESC = "Row reads happening out of files and memstore on store";
|
String MIXED_ROW_READS_ON_STORE_DESC = "Row reads happening out of files and memstore on store";
|
||||||
|
String STOREFILES_ACCESSED_DAYS_AND_SIZE_TEMPLATE = "storeFilesAccessed%sDaysSize";
|
||||||
|
String STOREFILES_ACCESSED_DAYS_AND_SIZE_DESC_TEMPLATE = "Store file accessed %s days size.";
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Close the region's metrics as this region is closing.
|
* Close the region's metrics as this region is closing.
|
||||||
|
|
|
@ -233,6 +233,18 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
|
||||||
this.regionWrapper.getNumReferenceFiles());
|
this.regionWrapper.getNumReferenceFiles());
|
||||||
mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
|
mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
|
||||||
MetricsRegionServerSource.STOREFILE_SIZE_DESC), this.regionWrapper.getStoreFileSize());
|
MetricsRegionServerSource.STOREFILE_SIZE_DESC), this.regionWrapper.getStoreFileSize());
|
||||||
|
|
||||||
|
Map<Integer, Long> sfAccessDaysAndSize = regionWrapper.getStoreFilesAccessedDaysAndSize();
|
||||||
|
if (sfAccessDaysAndSize != null) {
|
||||||
|
for (Map.Entry<Integer, Long> e : sfAccessDaysAndSize.entrySet()) {
|
||||||
|
mrb.addGauge(Interns.info(
|
||||||
|
regionNamePrefix + String
|
||||||
|
.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_TEMPLATE, e.getKey()),
|
||||||
|
String.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_DESC_TEMPLATE,
|
||||||
|
e.getKey())),
|
||||||
|
e.getValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
mrb.addCounter(
|
mrb.addCounter(
|
||||||
Interns.info(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
|
Interns.info(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
|
||||||
MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),
|
MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),
|
||||||
|
|
|
@ -161,4 +161,7 @@ public interface MetricsRegionWrapper {
|
||||||
/** Returns the number of row reads on memstore and file per store */
|
/** Returns the number of row reads on memstore and file per store */
|
||||||
Map<String, Long> getMixedRowReadsCount();
|
Map<String, Long> getMixedRowReadsCount();
|
||||||
|
|
||||||
|
/** Returns the region's store files accessed days and size */
|
||||||
|
Map<Integer, Long> getStoreFilesAccessedDaysAndSize();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -346,6 +346,18 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
|
||||||
Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE,
|
Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE,
|
||||||
MetricsRegionServerSource.STATIC_INDEX_SIZE),
|
MetricsRegionServerSource.STATIC_INDEX_SIZE),
|
||||||
tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString()));
|
tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString()));
|
||||||
|
Map<Integer, Long> sfAccessDaysAndSize =
|
||||||
|
tableWrapperAgg.getStoreFilesAccessedDaysAndSize(tableName.getNameAsString());
|
||||||
|
if (sfAccessDaysAndSize != null) {
|
||||||
|
for (Map.Entry<Integer, Long> e : sfAccessDaysAndSize.entrySet()) {
|
||||||
|
mrb.addGauge(Interns.info(
|
||||||
|
tableNamePrefix + String
|
||||||
|
.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_TEMPLATE, e.getKey()),
|
||||||
|
String.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_DESC_TEMPLATE,
|
||||||
|
e.getKey())),
|
||||||
|
e.getValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
mrb.addCounter(
|
mrb.addCounter(
|
||||||
Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT,
|
Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT,
|
||||||
MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC),
|
MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC),
|
||||||
|
|
|
@ -128,4 +128,7 @@ public interface MetricsTableWrapperAggregate {
|
||||||
|
|
||||||
/** Returns number of row reads from file and memstore per store for this table */
|
/** Returns number of row reads from file and memstore per store for this table */
|
||||||
Map<String, Long> getMixedRowReadsCount(String table);
|
Map<String, Long> getMixedRowReadsCount(String table);
|
||||||
|
|
||||||
|
/** Return the store files accessed days and size for this table */
|
||||||
|
Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table);
|
||||||
}
|
}
|
||||||
|
|
|
@ -154,4 +154,11 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
|
||||||
map.put("table#info", 3L);
|
map.put("table#info", 3L);
|
||||||
return map;
|
return map;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table) {
|
||||||
|
Map<Integer, Long> map = new HashMap<>();
|
||||||
|
map.put(7, 3L);
|
||||||
|
return map;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -232,5 +232,12 @@ public class TestMetricsRegionSourceImpl {
|
||||||
map.put("info", 0L);
|
map.put("info", 0L);
|
||||||
return map;
|
return map;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
|
||||||
|
Map<Integer, Long> map = new HashMap<>();
|
||||||
|
map.put(7, 0L);
|
||||||
|
return map;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,6 +42,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.ExecutorCompletionService;
|
import java.util.concurrent.ExecutorCompletionService;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
import java.util.concurrent.ThreadLocalRandom;
|
||||||
import java.util.concurrent.ThreadPoolExecutor;
|
import java.util.concurrent.ThreadPoolExecutor;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
@ -54,6 +55,7 @@ import java.util.function.ToLongFunction;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.LongStream;
|
import java.util.stream.LongStream;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.permission.FsAction;
|
import org.apache.hadoop.fs.permission.FsAction;
|
||||||
|
@ -211,6 +213,9 @@ public class HStore
|
||||||
private AtomicLong majorCompactedCellsSize = new AtomicLong();
|
private AtomicLong majorCompactedCellsSize = new AtomicLong();
|
||||||
|
|
||||||
private final StoreContext storeContext;
|
private final StoreContext storeContext;
|
||||||
|
private long nextSFileAccessTimeLoadTime;
|
||||||
|
private long sfileAccessTimeLoadInterval;
|
||||||
|
private Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap = new HashMap<>();
|
||||||
|
|
||||||
// Used to track the store files which are currently being written. For compaction, if we want to
|
// Used to track the store files which are currently being written. For compaction, if we want to
|
||||||
// compact store file [a, b, c] to [d], then here we will record 'd'. And we will also use it to
|
// compact store file [a, b, c] to [d], then here we will record 'd'. And we will also use it to
|
||||||
|
@ -320,6 +325,10 @@ public class HStore
|
||||||
confPrintThreshold = 10;
|
confPrintThreshold = 10;
|
||||||
}
|
}
|
||||||
this.parallelPutCountPrintThreshold = confPrintThreshold;
|
this.parallelPutCountPrintThreshold = confPrintThreshold;
|
||||||
|
int initialDelay = conf.getInt("hbase.hstore.hfile.load.initialDelay", 60 * 60 * 1000);
|
||||||
|
sfileAccessTimeLoadInterval = conf.getLong("hbase.hstore.hfile.load.interval", 60 * 60 * 1000);
|
||||||
|
nextSFileAccessTimeLoadTime =
|
||||||
|
EnvironmentEdgeManager.currentTime() + ThreadLocalRandom.current().nextInt(initialDelay);
|
||||||
|
|
||||||
LOG.info(
|
LOG.info(
|
||||||
"Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
|
"Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
|
||||||
|
@ -1809,6 +1818,35 @@ public class HStore
|
||||||
return getStoreFileAgeStream().average();
|
return getStoreFileAgeStream().average();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Pair<Long, Long>> getStoreFilesAccessTimeAndSize() {
|
||||||
|
long now = EnvironmentEdgeManager.currentTime();
|
||||||
|
if (now < nextSFileAccessTimeLoadTime) {
|
||||||
|
return sfAccessTimeAndSizeMap;
|
||||||
|
}
|
||||||
|
Collection<HStoreFile> storeFiles = this.storeEngine.getStoreFileManager().getStorefiles();
|
||||||
|
Map<String, Pair<Long, Long>> tmpSFAccessTimeAndSizeMap = new HashMap<>();
|
||||||
|
for (HStoreFile sf : storeFiles) {
|
||||||
|
if (sf.getReader() == null) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
FileStatus fileStatus;
|
||||||
|
try {
|
||||||
|
fileStatus = sf.getFileInfo().getFileStatus();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.warn(e.getMessage());
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
if (fileStatus != null) {
|
||||||
|
tmpSFAccessTimeAndSizeMap.put(fileStatus.getPath().getName(),
|
||||||
|
Pair.newPair(fileStatus.getAccessTime(), fileStatus.getLen()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
this.sfAccessTimeAndSizeMap = tmpSFAccessTimeAndSizeMap;
|
||||||
|
nextSFileAccessTimeLoadTime = now + sfileAccessTimeLoadInterval;
|
||||||
|
return tmpSFAccessTimeAndSizeMap;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getNumReferenceFiles() {
|
public long getNumReferenceFiles() {
|
||||||
return this.storeEngine.getStoreFileManager().getStorefiles().stream()
|
return this.storeEngine.getStoreFileManager().getStorefiles().stream()
|
||||||
|
|
|
@ -23,6 +23,7 @@ import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.OptionalDouble;
|
import java.util.OptionalDouble;
|
||||||
import java.util.OptionalLong;
|
import java.util.OptionalLong;
|
||||||
|
import java.util.TreeMap;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.ScheduledFuture;
|
import java.util.concurrent.ScheduledFuture;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -30,6 +31,7 @@ import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
import org.apache.hadoop.hbase.client.TableDescriptor;
|
import org.apache.hadoop.hbase.client.TableDescriptor;
|
||||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.metrics2.MetricsExecutor;
|
import org.apache.hadoop.metrics2.MetricsExecutor;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
|
@ -59,15 +61,32 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
||||||
private long maxCompactionQueueSize;
|
private long maxCompactionQueueSize;
|
||||||
private Map<String, Long> readsOnlyFromMemstore;
|
private Map<String, Long> readsOnlyFromMemstore;
|
||||||
private Map<String, Long> mixedReadsOnStore;
|
private Map<String, Long> mixedReadsOnStore;
|
||||||
|
private Map<Integer, Long> storeFilesAccessedDaysAndSize;
|
||||||
|
|
||||||
|
private int[] storeFilesAccessedDaysThresholds;
|
||||||
private ScheduledFuture<?> regionMetricsUpdateTask;
|
private ScheduledFuture<?> regionMetricsUpdateTask;
|
||||||
|
|
||||||
|
// Count the size of the region's storefiles according to the access time,
|
||||||
|
// and set the size to the accessed-days interval to expose it to metrics.
|
||||||
|
// the accessed-days interval default value is "7,30,90", we can change it through
|
||||||
|
// 'hbase.region.hfile.accessed.days.thresholds'. see HBASE-27483 for details.
|
||||||
|
public static final long ONE_DAY_MS = 24 * 60 * 60 * 1000;
|
||||||
|
public static final String STOREFILES_ACCESSED_DAYS_THRESHOLDS =
|
||||||
|
"hbase.region.hfile.accessed.days.thresholds";
|
||||||
|
public static final int[] STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT = { 7, 30, 90 };
|
||||||
|
|
||||||
public MetricsRegionWrapperImpl(HRegion region) {
|
public MetricsRegionWrapperImpl(HRegion region) {
|
||||||
this.region = region;
|
this.region = region;
|
||||||
this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
|
this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
|
||||||
this.runnable = new HRegionMetricsWrapperRunnable();
|
this.runnable = new HRegionMetricsWrapperRunnable();
|
||||||
this.regionMetricsUpdateTask =
|
this.regionMetricsUpdateTask =
|
||||||
this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS);
|
this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS);
|
||||||
|
|
||||||
|
storeFilesAccessedDaysThresholds =
|
||||||
|
region.getReadOnlyConfiguration().getInts(STOREFILES_ACCESSED_DAYS_THRESHOLDS);
|
||||||
|
if (storeFilesAccessedDaysThresholds == null || storeFilesAccessedDaysThresholds.length == 0) {
|
||||||
|
storeFilesAccessedDaysThresholds = STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -243,6 +262,11 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
||||||
return mixedReadsOnStore;
|
return mixedReadsOnStore;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
|
||||||
|
return storeFilesAccessedDaysAndSize;
|
||||||
|
}
|
||||||
|
|
||||||
public class HRegionMetricsWrapperRunnable implements Runnable {
|
public class HRegionMetricsWrapperRunnable implements Runnable {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -259,8 +283,12 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
||||||
long tempMaxFlushQueueSize = 0;
|
long tempMaxFlushQueueSize = 0;
|
||||||
long avgAgeNumerator = 0;
|
long avgAgeNumerator = 0;
|
||||||
long numHFiles = 0;
|
long numHFiles = 0;
|
||||||
if (region.stores != null) {
|
Map<Integer, Long> tmpStoreFileAccessDaysAndSize = new TreeMap<>();
|
||||||
for (HStore store : region.stores.values()) {
|
for (int threshold : storeFilesAccessedDaysThresholds) {
|
||||||
|
tmpStoreFileAccessDaysAndSize.put(threshold, 0L);
|
||||||
|
}
|
||||||
|
if (region.getStores() != null) {
|
||||||
|
for (HStore store : region.getStores()) {
|
||||||
tempNumStoreFiles += store.getStorefilesCount();
|
tempNumStoreFiles += store.getStorefilesCount();
|
||||||
int currentStoreRefCount = store.getStoreRefCount();
|
int currentStoreRefCount = store.getStoreRefCount();
|
||||||
tempStoreRefCount += currentStoreRefCount;
|
tempStoreRefCount += currentStoreRefCount;
|
||||||
|
@ -313,9 +341,24 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
|
||||||
tempVal += store.getMemstoreOnlyRowReadsCount();
|
tempVal += store.getMemstoreOnlyRowReadsCount();
|
||||||
}
|
}
|
||||||
readsOnlyFromMemstore.put(store.getColumnFamilyName(), tempVal);
|
readsOnlyFromMemstore.put(store.getColumnFamilyName(), tempVal);
|
||||||
|
|
||||||
|
Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap =
|
||||||
|
store.getStoreFilesAccessTimeAndSize();
|
||||||
|
long now = EnvironmentEdgeManager.currentTime();
|
||||||
|
for (Pair<Long, Long> pair : sfAccessTimeAndSizeMap.values()) {
|
||||||
|
long accessTime = pair.getFirst();
|
||||||
|
long size = pair.getSecond();
|
||||||
|
for (int threshold : storeFilesAccessedDaysThresholds) {
|
||||||
|
long sumSize = tmpStoreFileAccessDaysAndSize.get(threshold);
|
||||||
|
if ((now - accessTime) >= threshold * ONE_DAY_MS) {
|
||||||
|
sumSize = sumSize + size;
|
||||||
|
tmpStoreFileAccessDaysAndSize.put(threshold, sumSize);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
storeFilesAccessedDaysAndSize = tmpStoreFileAccessDaysAndSize;
|
||||||
numStoreFiles = tempNumStoreFiles;
|
numStoreFiles = tempNumStoreFiles;
|
||||||
storeRefCount = tempStoreRefCount;
|
storeRefCount = tempStoreRefCount;
|
||||||
maxCompactedStoreFileRefCount = tempMaxCompactedStoreFileRefCount;
|
maxCompactedStoreFileRefCount = tempMaxCompactedStoreFileRefCount;
|
||||||
|
|
|
@ -17,11 +17,16 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.hbase.regionserver;
|
package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.MetricsRegionWrapperImpl.ONE_DAY_MS;
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.MetricsRegionWrapperImpl.STOREFILES_ACCESSED_DAYS_THRESHOLDS;
|
||||||
|
import static org.apache.hadoop.hbase.regionserver.MetricsRegionWrapperImpl.STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.TreeMap;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
import java.util.concurrent.ScheduledExecutorService;
|
import java.util.concurrent.ScheduledExecutorService;
|
||||||
import java.util.concurrent.ScheduledFuture;
|
import java.util.concurrent.ScheduledFuture;
|
||||||
|
@ -29,6 +34,8 @@ import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.metrics2.MetricsExecutor;
|
import org.apache.hadoop.metrics2.MetricsExecutor;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
|
|
||||||
|
@ -43,6 +50,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
||||||
private ScheduledFuture<?> tableMetricsUpdateTask;
|
private ScheduledFuture<?> tableMetricsUpdateTask;
|
||||||
private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap =
|
private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap =
|
||||||
new ConcurrentHashMap<>();
|
new ConcurrentHashMap<>();
|
||||||
|
private int[] storeFilesAccessedDaysThresholds;
|
||||||
|
|
||||||
public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
|
public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
|
||||||
this.regionServer = regionServer;
|
this.regionServer = regionServer;
|
||||||
|
@ -52,6 +60,12 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
||||||
this.runnable = new TableMetricsWrapperRunnable();
|
this.runnable = new TableMetricsWrapperRunnable();
|
||||||
this.tableMetricsUpdateTask =
|
this.tableMetricsUpdateTask =
|
||||||
this.executor.scheduleWithFixedDelay(this.runnable, period, period, TimeUnit.MILLISECONDS);
|
this.executor.scheduleWithFixedDelay(this.runnable, period, period, TimeUnit.MILLISECONDS);
|
||||||
|
|
||||||
|
storeFilesAccessedDaysThresholds =
|
||||||
|
regionServer.getConfiguration().getInts(STOREFILES_ACCESSED_DAYS_THRESHOLDS);
|
||||||
|
if (storeFilesAccessedDaysThresholds == null || storeFilesAccessedDaysThresholds.length == 0) {
|
||||||
|
storeFilesAccessedDaysThresholds = STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public class TableMetricsWrapperRunnable implements Runnable {
|
public class TableMetricsWrapperRunnable implements Runnable {
|
||||||
|
@ -64,6 +78,11 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
||||||
MetricsTableValues mt = localMetricsTableMap.get(tbl);
|
MetricsTableValues mt = localMetricsTableMap.get(tbl);
|
||||||
if (mt == null) {
|
if (mt == null) {
|
||||||
mt = new MetricsTableValues();
|
mt = new MetricsTableValues();
|
||||||
|
Map<Integer, Long> tmpStoreFileAccessDaysAndSize = new TreeMap<>();
|
||||||
|
for (int threshold : storeFilesAccessedDaysThresholds) {
|
||||||
|
tmpStoreFileAccessDaysAndSize.put(threshold, 0L);
|
||||||
|
}
|
||||||
|
mt.storeFilesAccessedDaysAndSize = tmpStoreFileAccessDaysAndSize;
|
||||||
localMetricsTableMap.put(tbl, mt);
|
localMetricsTableMap.put(tbl, mt);
|
||||||
}
|
}
|
||||||
long memstoreReadCount = 0L;
|
long memstoreReadCount = 0L;
|
||||||
|
@ -115,10 +134,23 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
||||||
// accumulate the count
|
// accumulate the count
|
||||||
mt.perStoreMemstoreOnlyReadCount.put(tempKey, memstoreReadCount);
|
mt.perStoreMemstoreOnlyReadCount.put(tempKey, memstoreReadCount);
|
||||||
mt.perStoreMixedReadCount.put(tempKey, mixedReadCount);
|
mt.perStoreMixedReadCount.put(tempKey, mixedReadCount);
|
||||||
|
Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap =
|
||||||
|
store.getStoreFilesAccessTimeAndSize();
|
||||||
|
long now = EnvironmentEdgeManager.currentTime();
|
||||||
|
for (Pair<Long, Long> pair : sfAccessTimeAndSizeMap.values()) {
|
||||||
|
Long accessTime = pair.getFirst();
|
||||||
|
Long size = pair.getSecond();
|
||||||
|
for (int threshold : storeFilesAccessedDaysThresholds) {
|
||||||
|
Long sumSize = mt.storeFilesAccessedDaysAndSize.get(threshold);
|
||||||
|
if ((now - accessTime) >= threshold * ONE_DAY_MS) {
|
||||||
|
sumSize = sumSize + size;
|
||||||
|
mt.storeFilesAccessedDaysAndSize.put(threshold, sumSize);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
mt.regionCount += 1;
|
mt.regionCount += 1;
|
||||||
|
|
||||||
mt.readRequestCount += r.getReadRequestsCount();
|
mt.readRequestCount += r.getReadRequestsCount();
|
||||||
mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();
|
mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();
|
||||||
mt.writeRequestCount += r.getWriteRequestsCount();
|
mt.writeRequestCount += r.getWriteRequestsCount();
|
||||||
|
@ -178,6 +210,16 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table) {
|
||||||
|
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||||
|
if (metricsTable == null) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
return metricsTable.storeFilesAccessedDaysAndSize;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long getCpRequestsCount(String table) {
|
public long getCpRequestsCount(String table) {
|
||||||
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
|
||||||
|
@ -419,6 +461,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
|
||||||
long cpRequestCount;
|
long cpRequestCount;
|
||||||
Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
|
Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
|
||||||
Map<String, Long> perStoreMixedReadCount = new HashMap<>();
|
Map<String, Long> perStoreMixedReadCount = new HashMap<>();
|
||||||
|
Map<Integer, Long> storeFilesAccessedDaysAndSize = new TreeMap<>();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.OptionalDouble;
|
import java.util.OptionalDouble;
|
||||||
import java.util.OptionalLong;
|
import java.util.OptionalLong;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -28,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
import org.apache.hadoop.hbase.client.RegionInfo;
|
import org.apache.hadoop.hbase.client.RegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.yetus.audience.InterfaceAudience;
|
import org.apache.yetus.audience.InterfaceAudience;
|
||||||
import org.apache.yetus.audience.InterfaceStability;
|
import org.apache.yetus.audience.InterfaceStability;
|
||||||
|
|
||||||
|
@ -252,4 +254,7 @@ public interface Store {
|
||||||
* loaded.
|
* loaded.
|
||||||
*/
|
*/
|
||||||
long getBloomFilterEligibleRequestsCount();
|
long getBloomFilterEligibleRequestsCount();
|
||||||
|
|
||||||
|
/** Returns Access time and size of store files in this store */
|
||||||
|
Map<String, Pair<Long, Long>> getStoreFilesAccessTimeAndSize();
|
||||||
}
|
}
|
||||||
|
|
|
@ -198,4 +198,11 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
|
||||||
map.put("info", 0L);
|
map.put("info", 0L);
|
||||||
return map;
|
return map;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
|
||||||
|
Map<Integer, Long> map = new HashMap<>();
|
||||||
|
map.put(7, 0L);
|
||||||
|
return map;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,11 +21,18 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertNotNull;
|
import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertNull;
|
import static org.junit.Assert.assertNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.mockito.Mockito.doReturn;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
|
import java.util.OptionalDouble;
|
||||||
|
import java.util.OptionalLong;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.hbase.CompatibilityFactory;
|
import org.apache.hadoop.hbase.CompatibilityFactory;
|
||||||
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
import org.apache.hadoop.hbase.HBaseClassTestRule;
|
||||||
|
@ -35,6 +42,7 @@ import org.apache.hadoop.hbase.HRegionLocation;
|
||||||
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
import org.apache.hadoop.hbase.NamespaceDescriptor;
|
||||||
import org.apache.hadoop.hbase.SingleProcessHBaseCluster;
|
import org.apache.hadoop.hbase.SingleProcessHBaseCluster;
|
||||||
import org.apache.hadoop.hbase.TableName;
|
import org.apache.hadoop.hbase.TableName;
|
||||||
|
import org.apache.hadoop.hbase.Waiter;
|
||||||
import org.apache.hadoop.hbase.client.Admin;
|
import org.apache.hadoop.hbase.client.Admin;
|
||||||
import org.apache.hadoop.hbase.client.Append;
|
import org.apache.hadoop.hbase.client.Append;
|
||||||
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
|
||||||
|
@ -61,6 +69,8 @@ import org.apache.hadoop.hbase.test.MetricsAssertHelper;
|
||||||
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
import org.apache.hadoop.hbase.testclassification.LargeTests;
|
||||||
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
|
||||||
import org.apache.hadoop.hbase.util.Bytes;
|
import org.apache.hadoop.hbase.util.Bytes;
|
||||||
|
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||||
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.apache.hadoop.hbase.util.Threads;
|
import org.apache.hadoop.hbase.util.Threads;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
|
@ -116,6 +126,8 @@ public class TestRegionServerMetrics {
|
||||||
conf.setInt("hbase.hstore.compaction.max", 100);
|
conf.setInt("hbase.hstore.compaction.max", 100);
|
||||||
conf.setInt("hbase.regionserver.periodicmemstoreflusher.rangeofdelayseconds", 4 * 60);
|
conf.setInt("hbase.regionserver.periodicmemstoreflusher.rangeofdelayseconds", 4 * 60);
|
||||||
conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
|
conf.setInt(HConstants.REGIONSERVER_INFO_PORT, -1);
|
||||||
|
conf.set("hbase.region.hfile.accessed.days.thresholds", "7,30,90");
|
||||||
|
conf.setLong("hbase.hstore.hfile.load.initialDelay", 1 * 60 * 1000);
|
||||||
|
|
||||||
TEST_UTIL.startMiniCluster();
|
TEST_UTIL.startMiniCluster();
|
||||||
cluster = TEST_UTIL.getHBaseCluster();
|
cluster = TEST_UTIL.getHBaseCluster();
|
||||||
|
@ -382,6 +394,122 @@ public class TestRegionServerMetrics {
|
||||||
assertCounter("appendNumOps", 73);
|
assertCounter("appendNumOps", 73);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testRegionStoreFilesAccessedDaysAndSize() throws Exception {
|
||||||
|
HRegion region = mock(HRegion.class);
|
||||||
|
HStore hStore = mock(HStore.class);
|
||||||
|
MemStoreSize memStoreSize = mock(MemStoreSize.class);
|
||||||
|
|
||||||
|
doReturn(conf).when(region).getReadOnlyConfiguration();
|
||||||
|
doReturn(getSFAccessTimeAndSizeMap()).when(hStore).getStoreFilesAccessTimeAndSize();
|
||||||
|
doReturn(memStoreSize).when(hStore).getMemStoreSize();
|
||||||
|
doReturn(0L).when(memStoreSize).getDataSize();
|
||||||
|
doReturn(0L).when(hStore).getStorefilesSize();
|
||||||
|
doReturn(OptionalLong.empty()).when(hStore).getMaxStoreFileAge();
|
||||||
|
doReturn(OptionalLong.empty()).when(hStore).getMinStoreFileAge();
|
||||||
|
doReturn(0L).when(hStore).getNumHFiles();
|
||||||
|
doReturn(0L).when(hStore).getNumReferenceFiles();
|
||||||
|
doReturn(OptionalDouble.empty()).when(hStore).getAvgStoreFileAge();
|
||||||
|
doReturn(Collections.singletonList(hStore)).when(region).getStores();
|
||||||
|
|
||||||
|
TestMetricsRegionWrapperImpl mrwi = new TestMetricsRegionWrapperImpl(region);
|
||||||
|
MetricsRegion mr = new MetricsRegion(mrwi, conf);
|
||||||
|
MetricsRegionAggregateSource agg = mr.getSource().getAggregateSource();
|
||||||
|
|
||||||
|
String prefix = "namespace_" + mrwi.getNamespace() + "_table_" + mrwi.getTableName()
|
||||||
|
+ "_region_" + mrwi.getRegionName() + "_metric_";
|
||||||
|
|
||||||
|
TEST_UTIL.waitFor(120000, 1000, new Waiter.ExplainingPredicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() {
|
||||||
|
return metricsHelper.checkGaugeExists(prefix + "storeFilesAccessed7DaysSize", agg)
|
||||||
|
&& metricsHelper.getGaugeLong(prefix + "storeFilesAccessed7DaysSize", agg) > 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String explainFailure() {
|
||||||
|
return prefix
|
||||||
|
+ "storeFilesAccessed7DaysSize value is 0 or does not exists in the region metrics";
|
||||||
|
}
|
||||||
|
});
|
||||||
|
metricsHelper.assertGauge(prefix + "storeFilesAccessed7DaysSize", 3000L, agg);
|
||||||
|
metricsHelper.assertGauge(prefix + "storeFilesAccessed30DaysSize", 2000L, agg);
|
||||||
|
metricsHelper.assertGauge(prefix + "storeFilesAccessed90DaysSize", 1000L, agg);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTableStoreFilesAccessedDaysAndSize() throws Exception {
|
||||||
|
String testTable = "test_table";
|
||||||
|
HRegion region = mock(HRegion.class);
|
||||||
|
HStore hStore = mock(HStore.class);
|
||||||
|
MemStoreSize memStoreSize = mock(MemStoreSize.class);
|
||||||
|
|
||||||
|
doReturn(conf).when(region).getReadOnlyConfiguration();
|
||||||
|
TableDescriptor htd = TableDescriptorBuilder.newBuilder(TableName.valueOf(testTable)).build();
|
||||||
|
doReturn(htd).when(region).getTableDescriptor();
|
||||||
|
doReturn(Collections.singletonList(hStore)).when(region).getStores();
|
||||||
|
doReturn(0L).when(region).getReadRequestsCount();
|
||||||
|
doReturn(0L).when(region).getFilteredReadRequestsCount();
|
||||||
|
doReturn(0L).when(region).getWriteRequestsCount();
|
||||||
|
doReturn(0L).when(hStore).getStorefilesSize();
|
||||||
|
doReturn(OptionalLong.empty()).when(hStore).getMaxStoreFileAge();
|
||||||
|
doReturn(OptionalLong.empty()).when(hStore).getMinStoreFileAge();
|
||||||
|
doReturn(OptionalDouble.empty()).when(hStore).getAvgStoreFileAge();
|
||||||
|
doReturn(0L).when(hStore).getNumHFiles();
|
||||||
|
doReturn(0L).when(hStore).getNumReferenceFiles();
|
||||||
|
doReturn(0).when(hStore).getStorefilesCount();
|
||||||
|
doReturn(0L).when(hStore).getTotalStaticIndexSize();
|
||||||
|
doReturn(0L).when(hStore).getTotalStaticBloomSize();
|
||||||
|
doReturn(0L).when(hStore).getBloomFilterRequestsCount();
|
||||||
|
doReturn(0L).when(hStore).getBloomFilterNegativeResultsCount();
|
||||||
|
doReturn(0L).when(hStore).getBloomFilterEligibleRequestsCount();
|
||||||
|
doReturn(0L).when(hStore).getMemstoreOnlyRowReadsCount();
|
||||||
|
doReturn(0L).when(hStore).getMixedRowReadsCount();
|
||||||
|
doReturn(getSFAccessTimeAndSizeMap()).when(hStore).getStoreFilesAccessTimeAndSize();
|
||||||
|
doReturn(memStoreSize).when(hStore).getMemStoreSize();
|
||||||
|
|
||||||
|
doReturn(0L).when(memStoreSize).getDataSize();
|
||||||
|
doReturn(0L).when(memStoreSize).getHeapSize();
|
||||||
|
doReturn(0L).when(memStoreSize).getOffHeapSize();
|
||||||
|
|
||||||
|
HRegionServer rs = mock(HRegionServer.class);
|
||||||
|
doReturn(conf).when(rs).getConfiguration();
|
||||||
|
doReturn(Collections.singletonList(region)).when(rs).getOnlineRegionsLocalContext();
|
||||||
|
|
||||||
|
MetricsTableWrapperAggregateImpl mrwi = new MetricsTableWrapperAggregateImpl(rs);
|
||||||
|
MetricsTable mr = new MetricsTable(mrwi);
|
||||||
|
MetricsTableAggregateSource agg = mr.getTableSourceAgg();
|
||||||
|
|
||||||
|
String prefix = "namespace_" + NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR + "_table_"
|
||||||
|
+ testTable + "_metric_";
|
||||||
|
|
||||||
|
TEST_UTIL.waitFor(120000, 1000, new Waiter.ExplainingPredicate<Exception>() {
|
||||||
|
@Override
|
||||||
|
public boolean evaluate() {
|
||||||
|
return metricsHelper.checkGaugeExists(prefix + "storeFilesAccessed7DaysSize", agg)
|
||||||
|
&& metricsHelper.getGaugeLong(prefix + "storeFilesAccessed7DaysSize", agg) > 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String explainFailure() {
|
||||||
|
return prefix
|
||||||
|
+ "storeFilesAccessed7DaysSize value is 0 or does not exists in the table metrics";
|
||||||
|
}
|
||||||
|
});
|
||||||
|
metricsHelper.assertGauge(prefix + "storeFilesAccessed7DaysSize", 3000L, agg);
|
||||||
|
metricsHelper.assertGauge(prefix + "storeFilesAccessed30DaysSize", 2000L, agg);
|
||||||
|
metricsHelper.assertGauge(prefix + "storeFilesAccessed90DaysSize", 1000L, agg);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<String, Pair<Long, Long>> getSFAccessTimeAndSizeMap() {
|
||||||
|
long now = EnvironmentEdgeManager.currentTime();
|
||||||
|
Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap = new HashMap<>();
|
||||||
|
sfAccessTimeAndSizeMap.put("teststorefile1", Pair.newPair(now - 8 * 24 * 3600 * 1000L, 1000L));
|
||||||
|
sfAccessTimeAndSizeMap.put("teststorefile2", Pair.newPair(now - 31 * 24 * 3600 * 1000L, 1000L));
|
||||||
|
sfAccessTimeAndSizeMap.put("teststorefile3", Pair.newPair(now - 91 * 24 * 3600 * 1000L, 1000L));
|
||||||
|
return sfAccessTimeAndSizeMap;
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testScanSize() throws Exception {
|
public void testScanSize() throws Exception {
|
||||||
doNPuts(100, true); // batch put
|
doNPuts(100, true); // batch put
|
||||||
|
@ -598,4 +726,168 @@ public class TestRegionServerMetrics {
|
||||||
assertEquals("Total zero-byte read bytes should be equal to 0", 0,
|
assertEquals("Total zero-byte read bytes should be equal to 0", 0,
|
||||||
metricsRegionServer.getRegionServerWrapper().getZeroCopyBytesRead());
|
metricsRegionServer.getRegionServerWrapper().getZeroCopyBytesRead());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class TestMetricsRegionWrapperImpl extends MetricsRegionWrapperImpl {
|
||||||
|
public TestMetricsRegionWrapperImpl(HRegion region) {
|
||||||
|
super(region);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getTableName() {
|
||||||
|
return "testtable";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getNamespace() {
|
||||||
|
return "default";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getRegionName() {
|
||||||
|
return "testregion";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumStores() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumStoreFiles() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getMemStoreSize() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getStoreFileSize() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getStoreRefCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getMaxCompactedStoreFileRefCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getReadRequestCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getCpRequestCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getFilteredReadRequestCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getWriteRequestCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumFilesCompacted() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumBytesCompacted() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumCompactionsCompleted() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getLastMajorCompactionAge() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getTotalRequestCount() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumCompactionsFailed() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumCompactionsQueued() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumFlushesQueued() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getMaxCompactionQueueSize() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getMaxFlushQueueSize() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getMaxStoreFileAge() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getMinStoreFileAge() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getAvgStoreFileAge() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long getNumReferenceFiles() {
|
||||||
|
return 0L;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getRegionHashCode() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Long> getMemstoreOnlyRowReadsCount() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, Long> getMixedRowReadsCount() {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the replica id of this region.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public int getReplicaId() {
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue