You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2022/11/21 22:09:57 UTC

[GitHub] [hbase] taklwu commented on a diff in pull request #4891: HBASE-27483 Expose table and region storefiles accessed days and size to the metrics

taklwu commented on code in PR #4891:
URL: https://github.com/apache/hbase/pull/4891#discussion_r1028549700


##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java:
##########
@@ -59,15 +61,28 @@ 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;
 
+  public static long ONE_DAY_MS = 24 * 3600 * 1000;
+  public static final String STOREFILES_ACCESSED_DAYS_THRESHOLDS =
+    "hbase.region.storefiles.accessed.days.thresholds";
+  public static final int[] STOREFILES_ACCESSED_DAYS_THRESHOLDS_DEFAULT = { 7, 30, 90 };

Review Comment:
   can we add few line of javadoc or some line in README.md how to use this ?



##########
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java:
##########
@@ -52,18 +60,30 @@ 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 {
 
     @Override
     public void run() {
       Map<TableName, MetricsTableValues> localMetricsTableMap = new HashMap<>();
+

Review Comment:
   [nit] is this added by spotless? 
   ```suggestion
   ```



##########
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerMetrics.java:
##########
@@ -382,6 +385,63 @@ public void testAppend() throws Exception {
     assertCounter("appendNumOps", 73);
   }
 
+  @Test
+  public void testRegionStoreFilesAccessedDaysAndSize() throws Exception {
+    doNPuts(4000, true);
+    admin.flush(tableName);
+    try (RegionLocator locator = connection.getRegionLocator(tableName)) {
+      for (HRegionLocation location : locator.getAllRegionLocations()) {
+        RegionInfo hri = location.getRegion();
+        MetricsRegionAggregateSource agg =
+          rs.getRegion(hri.getRegionName()).getMetrics().getSource().getAggregateSource();
+        String prefix = "namespace_" + NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR + "_table_"
+          + tableName.getNameAsString() + "_region_" + hri.getEncodedName() + "_metric_";
+
+        TEST_UTIL.waitFor(60000, 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 does not exists in region metrics";
+          }
+        });
+
+        metricsHelper.assertGaugeGt(prefix + "storeFilesAccessed7DaysSize", 0, agg);
+        metricsHelper.assertGaugeGt(prefix + "storeFilesAccessed30DaysSize", 0, agg);
+        metricsHelper.assertGaugeGt(prefix + "storeFilesAccessed90DaysSize", 0, agg);
+      }
+    }
+  }
+
+  @Test
+  public void testTableStoreFilesAccessedDaysAndSize() throws Exception {
+    doNPuts(4000, true);
+    admin.flush(tableName);
+    MetricsTableAggregateSource agg = metricsRegionServer.getTableSourceAgg();
+    String prefix = "namespace_" + NamespaceDescriptor.DEFAULT_NAMESPACE_NAME_STR + "_table_"
+      + tableName.getNameAsString() + "_metric_";
+    TEST_UTIL.waitFor(60000, 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 does not exists in table metrics";
+      }
+    });
+
+    metricsHelper.assertGaugeGt(prefix + "storeFilesAccessed7DaysSize", 0, agg);
+    metricsHelper.assertGaugeGt(prefix + "storeFilesAccessed30DaysSize", 0, agg);
+    metricsHelper.assertGaugeGt(prefix + "storeFilesAccessed90DaysSize", 0, agg);

Review Comment:
   [nit] is there any way to verify non-zero metric for `storeFilesAccessed%sDaysSize` in test, e.g. if we can mock the access time ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@hbase.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org