You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2022/10/24 11:05:24 UTC

[hbase] branch branch-2.4 updated: HBASE-27440 fix table HistogramMetrics leak in table metrics map (#4838)

This is an automated email from the ASF dual-hosted git repository.

zhangduo pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new 098e59863bf HBASE-27440 fix table HistogramMetrics leak in table metrics map (#4838)
098e59863bf is described below

commit 098e59863bf8ce4d0c81488805532c4fc72f0332
Author: tianliuliu <64...@qq.com>
AuthorDate: Mon Oct 24 18:42:58 2022 +0800

    HBASE-27440 fix table HistogramMetrics leak in table metrics map (#4838)
    
    Signed-off-by: Duo Zhang <zh...@apache.org>
    (cherry picked from commit d187f6908e9be5723ff5bfb0e97cadca4092455a)
---
 .../metrics2/lib/DynamicMetricsRegistry.java       | 10 ++-
 .../regionserver/TestMetricsTableMetricsMap.java   | 81 ++++++++++++++++++++++
 2 files changed, 90 insertions(+), 1 deletion(-)

diff --git a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
index f1b1196cea4..9316fb0c82c 100644
--- a/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
+++ b/hbase-hadoop2-compat/src/main/java/org/apache/hadoop/metrics2/lib/DynamicMetricsRegistry.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.metrics2.lib;
 
+import com.google.errorprone.annotations.RestrictedApi;
 import java.util.Collection;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.hadoop.hbase.metrics.Interns;
@@ -391,8 +392,9 @@ public class DynamicMetricsRegistry {
 
   public void removeHistogramMetrics(String baseName) {
     for (String suffix : histogramSuffixes) {
-      removeMetric(baseName + suffix);
+      helper.removeObjectName(baseName + suffix);
     }
+    metricsMap.remove(baseName);
   }
 
   /**
@@ -535,4 +537,10 @@ public class DynamicMetricsRegistry {
     }
     metricsMap.clear();
   }
+
+  @RestrictedApi(explanation = "Should only be called in TestMetricsTableMetricsMap", link = "",
+      allowedOnPath = ".*/(DynamicMetricsRegistry|TestMetricsTableMetricsMap).java")
+  public ConcurrentMap<String, MutableMetric> getMetricsMap() {
+    return metricsMap;
+  }
 }
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableMetricsMap.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableMetricsMap.java
new file mode 100644
index 00000000000..2f6dc3db0b8
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMetricsTableMetricsMap.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.RegionServerTests;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({ RegionServerTests.class, SmallTests.class })
+public class TestMetricsTableMetricsMap {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+    HBaseClassTestRule.forClass(TestMetricsTableMetricsMap.class);
+
+  private String tableName = "testTableMetricsMap";
+
+  private MetricsTableWrapperStub tableWrapper;
+  private MetricsTable mt;
+  private MetricsRegionServerWrapper rsWrapper;
+  private MetricsRegionServer rsm;
+  private MetricsTableAggregateSourceImpl agg;
+
+  @Before
+  public void setUp() {
+    Configuration conf = new Configuration();
+
+    tableWrapper = new MetricsTableWrapperStub(tableName);
+    mt = new MetricsTable(tableWrapper);
+    rsWrapper = new MetricsRegionServerWrapperStub();
+
+    rsm = new MetricsRegionServer(rsWrapper, conf, mt);
+    MetricsTableAggregateSource tableSourceAgg = mt.getTableSourceAgg();
+    if (tableSourceAgg instanceof MetricsTableAggregateSourceImpl) {
+      agg = (MetricsTableAggregateSourceImpl) tableSourceAgg;
+    } else {
+      throw new RuntimeException(
+        "tableSourceAgg should be the instance of MetricsTableAggregateSourceImpl");
+    }
+  }
+
+  @Test
+  public void testMetricsMap() throws InterruptedException {
+    // do major compaction
+    rsm.updateCompaction(tableName, true, 100, 200, 300, 400, 500);
+
+    int metricsMapSize = agg.getMetricsRegistry().getMetricsMap().size();
+    assertTrue("table metrics added then metricsMapSize should larger than 0", metricsMapSize > 0);
+
+    // just for metrics update
+    Thread.sleep(1000);
+    // delete table all metrics
+    agg.deleteTableSource(tableName);
+
+    metricsMapSize = agg.getMetricsRegistry().getMetricsMap().size();
+    assertEquals("table metrics all deleted then metricsSize should be 0", 0, metricsMapSize);
+  }
+}