Skip to content

Commit f08e679

Browse files
committed
HBASE-27483 Expose table and region storefiles accessed days and size to the metrics
1 parent 5cea811 commit f08e679

File tree

13 files changed

+478
-4
lines changed

13 files changed

+478
-4
lines changed

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSource.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -56,6 +56,8 @@ public interface MetricsRegionSource extends Comparable<MetricsRegionSource> {
5656
String ROW_READS_ONLY_ON_MEMSTORE_DESC = "Row reads happening completely out of memstore";
5757
String MIXED_ROW_READS = "mixedRowReadsCount";
5858
String MIXED_ROW_READS_ON_STORE_DESC = "Row reads happening out of files and memstore on store";
59+
String STOREFILES_ACCESSED_DAYS_AND_SIZE_TEMPLATE = "storeFilesAccessed%sDaysSize";
60+
String STOREFILES_ACCESSED_DAYS_AND_SIZE_DESC_TEMPLATE = "Store file accessed %s days size.";
5961

6062
/**
6163
* Close the region's metrics as this region is closing.

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionSourceImpl.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -233,6 +233,18 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
233233
this.regionWrapper.getNumReferenceFiles());
234234
mrb.addGauge(Interns.info(regionNamePrefix + MetricsRegionServerSource.STOREFILE_SIZE,
235235
MetricsRegionServerSource.STOREFILE_SIZE_DESC), this.regionWrapper.getStoreFileSize());
236+
237+
Map<Integer, Long> sfAccessDaysAndSize = regionWrapper.getStoreFilesAccessedDaysAndSize();
238+
if (sfAccessDaysAndSize != null) {
239+
for (Map.Entry<Integer, Long> e : sfAccessDaysAndSize.entrySet()) {
240+
mrb.addGauge(Interns.info(
241+
regionNamePrefix + String
242+
.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_TEMPLATE, e.getKey()),
243+
String.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_DESC_TEMPLATE,
244+
e.getKey())),
245+
e.getValue());
246+
}
247+
}
236248
mrb.addCounter(
237249
Interns.info(regionNamePrefix + MetricsRegionSource.COMPACTIONS_COMPLETED_COUNT,
238250
MetricsRegionSource.COMPACTIONS_COMPLETED_DESC),

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapper.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -161,4 +161,7 @@ public interface MetricsRegionWrapper {
161161
/** Returns the number of row reads on memstore and file per store */
162162
Map<String, Long> getMixedRowReadsCount();
163163

164+
/** Returns the region's store files accessed days and size */
165+
Map<Integer, Long> getStoreFilesAccessedDaysAndSize();
166+
164167
}

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableSourceImpl.java

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -346,6 +346,18 @@ void snapshot(MetricsRecordBuilder mrb, boolean ignored) {
346346
Interns.info(tableNamePrefix + MetricsRegionServerSource.STATIC_INDEX_SIZE,
347347
MetricsRegionServerSource.STATIC_INDEX_SIZE),
348348
tableWrapperAgg.getStaticIndexSize(tableName.getNameAsString()));
349+
Map<Integer, Long> sfAccessDaysAndSize =
350+
tableWrapperAgg.getStoreFilesAccessedDaysAndSize(tableName.getNameAsString());
351+
if (sfAccessDaysAndSize != null) {
352+
for (Map.Entry<Integer, Long> e : sfAccessDaysAndSize.entrySet()) {
353+
mrb.addGauge(Interns.info(
354+
tableNamePrefix + String
355+
.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_TEMPLATE, e.getKey()),
356+
String.format(MetricsRegionSource.STOREFILES_ACCESSED_DAYS_AND_SIZE_DESC_TEMPLATE,
357+
e.getKey())),
358+
e.getValue());
359+
}
360+
}
349361
mrb.addCounter(
350362
Interns.info(tableNamePrefix + MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT,
351363
MetricsRegionServerSource.BLOOM_FILTER_REQUESTS_COUNT_DESC),

hbase-hadoop-compat/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregate.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -128,4 +128,7 @@ public interface MetricsTableWrapperAggregate {
128128

129129
/** Returns number of row reads from file and memstore per store for this table */
130130
Map<String, Long> getMixedRowReadsCount(String table);
131+
132+
/** Return the store files accessed days and size for this table */
133+
Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table);
131134
}

hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -154,4 +154,11 @@ public Map<String, Long> getMixedRowReadsCount(String table) {
154154
map.put("table#info", 3L);
155155
return map;
156156
}
157+
158+
@Override
159+
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table) {
160+
Map<Integer, Long> map = new HashMap<>();
161+
map.put(7, 3L);
162+
return map;
163+
}
157164
}

hbase-hadoop-compat/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsRegionSourceImpl.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -232,5 +232,12 @@ public Map<String, Long> getMixedRowReadsCount() {
232232
map.put("info", 0L);
233233
return map;
234234
}
235+
236+
@Override
237+
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
238+
Map<Integer, Long> map = new HashMap<>();
239+
map.put(7, 0L);
240+
return map;
241+
}
235242
}
236243
}

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java

Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@
4242
import java.util.concurrent.ExecutionException;
4343
import java.util.concurrent.ExecutorCompletionService;
4444
import java.util.concurrent.Future;
45+
import java.util.concurrent.ThreadLocalRandom;
4546
import java.util.concurrent.ThreadPoolExecutor;
4647
import java.util.concurrent.atomic.AtomicBoolean;
4748
import java.util.concurrent.atomic.AtomicInteger;
@@ -54,6 +55,7 @@
5455
import java.util.stream.Collectors;
5556
import java.util.stream.LongStream;
5657
import org.apache.hadoop.conf.Configuration;
58+
import org.apache.hadoop.fs.FileStatus;
5759
import org.apache.hadoop.fs.FileSystem;
5860
import org.apache.hadoop.fs.Path;
5961
import org.apache.hadoop.fs.permission.FsAction;
@@ -211,6 +213,9 @@ public class HStore
211213
private AtomicLong majorCompactedCellsSize = new AtomicLong();
212214

213215
private final StoreContext storeContext;
216+
private long nextSFileAccessTimeLoadTime;
217+
private long sfileAccessTimeLoadInterval;
218+
private Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap = new HashMap<>();
214219

215220
// Used to track the store files which are currently being written. For compaction, if we want to
216221
// 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 @@ protected HStore(final HRegion region, final ColumnFamilyDescriptor family,
320325
confPrintThreshold = 10;
321326
}
322327
this.parallelPutCountPrintThreshold = confPrintThreshold;
328+
int initialDelay = conf.getInt("hbase.hstore.hfile.load.initialDelay", 60 * 60 * 1000);
329+
sfileAccessTimeLoadInterval = conf.getLong("hbase.hstore.hfile.load.interval", 60 * 60 * 1000);
330+
nextSFileAccessTimeLoadTime =
331+
EnvironmentEdgeManager.currentTime() + ThreadLocalRandom.current().nextInt(initialDelay);
323332

324333
LOG.info(
325334
"Store={}, memstore type={}, storagePolicy={}, verifyBulkLoads={}, "
@@ -1809,6 +1818,35 @@ public OptionalDouble getAvgStoreFileAge() {
18091818
return getStoreFileAgeStream().average();
18101819
}
18111820

1821+
@Override
1822+
public Map<String, Pair<Long, Long>> getStoreFilesAccessTimeAndSize() {
1823+
long now = EnvironmentEdgeManager.currentTime();
1824+
if (now < nextSFileAccessTimeLoadTime) {
1825+
return sfAccessTimeAndSizeMap;
1826+
}
1827+
Collection<HStoreFile> storeFiles = this.storeEngine.getStoreFileManager().getStorefiles();
1828+
Map<String, Pair<Long, Long>> tmpSFAccessTimeAndSizeMap = new HashMap<>();
1829+
for (HStoreFile sf : storeFiles) {
1830+
if (sf.getReader() == null) {
1831+
continue;
1832+
}
1833+
FileStatus fileStatus;
1834+
try {
1835+
fileStatus = sf.getFileInfo().getFileStatus();
1836+
} catch (IOException e) {
1837+
LOG.warn(e.getMessage());
1838+
continue;
1839+
}
1840+
if (fileStatus != null) {
1841+
tmpSFAccessTimeAndSizeMap.put(fileStatus.getPath().getName(),
1842+
Pair.newPair(fileStatus.getAccessTime(), fileStatus.getLen()));
1843+
}
1844+
}
1845+
this.sfAccessTimeAndSizeMap = tmpSFAccessTimeAndSizeMap;
1846+
nextSFileAccessTimeLoadTime = now + sfileAccessTimeLoadInterval;
1847+
return tmpSFAccessTimeAndSizeMap;
1848+
}
1849+
18121850
@Override
18131851
public long getNumReferenceFiles() {
18141852
return this.storeEngine.getStoreFileManager().getStorefiles().stream()

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java

Lines changed: 46 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -23,13 +23,15 @@
2323
import java.util.Map;
2424
import java.util.OptionalDouble;
2525
import java.util.OptionalLong;
26+
import java.util.TreeMap;
2627
import java.util.concurrent.ScheduledExecutorService;
2728
import java.util.concurrent.ScheduledFuture;
2829
import java.util.concurrent.TimeUnit;
2930
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
3031
import org.apache.hadoop.hbase.client.RegionInfo;
3132
import org.apache.hadoop.hbase.client.TableDescriptor;
3233
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
34+
import org.apache.hadoop.hbase.util.Pair;
3335
import org.apache.hadoop.metrics2.MetricsExecutor;
3436
import org.apache.yetus.audience.InterfaceAudience;
3537
import org.slf4j.Logger;
@@ -59,15 +61,32 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
5961
private long maxCompactionQueueSize;
6062
private Map<String, Long> readsOnlyFromMemstore;
6163
private Map<String, Long> mixedReadsOnStore;
64+
private Map<Integer, Long> storeFilesAccessedDaysAndSize;
6265

66+
private int[] storeFilesAccessedDaysThresholds;
6367
private ScheduledFuture<?> regionMetricsUpdateTask;
6468

69+
// Count the size of the region's storefiles according to the access time,
70+
// and set the size to the accessed-days interval to expose it to metrics.
71+
// the accessed-days interval default value is "7,30,90", we can change it through
72+
// 'hbase.region.hfile.accessed.days.thresholds'. see HBASE-27483 for details.
73+
public static final long ONE_DAY_MS = 24 * 60 * 60 * 1000;
74+
public static final String STOREFILES_ACCESSED_DAYS_THRESHOLDS =
75+
"hbase.region.hfile.accessed.days.thresholds";
76+
public static final int[] STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT = { 7, 30, 90 };
77+
6578
public MetricsRegionWrapperImpl(HRegion region) {
6679
this.region = region;
6780
this.executor = CompatibilitySingletonFactory.getInstance(MetricsExecutor.class).getExecutor();
6881
this.runnable = new HRegionMetricsWrapperRunnable();
6982
this.regionMetricsUpdateTask =
7083
this.executor.scheduleWithFixedDelay(this.runnable, PERIOD, PERIOD, TimeUnit.SECONDS);
84+
85+
storeFilesAccessedDaysThresholds =
86+
region.getReadOnlyConfiguration().getInts(STOREFILES_ACCESSED_DAYS_THRESHOLDS);
87+
if (storeFilesAccessedDaysThresholds == null || storeFilesAccessedDaysThresholds.length == 0) {
88+
storeFilesAccessedDaysThresholds = STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT;
89+
}
7190
}
7291

7392
@Override
@@ -243,6 +262,11 @@ public Map<String, Long> getMixedRowReadsCount() {
243262
return mixedReadsOnStore;
244263
}
245264

265+
@Override
266+
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() {
267+
return storeFilesAccessedDaysAndSize;
268+
}
269+
246270
public class HRegionMetricsWrapperRunnable implements Runnable {
247271

248272
@Override
@@ -259,8 +283,12 @@ public void run() {
259283
long tempMaxFlushQueueSize = 0;
260284
long avgAgeNumerator = 0;
261285
long numHFiles = 0;
262-
if (region.stores != null) {
263-
for (HStore store : region.stores.values()) {
286+
Map<Integer, Long> tmpStoreFileAccessDaysAndSize = new TreeMap<>();
287+
for (int threshold : storeFilesAccessedDaysThresholds) {
288+
tmpStoreFileAccessDaysAndSize.put(threshold, 0L);
289+
}
290+
if (region.getStores() != null) {
291+
for (HStore store : region.getStores()) {
264292
tempNumStoreFiles += store.getStorefilesCount();
265293
int currentStoreRefCount = store.getStoreRefCount();
266294
tempStoreRefCount += currentStoreRefCount;
@@ -313,9 +341,24 @@ public void run() {
313341
tempVal += store.getMemstoreOnlyRowReadsCount();
314342
}
315343
readsOnlyFromMemstore.put(store.getColumnFamilyName(), tempVal);
344+
345+
Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap =
346+
store.getStoreFilesAccessTimeAndSize();
347+
long now = EnvironmentEdgeManager.currentTime();
348+
for (Pair<Long, Long> pair : sfAccessTimeAndSizeMap.values()) {
349+
long accessTime = pair.getFirst();
350+
long size = pair.getSecond();
351+
for (int threshold : storeFilesAccessedDaysThresholds) {
352+
long sumSize = tmpStoreFileAccessDaysAndSize.get(threshold);
353+
if ((now - accessTime) >= threshold * ONE_DAY_MS) {
354+
sumSize = sumSize + size;
355+
tmpStoreFileAccessDaysAndSize.put(threshold, sumSize);
356+
}
357+
}
358+
}
316359
}
317360
}
318-
361+
storeFilesAccessedDaysAndSize = tmpStoreFileAccessDaysAndSize;
319362
numStoreFiles = tempNumStoreFiles;
320363
storeRefCount = tempStoreRefCount;
321364
maxCompactedStoreFileRefCount = tempMaxCompactedStoreFileRefCount;

hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java

Lines changed: 44 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,18 +17,25 @@
1717
*/
1818
package org.apache.hadoop.hbase.regionserver;
1919

20+
import static org.apache.hadoop.hbase.regionserver.MetricsRegionWrapperImpl.ONE_DAY_MS;
21+
import static org.apache.hadoop.hbase.regionserver.MetricsRegionWrapperImpl.STOREFILES_ACCESSED_DAYS_THRESHOLDS;
22+
import static org.apache.hadoop.hbase.regionserver.MetricsRegionWrapperImpl.STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT;
23+
2024
import java.io.Closeable;
2125
import java.io.IOException;
2226
import java.util.HashMap;
2327
import java.util.Map;
2428
import java.util.Set;
29+
import java.util.TreeMap;
2530
import java.util.concurrent.ConcurrentHashMap;
2631
import java.util.concurrent.ScheduledExecutorService;
2732
import java.util.concurrent.ScheduledFuture;
2833
import java.util.concurrent.TimeUnit;
2934
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
3035
import org.apache.hadoop.hbase.HConstants;
3136
import org.apache.hadoop.hbase.TableName;
37+
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
38+
import org.apache.hadoop.hbase.util.Pair;
3239
import org.apache.hadoop.metrics2.MetricsExecutor;
3340
import org.apache.yetus.audience.InterfaceAudience;
3441

@@ -43,6 +50,7 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr
4350
private ScheduledFuture<?> tableMetricsUpdateTask;
4451
private ConcurrentHashMap<TableName, MetricsTableValues> metricsTableMap =
4552
new ConcurrentHashMap<>();
53+
private int[] storeFilesAccessedDaysThresholds;
4654

4755
public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
4856
this.regionServer = regionServer;
@@ -52,6 +60,12 @@ public MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) {
5260
this.runnable = new TableMetricsWrapperRunnable();
5361
this.tableMetricsUpdateTask =
5462
this.executor.scheduleWithFixedDelay(this.runnable, period, period, TimeUnit.MILLISECONDS);
63+
64+
storeFilesAccessedDaysThresholds =
65+
regionServer.getConfiguration().getInts(STOREFILES_ACCESSED_DAYS_THRESHOLDS);
66+
if (storeFilesAccessedDaysThresholds == null || storeFilesAccessedDaysThresholds.length == 0) {
67+
storeFilesAccessedDaysThresholds = STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT;
68+
}
5569
}
5670

5771
public class TableMetricsWrapperRunnable implements Runnable {
@@ -64,6 +78,11 @@ public void run() {
6478
MetricsTableValues mt = localMetricsTableMap.get(tbl);
6579
if (mt == null) {
6680
mt = new MetricsTableValues();
81+
Map<Integer, Long> tmpStoreFileAccessDaysAndSize = new TreeMap<>();
82+
for (int threshold : storeFilesAccessedDaysThresholds) {
83+
tmpStoreFileAccessDaysAndSize.put(threshold, 0L);
84+
}
85+
mt.storeFilesAccessedDaysAndSize = tmpStoreFileAccessDaysAndSize;
6786
localMetricsTableMap.put(tbl, mt);
6887
}
6988
long memstoreReadCount = 0L;
@@ -115,10 +134,23 @@ public void run() {
115134
// accumulate the count
116135
mt.perStoreMemstoreOnlyReadCount.put(tempKey, memstoreReadCount);
117136
mt.perStoreMixedReadCount.put(tempKey, mixedReadCount);
137+
Map<String, Pair<Long, Long>> sfAccessTimeAndSizeMap =
138+
store.getStoreFilesAccessTimeAndSize();
139+
Long now = EnvironmentEdgeManager.currentTime();
140+
for (Pair<Long, Long> pair : sfAccessTimeAndSizeMap.values()) {
141+
Long accessTime = pair.getFirst();
142+
Long size = pair.getSecond();
143+
for (int threshold : storeFilesAccessedDaysThresholds) {
144+
Long sumSize = mt.storeFilesAccessedDaysAndSize.get(threshold);
145+
if ((now - accessTime) >= threshold * ONE_DAY_MS) {
146+
sumSize = sumSize + size;
147+
mt.storeFilesAccessedDaysAndSize.put(threshold, sumSize);
148+
}
149+
}
150+
}
118151
}
119152

120153
mt.regionCount += 1;
121-
122154
mt.readRequestCount += r.getReadRequestsCount();
123155
mt.filteredReadRequestCount += r.getFilteredReadRequestsCount();
124156
mt.writeRequestCount += r.getWriteRequestsCount();
@@ -178,6 +210,16 @@ public Map<String, Long> getMixedRowReadsCount(String table) {
178210
}
179211
}
180212

213+
@Override
214+
public Map<Integer, Long> getStoreFilesAccessedDaysAndSize(String table) {
215+
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
216+
if (metricsTable == null) {
217+
return null;
218+
} else {
219+
return metricsTable.storeFilesAccessedDaysAndSize;
220+
}
221+
}
222+
181223
@Override
182224
public long getCpRequestsCount(String table) {
183225
MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table));
@@ -419,6 +461,7 @@ private static class MetricsTableValues {
419461
long cpRequestCount;
420462
Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>();
421463
Map<String, Long> perStoreMixedReadCount = new HashMap<>();
464+
Map<Integer, Long> storeFilesAccessedDaysAndSize = new TreeMap<>();
422465
}
423466

424467
}

0 commit comments

Comments
 (0)