|
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.balancer;
|
|
|
import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CLIENT_CONNECT_MAX_RETRIES_ON_SASL_KEY;
|
|
|
import static org.apache.hadoop.fs.StorageType.DEFAULT;
|
|
|
import static org.apache.hadoop.fs.StorageType.RAM_DISK;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BALANCER_MAX_SIZE_TO_MOVE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_KEYSTORE_RESOURCE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
|
|
@@ -46,6 +47,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBER
|
|
|
import static org.apache.hadoop.test.PlatformAssumptions.assumeNotWindows;
|
|
|
|
|
|
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
|
|
|
+import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
import org.junit.AfterClass;
|
|
|
import static org.junit.Assert.assertEquals;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
@@ -2209,6 +2211,106 @@ public class TestBalancer {
|
|
|
getBlocksMaxQps, getBlockCallsPerSecond <= getBlocksMaxQps);
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testBalancerWithSortTopNodes() throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ initConf(conf);
|
|
|
+ conf.setInt(DFS_HEARTBEAT_INTERVAL_KEY, 30000);
|
|
|
+
|
|
|
+ final long capacity = 1000L;
|
|
|
+ final int diffBetweenNodes = 50;
|
|
|
+
|
|
|
+ // Set up the datanodes with two groups:
|
|
|
+ // 5 over-utilized nodes with 80%, 85%, 90%, 95%, 100% usage
|
|
|
+ // 2 under-utilizaed nodes with 0%, 5% usage
|
|
|
+ // With sortTopNodes option, 100% and 95% used ones will be chosen.
|
|
|
+ final int numOfOverUtilizedDn = 5;
|
|
|
+ final int numOfUnderUtilizedDn = 2;
|
|
|
+ final int totalNumOfDn = numOfOverUtilizedDn + numOfUnderUtilizedDn;
|
|
|
+ final long[] capacityArray = new long[totalNumOfDn];
|
|
|
+ Arrays.fill(capacityArray, capacity);
|
|
|
+
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(totalNumOfDn)
|
|
|
+ .simulatedCapacities(capacityArray)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ cluster.setDataNodesDead();
|
|
|
+
|
|
|
+ List<DataNode> dataNodes = cluster.getDataNodes();
|
|
|
+
|
|
|
+ // Create top used nodes
|
|
|
+ for (int i = 0; i < numOfOverUtilizedDn; i++) {
|
|
|
+ // Bring one node alive
|
|
|
+ DataNodeTestUtils.triggerHeartbeat(dataNodes.get(i));
|
|
|
+ DataNodeTestUtils.triggerBlockReport(dataNodes.get(i));
|
|
|
+ // Create nodes with: 80%, 85%, 90%, 95%, 100%.
|
|
|
+ int capacityForThisDatanode = (int)capacity
|
|
|
+ - diffBetweenNodes * (numOfOverUtilizedDn - i - 1);
|
|
|
+ createFile(cluster, new Path("test_big" + i),
|
|
|
+ capacityForThisDatanode, (short) 1, 0);
|
|
|
+ cluster.setDataNodesDead();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Create under utilized nodes
|
|
|
+ for (int i = numOfUnderUtilizedDn - 1; i >= 0; i--) {
|
|
|
+ int index = i + numOfOverUtilizedDn;
|
|
|
+ // Bring one node alive
|
|
|
+ DataNodeTestUtils.triggerHeartbeat(dataNodes.get(index));
|
|
|
+ DataNodeTestUtils.triggerBlockReport(dataNodes.get(index));
|
|
|
+ // Create nodes with: 5%, 0%
|
|
|
+ int capacityForThisDatanode = diffBetweenNodes * i;
|
|
|
+ createFile(cluster,
|
|
|
+ new Path("test_small" + i),
|
|
|
+ capacityForThisDatanode, (short) 1, 0);
|
|
|
+ cluster.setDataNodesDead();
|
|
|
+ }
|
|
|
+
|
|
|
+ // Bring all nodes alive
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+ cluster.triggerBlockReports();
|
|
|
+ cluster.waitFirstBRCompleted(0, 6000);
|
|
|
+
|
|
|
+ final BalancerParameters p = Balancer.Cli.parse(new String[] {
|
|
|
+ "-policy", BalancingPolicy.Node.INSTANCE.getName(),
|
|
|
+ "-threshold", "1",
|
|
|
+ "-sortTopNodes"
|
|
|
+ });
|
|
|
+
|
|
|
+ client = NameNodeProxies.createProxy(conf,
|
|
|
+ cluster.getFileSystem(0).getUri(),
|
|
|
+ ClientProtocol.class).getProxy();
|
|
|
+
|
|
|
+ // Set max-size-to-move to small number
|
|
|
+ // so only top two nodes will be chosen in one iteration.
|
|
|
+ conf.setLong(DFS_BALANCER_MAX_SIZE_TO_MOVE_KEY, 99L);
|
|
|
+
|
|
|
+ final Collection<URI> namenodes = DFSUtil.getInternalNsRpcUris(conf);
|
|
|
+
|
|
|
+ List<NameNodeConnector> connectors = NameNodeConnector
|
|
|
+ .newNameNodeConnectors(namenodes,
|
|
|
+ Balancer.class.getSimpleName(), Balancer.BALANCER_ID_PATH, conf,
|
|
|
+ BalancerParameters.DEFAULT.getMaxIdleIteration());
|
|
|
+ final Balancer b = new Balancer(connectors.get(0), p, conf);
|
|
|
+ Result balancerResult = b.runOneIteration();
|
|
|
+
|
|
|
+ cluster.triggerDeletionReports();
|
|
|
+ cluster.triggerBlockReports();
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+
|
|
|
+ DatanodeInfo[] datanodeReport = client
|
|
|
+ .getDatanodeReport(DatanodeReportType.ALL);
|
|
|
+
|
|
|
+ long maxUsage = 0;
|
|
|
+ for (int i = 0; i < totalNumOfDn; i++) {
|
|
|
+ maxUsage = Math.max(maxUsage, datanodeReport[i].getDfsUsed());
|
|
|
+ }
|
|
|
+
|
|
|
+ assertEquals(200, balancerResult.bytesAlreadyMoved);
|
|
|
+ // 100% and 95% used nodes will be balanced, so top used will be 900
|
|
|
+ assertEquals(900, maxUsage);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* @param args
|
|
|
*/
|