Skip to content

Commit 00cddf5

Browse files
HDFS-17646. Add Option to limit Balancer overUtilized nodes num in each iteration. (#7120). Contributed by Zhaobo Huang.
Reviewed-by: Haiyang Hu <[email protected]> Signed-off-by: He Xiaoqiao <[email protected]>
1 parent 7a7b346 commit 00cddf5

File tree

4 files changed

+143
-3
lines changed

4 files changed

+143
-3
lines changed

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

Lines changed: 34 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -208,6 +208,8 @@ public class Balancer {
208208
+ "\n\t[-sortTopNodes]"
209209
+ "\tSort datanodes based on the utilization so "
210210
+ "that highly utilized datanodes get scheduled first."
211+
+ "\n\t[-limitOverUtilizedNum <specified maximum number of overUtilized datanodes>]"
212+
+ "\tLimit the maximum number of overUtilized datanodes."
211213
+ "\n\t[-hotBlockTimeInterval]\tprefer to move cold blocks.";
212214

213215
@VisibleForTesting
@@ -227,6 +229,7 @@ public class Balancer {
227229
private final long maxSizeToMove;
228230
private final long defaultBlockSize;
229231
private final boolean sortTopNodes;
232+
private final int limitOverUtilizedNum;
230233
private final BalancerMetrics metrics;
231234

232235
// all data node lists
@@ -352,6 +355,7 @@ static int getFailedTimesSinceLastSuccessfulBalance() {
352355
this.sourceNodes = p.getSourceNodes();
353356
this.runDuringUpgrade = p.getRunDuringUpgrade();
354357
this.sortTopNodes = p.getSortTopNodes();
358+
this.limitOverUtilizedNum = p.getLimitOverUtilizedNum();
355359

356360
this.maxSizeToMove = getLongBytes(conf,
357361
DFSConfigKeys.DFS_BALANCER_MAX_SIZE_TO_MOVE_KEY,
@@ -456,11 +460,18 @@ private long init(List<DatanodeStorageReport> reports) {
456460
sortOverUtilized(overUtilizedPercentage);
457461
}
458462

463+
// Limit the maximum number of overUtilized datanodes
464+
// If excludedOverUtilizedNum is greater than 0, The overUtilized nodes num is limited
465+
int excludedOverUtilizedNum = Math.max(overUtilized.size() - limitOverUtilizedNum, 0);
466+
if (excludedOverUtilizedNum > 0) {
467+
limitOverUtilizedNum();
468+
}
469+
459470
logUtilizationCollections();
460471
metrics.setNumOfOverUtilizedNodes(overUtilized.size());
461472
metrics.setNumOfUnderUtilizedNodes(underUtilized.size());
462473

463-
Preconditions.checkState(dispatcher.getStorageGroupMap().size()
474+
Preconditions.checkState(dispatcher.getStorageGroupMap().size() - excludedOverUtilizedNum
464475
== overUtilized.size() + underUtilized.size() + aboveAvgUtilized.size()
465476
+ belowAvgUtilized.size(),
466477
"Mismatched number of storage groups");
@@ -484,6 +495,20 @@ private void sortOverUtilized(Map<Source, Double> overUtilizedPercentage) {
484495
);
485496
}
486497

498+
private void limitOverUtilizedNum() {
499+
Preconditions.checkState(overUtilized instanceof LinkedList,
500+
"Collection overUtilized is not a LinkedList.");
501+
LinkedList<Source> list = (LinkedList<Source>) overUtilized;
502+
503+
LOG.info("Limiting over-utilized nodes num, if using the '-sortTopNodes' param," +
504+
" the overUtilized nodes of top will be retained");
505+
506+
int size = overUtilized.size();
507+
for (int i = 0; i < size - limitOverUtilizedNum; i++) {
508+
list.removeLast();
509+
}
510+
}
511+
487512
private static long computeMaxSize2Move(final long capacity, final long remaining,
488513
final double utilizationDiff, final long max) {
489514
final double diff = Math.abs(utilizationDiff);
@@ -1071,6 +1096,14 @@ static BalancerParameters parse(String[] args) {
10711096
b.setSortTopNodes(true);
10721097
LOG.info("Balancer will sort nodes by" +
10731098
" capacity usage percentage to prioritize top used nodes");
1099+
} else if ("-limitOverUtilizedNum".equalsIgnoreCase(args[i])) {
1100+
Preconditions.checkArgument(++i < args.length,
1101+
"limitOverUtilizedNum value is missing: args = " + Arrays.toString(args));
1102+
int limitNum = Integer.parseInt(args[i]);
1103+
Preconditions.checkArgument(limitNum >= 0,
1104+
"limitOverUtilizedNum must be non-negative");
1105+
LOG.info("Using a limitOverUtilizedNum of {}", limitNum);
1106+
b.setLimitOverUtilizedNum(limitNum);
10741107
} else {
10751108
throw new IllegalArgumentException("args = "
10761109
+ Arrays.toString(args));

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

Lines changed: 15 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,8 @@ final class BalancerParameters {
5050

5151
private final boolean sortTopNodes;
5252

53+
private final int limitOverUtilizedNum;
54+
5355
static final BalancerParameters DEFAULT = new BalancerParameters();
5456

5557
private BalancerParameters() {
@@ -67,6 +69,7 @@ private BalancerParameters(Builder builder) {
6769
this.runDuringUpgrade = builder.runDuringUpgrade;
6870
this.runAsService = builder.runAsService;
6971
this.sortTopNodes = builder.sortTopNodes;
72+
this.limitOverUtilizedNum = builder.limitOverUtilizedNum;
7073
this.hotBlockTimeInterval = builder.hotBlockTimeInterval;
7174
}
7275

@@ -110,6 +113,10 @@ boolean getSortTopNodes() {
110113
return this.sortTopNodes;
111114
}
112115

116+
int getLimitOverUtilizedNum() {
117+
return this.limitOverUtilizedNum;
118+
}
119+
113120
long getHotBlockTimeInterval() {
114121
return this.hotBlockTimeInterval;
115122
}
@@ -120,12 +127,12 @@ public String toString() {
120127
+ " max idle iteration = %s," + " #excluded nodes = %s,"
121128
+ " #included nodes = %s," + " #source nodes = %s,"
122129
+ " #blockpools = %s," + " run during upgrade = %s,"
123-
+ " sort top nodes = %s,"
130+
+ " sort top nodes = %s," + " limit overUtilized nodes num = %s,"
124131
+ " hot block time interval = %s]",
125132
Balancer.class.getSimpleName(), getClass().getSimpleName(), policy,
126133
threshold, maxIdleIteration, excludedNodes.size(),
127134
includedNodes.size(), sourceNodes.size(), blockpools.size(),
128-
runDuringUpgrade, sortTopNodes, hotBlockTimeInterval);
135+
runDuringUpgrade, sortTopNodes, limitOverUtilizedNum, hotBlockTimeInterval);
129136
}
130137

131138
static class Builder {
@@ -141,6 +148,7 @@ static class Builder {
141148
private boolean runDuringUpgrade = false;
142149
private boolean runAsService = false;
143150
private boolean sortTopNodes = false;
151+
private int limitOverUtilizedNum = Integer.MAX_VALUE;
144152
private long hotBlockTimeInterval = 0;
145153

146154
Builder() {
@@ -201,6 +209,11 @@ Builder setSortTopNodes(boolean shouldSortTopNodes) {
201209
return this;
202210
}
203211

212+
Builder setLimitOverUtilizedNum(int overUtilizedNum) {
213+
this.limitOverUtilizedNum = overUtilizedNum;
214+
return this;
215+
}
216+
204217
BalancerParameters build() {
205218
return new BalancerParameters(this);
206219
}

hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -293,6 +293,7 @@ Usage:
293293
[-runDuringUpgrade]
294294
[-asService]
295295
[-sortTopNodes]
296+
[-limitOverUtilizedNum <specified maximum number of overUtilized datanodes>]
296297
[-hotBlockTimeInterval <specified time interval>]
297298

298299
| COMMAND\_OPTION | Description |
@@ -307,6 +308,7 @@ Usage:
307308
| `-runDuringUpgrade` | Whether to run the balancer during an ongoing HDFS upgrade. This is usually not desired since it will not affect used space on over-utilized machines. |
308309
| `-asService` | Run Balancer as a long running service. |
309310
| `-sortTopNodes` | Sort datanodes based on the utilization so that highly utilized datanodes get scheduled first. |
311+
| `-limitOverUtilizedNum` | Limit the maximum number of overUtilized datanodes. |
310312
| `-hotBlockTimeInterval` | Prefer moving cold blocks i.e blocks associated with files accessed or modified before the specified time interval. |
311313
| `-h`\|`--help` | Display the tool usage and help information and exit. |
312314

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

Lines changed: 92 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -672,6 +672,98 @@ public void testBalancerWithSortTopNodes() throws Exception {
672672
assertEquals(900, maxUsage);
673673
}
674674

675+
@Test(timeout = 60000)
676+
public void testBalancerWithLimitOverUtilizedNum() throws Exception {
677+
final Configuration conf = new HdfsConfiguration();
678+
// Init the config (block size to 100)
679+
initConf(conf);
680+
conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 30000);
681+
682+
final long totalCapacity = 1000L;
683+
final int diffBetweenNodes = 50;
684+
685+
// Set up the nodes with two groups:
686+
// 5 over-utilized nodes with 80%, 85%, 90%, 95%, 100% usage
687+
// 2 under-utilized nodes with 0%, 5% usage
688+
// With sortTopNodes and limitOverUtilizedNum option, 100% used ones will be chosen
689+
final int numOfOverUtilizedDn = 5;
690+
final int numOfUnderUtilizedDn = 2;
691+
final int totalNumOfDn = numOfOverUtilizedDn + numOfUnderUtilizedDn;
692+
final long[] capacityArray = new long[totalNumOfDn];
693+
Arrays.fill(capacityArray, totalCapacity);
694+
695+
try (MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
696+
.numDataNodes(totalNumOfDn)
697+
.simulatedCapacities(capacityArray)
698+
.build()) {
699+
cluster.setDataNodesDead();
700+
List<DataNode> dataNodes = cluster.getDataNodes();
701+
// Create top used nodes
702+
for (int i = 0; i < numOfOverUtilizedDn; i++) {
703+
// Bring one node alive
704+
DataNodeTestUtils.triggerHeartbeat(dataNodes.get(i));
705+
DataNodeTestUtils.triggerBlockReport(dataNodes.get(i));
706+
// Create nodes with: 80%, 85%, 90%, 95%, 100%
707+
int nodeCapacity = (int) totalCapacity - diffBetweenNodes * (numOfOverUtilizedDn - i - 1);
708+
TestBalancer.createFile(cluster, new Path("test_big" + i), nodeCapacity, (short) 1, 0);
709+
cluster.setDataNodesDead();
710+
}
711+
712+
// Create under utilized nodes
713+
for (int i = numOfUnderUtilizedDn - 1; i >= 0; i--) {
714+
int index = i + numOfOverUtilizedDn;
715+
// Bring one node alive
716+
DataNodeTestUtils.triggerHeartbeat(dataNodes.get(index));
717+
DataNodeTestUtils.triggerBlockReport(dataNodes.get(index));
718+
// Create nodes with: 5%, 0%
719+
int nodeCapacity = diffBetweenNodes * i;
720+
TestBalancer.createFile(cluster, new Path("test_small" + i), nodeCapacity, (short) 1, 0);
721+
cluster.setDataNodesDead();
722+
}
723+
724+
// Bring all nodes alive
725+
cluster.triggerHeartbeats();
726+
cluster.triggerBlockReports();
727+
cluster.waitFirstBRCompleted(0, 6000);
728+
729+
final BalancerParameters balancerParameters = Balancer.Cli.parse(new String[] {
730+
"-policy", BalancingPolicy.Node.INSTANCE.getName(),
731+
"-threshold", "1",
732+
"-sortTopNodes",
733+
"-limitOverUtilizedNum", "1"
734+
});
735+
736+
client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0)
737+
.getUri(), ClientProtocol.class)
738+
.getProxy();
739+
740+
// Set max-size-to-move to small number
741+
// so only top two nodes will be chosen in one iteration
742+
conf.setLong(DFS_BALANCER_MAX_SIZE_TO_MOVE_KEY, 99L);
743+
final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
744+
List<NameNodeConnector> connectors =
745+
NameNodeConnector.newNameNodeConnectors(namenodes, Balancer.class.getSimpleName(),
746+
Balancer.BALANCER_ID_PATH, conf, BalancerParameters.DEFAULT.getMaxIdleIteration());
747+
final Balancer balancer = new Balancer(connectors.get(0), balancerParameters, conf);
748+
Balancer.Result balancerResult = balancer.runOneIteration();
749+
750+
cluster.triggerDeletionReports();
751+
cluster.triggerBlockReports();
752+
cluster.triggerHeartbeats();
753+
754+
DatanodeInfo[] datanodeReport =
755+
client.getDatanodeReport(HdfsConstants.DatanodeReportType.ALL);
756+
long maxUsage = 0;
757+
for (int i = 0; i < totalNumOfDn; i++) {
758+
maxUsage = Math.max(maxUsage, datanodeReport[i].getDfsUsed());
759+
}
760+
// The maxUsage value is 950, only 100% of the nodes will be balanced
761+
assertEquals(950, maxUsage);
762+
assertTrue("BalancerResult is not as expected. " + balancerResult,
763+
(balancerResult.getBytesAlreadyMoved() == 100 && balancerResult.getBlocksMoved() == 1));
764+
}
765+
}
766+
675767
@Test(timeout = 100000)
676768
public void testMaxIterationTime() throws Exception {
677769
final Configuration conf = new HdfsConfiguration();

0 commit comments

Comments
 (0)