- 
                Notifications
    You must be signed in to change notification settings 
- Fork 3.4k
HBASE-27483 Expose table and region storefiles accessed days and size to the metrics #4891
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
|  | @@ -23,13 +23,15 @@ | |
| 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; | ||
| 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 }; | ||
| There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we add few line of javadoc or some line in README.md how to use this ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. One more question, if we are using cache / bucket cache heavily, will this metric retain at the time the store file flushes or opened for cache? It’s fine as is without caring the cache , just want to learn more here There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, in this situation, the metrics will retain at the time the store file flushes or opened for cache, although the cached data of store file is accessed at a new time. I'm not sure whether it is critical issue. | ||
|  | ||
| 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 Map<String, Long> getMixedRowReadsCount() { | |
| return mixedReadsOnStore; | ||
| } | ||
|  | ||
| @Override | ||
| public Map<Integer, Long> getStoreFilesAccessedDaysAndSize() { | ||
| return storeFilesAccessedDaysAndSize; | ||
| } | ||
|  | ||
| public class HRegionMetricsWrapperRunnable implements Runnable { | ||
|  | ||
| @Override | ||
|  | @@ -259,8 +283,12 @@ public void run() { | |
| 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 void run() { | |
| 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; | ||
|  | ||
| Original file line number | Diff line number | Diff line change | 
|---|---|---|
|  | @@ -17,18 +17,25 @@ | |
| */ | ||
| 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; | ||
| 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 MetricsTableWrapperAggregateImpl(final HRegionServer regionServer) { | |
| 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 void run() { | |
| 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 void run() { | |
| // 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) { | ||
| There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So here if the accessTime is very old, we will increase the sumSize for all the thresholds? Is this by design? Seems a bit strange... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @Apache9 sir. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK, so the metric here is 'the number if store files which are not accessed by at least xx days'? Then I think we;d better have a better name for it? Looking at 'storeFilesAccessedDays', I can not get the meaning unless you explain it... There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, sir. | ||
| 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 Map<String, Long> getMixedRowReadsCount(String table) { | |
| } | ||
| } | ||
|  | ||
| @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 @@ private static class MetricsTableValues { | |
| long cpRequestCount; | ||
| Map<String, Long> perStoreMemstoreOnlyReadCount = new HashMap<>(); | ||
| Map<String, Long> perStoreMixedReadCount = new HashMap<>(); | ||
| Map<Integer, Long> storeFilesAccessedDaysAndSize = new TreeMap<>(); | ||
| } | ||
|  | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.