HBASE-27483 Expose table and region storefiles accessed days and size to the metrics

This commit is contained in:
Liangjun He 2023-05-11 17:51:37 +08:00
parent 5cea8112fd
commit c6097e18dd
13 changed files with 478 additions and 4 deletions

View File

@ -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 MIXED_ROW_READS = "mixedRowReadsCount";
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.

View File

@ -233,6 +233,18 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
this.regionWrapper.getNumReferenceFiles());
mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
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(
Interns.info(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),

View File

@ -161,4 +161,7 @@ public interface MetricsRegionWrapper {
/** Returns the number of row reads on memstore and file per store */
Map<String, Long> getMixedRowReadsCount();
/** Returns the region's store files accessed days and size */
Map<Integer, Long> getStoreFilesAccessedDaysAndSize();
}

View File

@ -346,6 +346,18 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE,
MetricsRegionServerSource.STATIC_INDEX_SIZE),
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(
Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT,
MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC),

View File

@ -128,4 +128,7 @@ public interface MetricsTableWrapperAggregate {
/** Returns number of row reads from file and memstore per store for this table */
Map<String, Long> getMixedRowReadsCount(String table);
/** Return the store files accessed days and size for this table */
Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table);
}

View File

@ -154,4 +154,11 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
map.put("table#info", 3L);
return map;
}
@Override
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table) {
Map<Integer, Long> map = new HashMap<>();
map.put(7, 3L);
return map;
}
}

View File

@ -232,5 +232,12 @@ public class TestMetricsRegionSourceImpl {
map.put("info", 0L);
return map;
}
@Override
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
Map<Integer, Long> map = new HashMap<>();
map.put(7, 0L);
return map;
}
}
}

View File

@ -42,6 +42,7 @@ import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
@ -54,6 +55,7 @@ import java.util.function.ToLongFunction;
import java.util.stream.Collectors;
import java.util.stream.LongStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsAction;
@ -211,6 +213,9 @@ public class HStore
private AtomicLong majorCompactedCellsSize = new AtomicLong();
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
// 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;
}
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(
"Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
@ -1809,6 +1818,35 @@ public class HStore
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
public long getNumReferenceFiles() {
return this.storeEngine.getStoreFileManager().getStorefiles().stream()

View File

@ -23,6 +23,7 @@ import java.util.HashMap;
import java.util.Map;
import java.util.OptionalDouble;
import java.util.OptionalLong;
import java.util.TreeMap;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
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.TableDescriptor;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.metrics2.MetricsExecutor;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@ -59,15 +61,32 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
private long maxCompactionQueueSize;
private Map<String, Long> readsOnlyFromMemstore;
private Map<String, Long> mixedReadsOnStore;
private Map<Integer, Long> storeFilesAccessedDaysAndSize;
private int[] storeFilesAccessedDaysThresholds;
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) {
this.region = region;
this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
this.runnable = new HRegionMetricsWrapperRunnable();
this.regionMetricsUpdateTask =
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
@ -243,6 +262,11 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
return mixedReadsOnStore;
}
@Override
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
return storeFilesAccessedDaysAndSize;
}
public class HRegionMetricsWrapperRunnable implements Runnable {
@Override
@ -259,8 +283,12 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
long tempMaxFlushQueueSize = 0;
long avgAgeNumerator = 0;
long numHFiles = 0;
if (region.stores != null) {
for (HStore store : region.stores.values()) {
Map<Integer, Long> tmpStoreFileAccessDaysAndSize = new TreeMap<>();
for (int threshold : storeFilesAccessedDaysThresholds) {
tmpStoreFileAccessDaysAndSize.put(threshold, 0L);
}
if (region.getStores() != null) {
for (HStore store : region.getStores()) {
tempNumStoreFiles += store.getStorefilesCount();
int currentStoreRefCount = store.getStoreRefCount();
tempStoreRefCount += currentStoreRefCount;
@ -313,9 +341,24 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
tempVal += store.getMemstoreOnlyRowReadsCount();
}
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;
storeRefCount = tempStoreRefCount;
maxCompactedStoreFileRefCount = tempMaxCompactedStoreFileRefCount;

View File

@ -17,11 +17,16 @@
*/
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.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ScheduledExecutorService;
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.HConstants;
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.yetus.audience.InterfaceAudience;
@ -43,6 +50,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
private ScheduledFuture<?> tableMetricsUpdateTask;
private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap =
new ConcurrentHashMap<>();
private int[] storeFilesAccessedDaysThresholds;
public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
this.regionServer = regionServer;
@ -52,6 +60,12 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
this.runnable = new TableMetricsWrapperRunnable();
this.tableMetricsUpdateTask =
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 {
@ -64,6 +78,11 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
MetricsTableValues mt = localMetricsTableMap.get(tbl);
if (mt == null) {
mt = new MetricsTableValues();
Map<Integer, Long> tmpStoreFileAccessDaysAndSize = new TreeMap<>();
for (int threshold : storeFilesAccessedDaysThresholds) {
tmpStoreFileAccessDaysAndSize.put(threshold, 0L);
}
mt.storeFilesAccessedDaysAndSize = tmpStoreFileAccessDaysAndSize;
localMetricsTableMap.put(tbl, mt);
}
long memstoreReadCount = 0L;
@ -115,10 +134,23 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
// accumulate the count
mt.perStoreMemstoreOnlyReadCount.put(tempKey, memstoreReadCount);
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.readRequestCount += r.getReadRequestsCount();
mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();
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
public long getCpRequestsCount(String table) {
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
@ -419,6 +461,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
long cpRequestCount;
Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
Map<String, Long> perStoreMixedReadCount = new HashMap<>();
Map<Integer, Long> storeFilesAccessedDaysAndSize = new TreeMap<>();
}
}

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
import java.util.OptionalDouble;
import java.util.OptionalLong;
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.client.ColumnFamilyDescriptor;
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.InterfaceStability;
@ -252,4 +254,7 @@ public interface Store {
* loaded.
*/
long getBloomFilterEligibleRequestsCount();
/** Returns Access time and size of store files in this store */
Map<String, Pair<Long, Long>> getStoreFilesAccessTimeAndSize();
}

View File

@ -198,4 +198,11 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
map.put("info", 0L);
return map;
}
@Override
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
Map<Integer, Long> map = new HashMap<>();
map.put(7, 0L);
return map;
}
}

View File

@ -21,11 +21,18 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalDouble;
import java.util.OptionalLong;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CompatibilityFactory;
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.SingleProcessHBaseCluster;
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.Append;
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.RegionServerTests;
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.junit.After;
import org.junit.AfterClass;
@ -116,6 +126,8 @@ public class TestRegionServerMetrics {
conf.setInt("hbase.hstore.compaction.max", 100);
conf.setInt("hbase.regionserver.periodicmemstoreflusher.rangeofdelayseconds", 4 * 60);
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();
cluster = TEST_UTIL.getHBaseCluster();
@ -382,6 +394,122 @@ public class TestRegionServerMetrics {
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
public void testScanSize() throws Exception {
doNPuts(100, true); // batch put
@ -598,4 +726,168 @@ public class TestRegionServerMetrics {
assertEquals("Total zero-byte read bytes should be equal to 0", 0,
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;
}
}
}