Skip to content

Commit b143181

Browse files
authored
HDFS-10648. Expose Balancer metrics through Metrics2 (#3427)
1 parent 8f4456d commit b143181

File tree

3 files changed

+119
-1
lines changed

3 files changed

+119
-1
lines changed

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

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,6 +40,8 @@
4040
import java.util.concurrent.atomic.AtomicInteger;
4141

4242
import org.apache.commons.lang3.builder.ToStringBuilder;
43+
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
44+
import org.apache.hadoop.metrics2.source.JvmMetrics;
4345
import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
4446
import org.apache.hadoop.hdfs.DFSUtilClient;
4547
import org.slf4j.Logger;
@@ -226,6 +228,7 @@ public class Balancer {
226228
private final long maxSizeToMove;
227229
private final long defaultBlockSize;
228230
private final boolean sortTopNodes;
231+
private final BalancerMetrics metrics;
229232

230233
// all data node lists
231234
private final Collection<Source> overUtilized = new LinkedList<Source>();
@@ -357,6 +360,7 @@ static int getFailedTimesSinceLastSuccessfulBalance() {
357360
this.defaultBlockSize = getLongBytes(conf,
358361
DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
359362
DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT);
363+
this.metrics = BalancerMetrics.create(this);
360364
}
361365

362366
private static long getCapacity(DatanodeStorageReport report, StorageType t) {
@@ -454,6 +458,8 @@ private long init(List<DatanodeStorageReport> reports) {
454458
}
455459

456460
logUtilizationCollections();
461+
metrics.setNumOfOverUtilizedNodes(overUtilized.size());
462+
metrics.setNumOfUnderUtilizedNodes(underUtilized.size());
457463

458464
Preconditions.checkState(dispatcher.getStorageGroupMap().size()
459465
== overUtilized.size() + underUtilized.size() + aboveAvgUtilized.size()
@@ -636,7 +642,11 @@ void resetData(Configuration conf) {
636642
this.belowAvgUtilized.clear();
637643
this.underUtilized.clear();
638644
this.policy.reset();
639-
dispatcher.reset(conf);;
645+
dispatcher.reset(conf);
646+
}
647+
648+
NameNodeConnector getNnc() {
649+
return nnc;
640650
}
641651

642652
static class Result {
@@ -710,8 +720,10 @@ Result newResult(ExitStatus exitStatus) {
710720
/** Run an iteration for all datanodes. */
711721
Result runOneIteration() {
712722
try {
723+
metrics.setIterateRunning(true);
713724
final List<DatanodeStorageReport> reports = dispatcher.init();
714725
final long bytesLeftToMove = init(reports);
726+
metrics.setBytesLeftToMove(bytesLeftToMove);
715727
if (bytesLeftToMove == 0) {
716728
return newResult(ExitStatus.SUCCESS, bytesLeftToMove, 0);
717729
} else {
@@ -766,6 +778,7 @@ Result runOneIteration() {
766778
System.out.println(e + ". Exiting ...");
767779
return newResult(ExitStatus.INTERRUPTED);
768780
} finally {
781+
metrics.setIterateRunning(false);
769782
dispatcher.shutdownNow();
770783
}
771784
}
@@ -848,6 +861,10 @@ static int run(Collection<URI> namenodes, final BalancerParameters p,
848861
static int run(Collection<URI> namenodes, Collection<String> nsIds,
849862
final BalancerParameters p, Configuration conf)
850863
throws IOException, InterruptedException {
864+
DefaultMetricsSystem.initialize("Balancer");
865+
JvmMetrics.create("Balancer",
866+
conf.get(DFSConfigKeys.DFS_METRICS_SESSION_ID_KEY),
867+
DefaultMetricsSystem.instance());
851868
if (!p.getRunAsService()) {
852869
return doBalance(namenodes, nsIds, p, conf);
853870
}
@@ -893,6 +910,8 @@ static int run(Collection<URI> namenodes, Collection<String> nsIds,
893910
time2Str(scheduleInterval));
894911
Thread.sleep(scheduleInterval);
895912
}
913+
DefaultMetricsSystem.shutdown();
914+
896915
// normal stop
897916
return 0;
898917
}
Lines changed: 80 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,80 @@
1+
/**
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
package org.apache.hadoop.hdfs.server.balancer;
19+
20+
import org.apache.hadoop.metrics2.annotation.Metric;
21+
import org.apache.hadoop.metrics2.annotation.Metrics;
22+
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
23+
import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
24+
import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
25+
26+
/**
27+
* Metrics for individual Balancer.
28+
*/
29+
@Metrics(about="Balancer metrics", context="dfs")
30+
final class BalancerMetrics {
31+
32+
private final Balancer balancer;
33+
34+
@Metric("If a balancer iterate is running")
35+
private MutableGaugeInt iterateRunning;
36+
37+
@Metric("Bytes left to move to make cluster balanced")
38+
private MutableGaugeLong bytesLeftToMove;
39+
40+
@Metric("Number of under utilized nodes")
41+
private MutableGaugeInt numOfUnderUtilizedNodes;
42+
43+
@Metric("Number of over utilized nodes")
44+
private MutableGaugeInt numOfOverUtilizedNodes;
45+
46+
private BalancerMetrics(Balancer b) {
47+
this.balancer = b;
48+
}
49+
50+
public static BalancerMetrics create(Balancer b) {
51+
BalancerMetrics m = new BalancerMetrics(b);
52+
return DefaultMetricsSystem.instance().register(
53+
m.getName(), null, m);
54+
}
55+
56+
String getName() {
57+
return "Balancer-" + balancer.getNnc().getBlockpoolID();
58+
}
59+
60+
@Metric("Bytes that already moved in current doBalance run.")
61+
public long getBytesMovedInCurrentRun() {
62+
return balancer.getNnc().getBytesMoved().get();
63+
}
64+
65+
void setIterateRunning(boolean iterateRunning) {
66+
this.iterateRunning.set(iterateRunning ? 1 : 0);
67+
}
68+
69+
void setBytesLeftToMove(long bytesLeftToMove) {
70+
this.bytesLeftToMove.set(bytesLeftToMove);
71+
}
72+
73+
void setNumOfUnderUtilizedNodes(int numOfUnderUtilizedNodes) {
74+
this.numOfUnderUtilizedNodes.set(numOfUnderUtilizedNodes);
75+
}
76+
77+
void setNumOfOverUtilizedNodes(int numOfOverUtilizedNodes) {
78+
this.numOfOverUtilizedNodes.set(numOfOverUtilizedNodes);
79+
}
80+
}

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerService.java

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,10 @@
2828
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
2929
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
3030
import org.apache.hadoop.hdfs.server.namenode.ha.HATestUtil;
31+
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
32+
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
3133
import org.apache.hadoop.test.GenericTestUtils;
34+
import org.apache.hadoop.test.MetricsAsserts;
3235
import org.apache.hadoop.util.Tool;
3336
import org.junit.Test;
3437

@@ -129,8 +132,24 @@ public void testBalancerServiceBalanceTwice() throws Exception {
129132
newBalancerService(conf, new String[] {"-asService"});
130133
balancerThread.start();
131134

135+
// Check metrics
136+
final String balancerMetricsName = "Balancer-"
137+
+ cluster.getNameNode(0).getNamesystem().getBlockPoolId();
138+
GenericTestUtils.waitFor(() -> {
139+
// Validate metrics after metrics system initiated.
140+
if (DefaultMetricsSystem.instance().getSource(balancerMetricsName) == null) {
141+
return false;
142+
}
143+
MetricsRecordBuilder rb = MetricsAsserts.getMetrics(balancerMetricsName);
144+
return rb != null && MetricsAsserts.getLongGauge("BytesLeftToMove", rb) > 0;
145+
}, 100, 2000);
146+
132147
TestBalancer.waitForBalancer(totalUsedSpace, totalCapacity, client,
133148
cluster, BalancerParameters.DEFAULT);
149+
150+
MetricsRecordBuilder rb = MetricsAsserts.getMetrics(balancerMetricsName);
151+
assertTrue(MetricsAsserts.getLongGauge("BytesMovedInCurrentRun", rb) > 0);
152+
134153
cluster.triggerHeartbeats();
135154
cluster.triggerBlockReports();
136155

0 commit comments

Comments
 (0)