소스 검색

HDFS-17648. Fix BalancerMetrics duplicate registration issue. (#7133). Contributed by Zhaobo Huang.

Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
Zhaobo Huang 6 달 전
부모
커밋
93a4dd02cf

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -686,7 +686,8 @@ public class Balancer {
     this.belowAvgUtilized.clear();
     this.underUtilized.clear();
     this.policy.reset();
-    dispatcher.reset(conf);
+    this.dispatcher.reset(conf);
+    DefaultMetricsSystem.removeSourceName(metrics.getName());
   }
 
   NameNodeConnector getNnc() {

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerLongRunningTasks.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -46,6 +47,8 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.metrics2.MetricsSystem;
+import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
@@ -764,6 +767,63 @@ public class TestBalancerLongRunningTasks {
     }
   }
 
+  @Test(timeout = 60000)
+  public void testBalancerMetricsDuplicate() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    // Init the config (block size to 100)
+    initConf(conf);
+    final long totalCapacity = 1000L;
+    final int numOfOverUtilizedDn = 1;
+    final int numOfUnderUtilizedDn = 2;
+    final int totalNumOfDn = numOfOverUtilizedDn + numOfUnderUtilizedDn;
+    final long[] capacityArray = new long[totalNumOfDn];
+    Arrays.fill(capacityArray, totalCapacity);
+    try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(totalNumOfDn)
+        .simulatedCapacities(capacityArray)
+        .build()) {
+      cluster.setDataNodesDead();
+      List<DataNode> dataNodes = cluster.getDataNodes();
+
+      DataNodeTestUtils.triggerHeartbeat(dataNodes.get(0));
+      DataNodeTestUtils.triggerBlockReport(dataNodes.get(0));
+      // Create nodes with: 100%
+      TestBalancer.createFile(cluster, new Path("test_big" + 0), 1000, (short) 1, 0);
+      cluster.setDataNodesDead();
+
+      // Two UnderUtilized in the cluster, execute at least twice: b.runOneIteration()
+      for (int i = 1; i <= numOfUnderUtilizedDn; i++) {
+        DataNodeTestUtils.triggerHeartbeat(dataNodes.get(i));
+        DataNodeTestUtils.triggerBlockReport(dataNodes.get(i));
+        // Create nodes with: 0%
+        TestBalancer.createFile(cluster, new Path("test_small" + i), 0, (short) 1, 0);
+        cluster.setDataNodesDead();
+      }
+
+      cluster.triggerDeletionReports();
+      cluster.triggerBlockReports();
+      cluster.triggerHeartbeats();
+
+      Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
+      Collection<String> nsIds = DFSUtilClient.getNameServiceIds(conf);
+      assertEquals(1, namenodes.size());
+
+      // Throw an error when we double-initialize BalancerMetrics
+      DefaultMetricsSystem.setMiniClusterMode(false);
+      MetricsSystem instance = DefaultMetricsSystem.instance();
+      // Avoid the impact of cluster metric, remove cluster JvmMetrics
+      instance.unregisterSource("JvmMetrics");
+
+      final BalancerParameters balancerParameters = Balancer.Cli.parse(new String[] {
+          "-policy", BalancingPolicy.Node.INSTANCE.getName(),
+          "-threshold", "10",
+      });
+      int r = Balancer.run(namenodes, nsIds, balancerParameters, conf);
+      assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
+      DefaultMetricsSystem.setMiniClusterMode(true);
+    }
+  }
+
   @Test(timeout = 100000)
   public void testMaxIterationTime() throws Exception {
     final Configuration conf = new HdfsConfiguration();