Skip to content

Commit 9c98951

Browse files
authored
HDFS-17037. Consider nonDfsUsed when running balancer. (#5715). Contributed by Shuyan Zhang.
Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
1 parent 7bb09f1 commit 9c98951

File tree

4 files changed

+77
-37
lines changed

4 files changed

+77
-37
lines changed

hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/StorageReport.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ public long getNonDfsUsed() {
7575
}
7676

7777
public long getRemaining() {
78-
return remaining;
78+
return Math.max(remaining, 0L);
7979
}
8080

8181
public long getBlockPoolUsed() {

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

Lines changed: 13 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -104,21 +104,21 @@ void accumulateSpaces(DatanodeStorageReport r) {
104104
for(StorageReport s : r.getStorageReports()) {
105105
final StorageType t = s.getStorage().getStorageType();
106106
totalCapacities.add(t, s.getCapacity());
107-
totalUsedSpaces.add(t, s.getDfsUsed());
107+
totalUsedSpaces.add(t, s.getCapacity() - s.getRemaining());
108108
}
109109
}
110110

111111
@Override
112112
Double getUtilization(DatanodeStorageReport r, final StorageType t) {
113113
long capacity = 0L;
114-
long dfsUsed = 0L;
114+
long totalUsed = 0L;
115115
for(StorageReport s : r.getStorageReports()) {
116116
if (s.getStorage().getStorageType() == t) {
117117
capacity += s.getCapacity();
118-
dfsUsed += s.getDfsUsed();
118+
totalUsed += s.getCapacity() - s.getRemaining();
119119
}
120120
}
121-
return capacity == 0L? null: dfsUsed*100.0/capacity;
121+
return capacity == 0L ? null : totalUsed * 100.0 / capacity;
122122
}
123123
}
124124

@@ -138,7 +138,13 @@ String getName() {
138138
void accumulateSpaces(DatanodeStorageReport r) {
139139
for(StorageReport s : r.getStorageReports()) {
140140
final StorageType t = s.getStorage().getStorageType();
141-
totalCapacities.add(t, s.getCapacity());
141+
// Use s.getRemaining() + s.getBlockPoolUsed() instead of
142+
// s.getCapacity() here to avoid moving blocks towards nodes with
143+
// little actual available space.
144+
// The util is computed as blockPoolUsed/(remaining+blockPoolUsed),
145+
// which means nodes with more remaining space and less blockPoolUsed
146+
// will serve as the recipient during the balancing process.
147+
totalCapacities.add(t, s.getRemaining() + s.getBlockPoolUsed());
142148
totalUsedSpaces.add(t, s.getBlockPoolUsed());
143149
}
144150
}
@@ -149,11 +155,11 @@ Double getUtilization(DatanodeStorageReport r, final StorageType t) {
149155
long blockPoolUsed = 0L;
150156
for(StorageReport s : r.getStorageReports()) {
151157
if (s.getStorage().getStorageType() == t) {
152-
capacity += s.getCapacity();
158+
capacity += s.getRemaining() + s.getBlockPoolUsed();
153159
blockPoolUsed += s.getBlockPoolUsed();
154160
}
155161
}
156-
return capacity == 0L? null: blockPoolUsed*100.0/capacity;
162+
return capacity == 0L ? null : blockPoolUsed * 100.0 / capacity;
157163
}
158164
}
159165
}

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

Lines changed: 48 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,8 @@
4141
import java.lang.reflect.Field;
4242
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
4343
import org.junit.AfterClass;
44+
45+
import static org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset.CONFIG_PROPERTY_NONDFSUSED;
4446
import static org.junit.Assert.assertEquals;
4547
import static org.junit.Assert.assertFalse;
4648
import static org.junit.Assert.assertTrue;
@@ -502,8 +504,9 @@ static void waitForBalancer(long totalUsedSpace, long totalCapacity,
502504
balanced = true;
503505
int actualExcludedNodeCount = 0;
504506
for (DatanodeInfo datanode : datanodeReport) {
505-
double nodeUtilization = ((double)datanode.getDfsUsed())
506-
/ datanode.getCapacity();
507+
double nodeUtilization =
508+
((double) datanode.getDfsUsed() + datanode.getNonDfsUsed()) /
509+
datanode.getCapacity();
507510
if (Dispatcher.Util.isExcluded(p.getExcludedNodes(), datanode)) {
508511
if (checkExcludeNodesUtilization) {
509512
assertTrue(nodeUtilization == 0);
@@ -641,7 +644,7 @@ private void doTest(Configuration conf, long[] capacities, String[] racks,
641644
private void doTest(Configuration conf, long[] capacities, String[] racks,
642645
long newCapacity, String newRack, NewNodeInfo nodes,
643646
boolean useTool, boolean useFile) throws Exception {
644-
doTest(conf, capacities, racks, newCapacity, newRack, nodes,
647+
doTest(conf, capacities, racks, newCapacity, 0L, newRack, nodes,
645648
useTool, useFile, false, 0.3);
646649
}
647650

@@ -666,8 +669,8 @@ private void doTest(Configuration conf, long[] capacities, String[] racks,
666669
* @throws Exception
667670
*/
668671
private void doTest(Configuration conf, long[] capacities,
669-
String[] racks, long newCapacity, String newRack, NewNodeInfo nodes,
670-
boolean useTool, boolean useFile,
672+
String[] racks, long newCapacity, long newNonDfsUsed, String newRack,
673+
NewNodeInfo nodes, boolean useTool, boolean useFile,
671674
boolean useNamesystemSpy, double clusterUtilization) throws Exception {
672675
LOG.info("capacities = " + long2String(capacities));
673676
LOG.info("racks = " + Arrays.asList(racks));
@@ -701,10 +704,11 @@ private void doTest(Configuration conf, long[] capacities,
701704
long totalCapacity = sum(capacities);
702705

703706
// fill up the cluster to be `clusterUtilization` full
704-
long totalUsedSpace = (long) (totalCapacity * clusterUtilization);
705-
createFile(cluster, filePath, totalUsedSpace / numOfDatanodes,
707+
long totalDfsUsedSpace = (long) (totalCapacity * clusterUtilization);
708+
createFile(cluster, filePath, totalDfsUsedSpace / numOfDatanodes,
706709
(short) numOfDatanodes, 0);
707710

711+
conf.setLong(CONFIG_PROPERTY_NONDFSUSED, newNonDfsUsed);
708712
if (nodes == null) { // there is no specification of new nodes.
709713
// start up an empty node with the same capacity and on the same rack
710714
cluster.startDataNodes(conf, 1, true, null,
@@ -774,9 +778,11 @@ private void doTest(Configuration conf, long[] capacities,
774778

775779
// run balancer and validate results
776780
if (useTool) {
777-
runBalancerCli(conf, totalUsedSpace, totalCapacity, p, useFile, expectedExcludedNodes);
781+
runBalancerCli(conf, totalDfsUsedSpace, newNonDfsUsed,
782+
totalCapacity, p, useFile, expectedExcludedNodes);
778783
} else {
779-
runBalancer(conf, totalUsedSpace, totalCapacity, p, expectedExcludedNodes);
784+
runBalancer(conf, totalDfsUsedSpace, newNonDfsUsed,
785+
totalCapacity, p, expectedExcludedNodes, true);
780786
}
781787
} finally {
782788
if(cluster != null) {
@@ -791,16 +797,18 @@ private void runBalancer(Configuration conf, long totalUsedSpace,
791797
BalancerParameters.DEFAULT, 0);
792798
}
793799

794-
private void runBalancer(Configuration conf, long totalUsedSpace,
800+
private void runBalancer(Configuration conf, long totalDfsUsedSpace,
795801
long totalCapacity, BalancerParameters p, int excludedNodes)
796802
throws Exception {
797-
runBalancer(conf, totalUsedSpace, totalCapacity, p, excludedNodes, true);
803+
runBalancer(conf, totalDfsUsedSpace, 0, totalCapacity, p, excludedNodes,
804+
true);
798805
}
799806

800-
private void runBalancer(Configuration conf, long totalUsedSpace,
801-
long totalCapacity, BalancerParameters p, int excludedNodes,
802-
boolean checkExcludeNodesUtilization) throws Exception {
803-
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
807+
private void runBalancer(Configuration conf, long totalDfsUsedSpace,
808+
long totalNonDfsUsedSpace, long totalCapacity, BalancerParameters p,
809+
int excludedNodes, boolean checkExcludeNodesUtilization)
810+
throws Exception {
811+
waitForHeartBeat(totalDfsUsedSpace, totalCapacity, client, cluster);
804812

805813
int retry = 5;
806814
while (retry > 0) {
@@ -816,9 +824,10 @@ private void runBalancer(Configuration conf, long totalUsedSpace,
816824
} else {
817825
assertEquals(ExitStatus.SUCCESS.getExitCode(), run);
818826
}
819-
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
827+
waitForHeartBeat(totalDfsUsedSpace, totalCapacity, client, cluster);
820828
LOG.info(" .");
821829
try {
830+
long totalUsedSpace = totalDfsUsedSpace + totalNonDfsUsedSpace;
822831
waitForBalancer(totalUsedSpace, totalCapacity, client, cluster, p,
823832
excludedNodes, checkExcludeNodesUtilization);
824833
} catch (TimeoutException e) {
@@ -892,10 +901,10 @@ private static int runBalancer(Collection<URI> namenodes,
892901
return ExitStatus.SUCCESS.getExitCode();
893902
}
894903

895-
private void runBalancerCli(Configuration conf, long totalUsedSpace,
896-
long totalCapacity, BalancerParameters p, boolean useFile,
897-
int expectedExcludedNodes) throws Exception {
898-
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
904+
private void runBalancerCli(Configuration conf, long totalDfsUsedSpace,
905+
long totalNonDfsUsedSpace, long totalCapacity, BalancerParameters p,
906+
boolean useFile, int expectedExcludedNodes) throws Exception {
907+
waitForHeartBeat(totalDfsUsedSpace, totalCapacity, client, cluster);
899908
List <String> args = new ArrayList<String>();
900909
args.add("-policy");
901910
args.add("datanode");
@@ -939,8 +948,9 @@ private void runBalancerCli(Configuration conf, long totalUsedSpace,
939948
final int r = tool.run(args.toArray(new String[0])); // start rebalancing
940949

941950
assertEquals("Tools should exit 0 on success", 0, r);
942-
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
951+
waitForHeartBeat(totalDfsUsedSpace, totalCapacity, client, cluster);
943952
LOG.info("Rebalancing with default ctor.");
953+
long totalUsedSpace = totalDfsUsedSpace + totalNonDfsUsedSpace;
944954
waitForBalancer(totalUsedSpace, totalCapacity, client, cluster, p, expectedExcludedNodes);
945955

946956
if (excludeHostsFile != null && excludeHostsFile.exists()) {
@@ -1112,6 +1122,16 @@ void testBalancer2Internal(Configuration conf) throws Exception {
11121122
new String[]{RACK0, RACK1}, CAPACITY, RACK2);
11131123
}
11141124

1125+
/** Test a cluster with even distribution,
1126+
* then a new node with nonDfsUsed is added to the cluster. */
1127+
@Test(timeout=100000)
1128+
public void testBalancer3() throws Exception {
1129+
Configuration conf = new HdfsConfiguration();
1130+
initConf(conf);
1131+
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1},
1132+
CAPACITY, 1000L, RACK2, null, false, false, false, 0.3);
1133+
}
1134+
11151135
private void testBalancerDefaultConstructor(Configuration conf,
11161136
long[] capacities, String[] racks, long newCapacity, String newRack)
11171137
throws Exception {
@@ -1504,10 +1524,11 @@ public void testBalancerDuringUpgrade() throws Exception {
15041524
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
15051525

15061526
final int BLOCK_SIZE = 1024*1024;
1527+
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
15071528
cluster = new MiniDFSCluster
15081529
.Builder(conf)
15091530
.numDataNodes(1)
1510-
.storageCapacities(new long[] { BLOCK_SIZE * 10 })
1531+
.simulatedCapacities(new long[]{BLOCK_SIZE * 10})
15111532
.storageTypes(new StorageType[] { DEFAULT })
15121533
.storagesPerDatanode(1)
15131534
.build();
@@ -1517,11 +1538,12 @@ public void testBalancerDuringUpgrade() throws Exception {
15171538
final Path path1 = new Path("/" + METHOD_NAME + ".01.dat");
15181539

15191540
DistributedFileSystem fs = cluster.getFileSystem();
1520-
DFSTestUtil.createFile(fs, path1, BLOCK_SIZE, BLOCK_SIZE * 2, BLOCK_SIZE,
1541+
DFSTestUtil.createFile(fs, path1, BLOCK_SIZE, BLOCK_SIZE * 3, BLOCK_SIZE,
15211542
(short) 1, SEED);
15221543

15231544
// Add another DN with the same capacity, cluster is now unbalanced
1524-
cluster.startDataNodes(conf, 1, true, null, null);
1545+
cluster.startDataNodes(conf, 1, true, null, null, null,
1546+
new long[]{BLOCK_SIZE * 10}, false);
15251547
cluster.triggerHeartbeats();
15261548
Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
15271549

@@ -1773,7 +1795,7 @@ private void doTestBalancerWithExcludeListWithStripedFile(Configuration conf) th
17731795
pBuilder.setExcludedNodes(excludedList);
17741796

17751797
// start balancer and check the failed num of moving task
1776-
runBalancer(conf, totalUsedSpace, totalCapacity, pBuilder.build(),
1798+
runBalancer(conf, totalUsedSpace, 0, totalCapacity, pBuilder.build(),
17771799
excludedList.size(), false);
17781800

17791801
// check total blocks, max wait time 60s
@@ -1891,7 +1913,7 @@ void testBalancerRPCDelay(int getBlocksMaxQps) throws Exception {
18911913
capacities[i] = CAPACITY;
18921914
racks[i] = (i < numDNs/2 ? RACK0 : RACK1);
18931915
}
1894-
doTest(conf, capacities, racks, CAPACITY, RACK2,
1916+
doTest(conf, capacities, racks, CAPACITY, 0L, RACK2,
18951917
// Use only 1 node and set the starting capacity to 50% to allow the
18961918
// balancing to complete in only one iteration. This is necessary
18971919
// because the startGetBlocksTime and endGetBlocksTime measures across

hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -162,6 +162,11 @@ public static byte simulatedByte(Block b, long offsetInBlk) {
162162
private static final DatanodeStorage.State DEFAULT_STATE =
163163
DatanodeStorage.State.NORMAL;
164164

165+
public static final String CONFIG_PROPERTY_NONDFSUSED =
166+
"dfs.datanode.simulateddatastorage.nondfsused";
167+
168+
public static final long DEFAULT_NONDFSUSED = 0L;
169+
165170
static final byte[] nullCrcFileData;
166171

167172
private final DataNodeLockManager datasetLockManager;
@@ -467,11 +472,12 @@ static class SimulatedStorage {
467472
new ConcurrentHashMap<>();
468473

469474
private final long capacity; // in bytes
475+
private long nonDfsUsed;
470476
private final DatanodeStorage dnStorage;
471477
private final SimulatedVolume volume;
472478

473479
synchronized long getFree() {
474-
return capacity - getUsed();
480+
return capacity - getUsed() - getNonDfsUsed();
475481
}
476482

477483
long getCapacity() {
@@ -486,6 +492,10 @@ synchronized long getUsed() {
486492
return used;
487493
}
488494

495+
synchronized long getNonDfsUsed() {
496+
return nonDfsUsed;
497+
}
498+
489499
synchronized long getBlockPoolUsed(String bpid) throws IOException {
490500
return getBPStorage(bpid).getUsed();
491501
}
@@ -506,7 +516,7 @@ synchronized void free(String bpid, long amount) throws IOException {
506516
getBPStorage(bpid).free(amount);
507517
}
508518

509-
SimulatedStorage(long cap, DatanodeStorage.State state,
519+
SimulatedStorage(long cap, DatanodeStorage.State state, long nonDfsUsed,
510520
FileIoProvider fileIoProvider, Configuration conf) {
511521
capacity = cap;
512522
dnStorage = new DatanodeStorage(
@@ -515,6 +525,7 @@ synchronized void free(String bpid, long amount) throws IOException {
515525
DataNodeVolumeMetrics volumeMetrics =
516526
DataNodeVolumeMetrics.create(conf, dnStorage.getStorageID());
517527
this.volume = new SimulatedVolume(this, fileIoProvider, volumeMetrics);
528+
this.nonDfsUsed = nonDfsUsed;
518529
}
519530

520531
synchronized void addBlockPool(String bpid) {
@@ -548,7 +559,7 @@ DatanodeStorage getDnStorage() {
548559
synchronized StorageReport getStorageReport(String bpid) {
549560
return new StorageReport(dnStorage,
550561
false, getCapacity(), getUsed(), getFree(),
551-
map.get(bpid).getUsed(), 0L);
562+
map.get(bpid).getUsed(), getNonDfsUsed());
552563
}
553564

554565
SimulatedVolume getVolume() {
@@ -733,6 +744,7 @@ public SimulatedFSDataset(DataNode datanode, DataStorage storage, Configuration
733744
this.storages.add(new SimulatedStorage(
734745
conf.getLong(CONFIG_PROPERTY_CAPACITY, DEFAULT_CAPACITY),
735746
conf.getEnum(CONFIG_PROPERTY_STATE, DEFAULT_STATE),
747+
conf.getLong(CONFIG_PROPERTY_NONDFSUSED, DEFAULT_NONDFSUSED),
736748
fileIoProvider, conf));
737749
}
738750
}

0 commit comments

Comments
 (0)