|
@@ -21,8 +21,13 @@ import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
+import java.util.Set;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -42,6 +47,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
|
|
|
+import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
import org.apache.log4j.Level;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
@@ -60,6 +67,7 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
private static final long CAPACITY = 500L;
|
|
|
private static final String RACK0 = "/rack0";
|
|
|
private static final String RACK1 = "/rack1";
|
|
|
+ private static final String RACK2 = "/rack2";
|
|
|
|
|
|
private static final String FILE_NAME = "/tmp.txt";
|
|
|
private static final Path FILE_PATH = new Path(FILE_NAME);
|
|
@@ -76,16 +84,20 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
final MiniDFSCluster cluster;
|
|
|
final ClientProtocol[] clients;
|
|
|
final short replication;
|
|
|
-
|
|
|
+ final Balancer.Parameters parameters;
|
|
|
+
|
|
|
Suite(MiniDFSCluster cluster, final int nNameNodes, final int nDataNodes,
|
|
|
- Configuration conf) throws IOException {
|
|
|
+ Balancer.Parameters parameters, Configuration conf) throws IOException {
|
|
|
this.conf = conf;
|
|
|
this.cluster = cluster;
|
|
|
clients = new ClientProtocol[nNameNodes];
|
|
|
for(int i = 0; i < nNameNodes; i++) {
|
|
|
clients[i] = cluster.getNameNode(i).getRpcServer();
|
|
|
}
|
|
|
- replication = (short)Math.max(1, nDataNodes - 1);
|
|
|
+ // hard coding replication factor to 1 so logical and raw HDFS size are
|
|
|
+ // equal
|
|
|
+ replication = 1;
|
|
|
+ this.parameters = parameters;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -104,11 +116,9 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
) throws IOException, InterruptedException, TimeoutException {
|
|
|
final ExtendedBlock[][] blocks = new ExtendedBlock[s.clients.length][];
|
|
|
for(int n = 0; n < s.clients.length; n++) {
|
|
|
- final long fileLen = size/s.replication;
|
|
|
- createFile(s, n, fileLen);
|
|
|
-
|
|
|
- final List<LocatedBlock> locatedBlocks = s.clients[n].getBlockLocations(
|
|
|
- FILE_NAME, 0, fileLen).getLocatedBlocks();
|
|
|
+ createFile(s, n, size);
|
|
|
+ final List<LocatedBlock> locatedBlocks =
|
|
|
+ s.clients[n].getBlockLocations(FILE_NAME, 0, size).getLocatedBlocks();
|
|
|
|
|
|
final int numOfBlocks = locatedBlocks.size();
|
|
|
blocks[n] = new ExtendedBlock[numOfBlocks];
|
|
@@ -151,9 +161,14 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
wait(s.clients, totalUsed, totalCapacity);
|
|
|
LOG.info("BALANCER 1");
|
|
|
|
|
|
+ // get storage reports for relevant blockpools so that we can compare
|
|
|
+ // blockpool usages after balancer has run
|
|
|
+ Map<Integer, DatanodeStorageReport[]> preBalancerPoolUsages =
|
|
|
+ getStorageReports(s);
|
|
|
+
|
|
|
// start rebalancing
|
|
|
final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(s.conf);
|
|
|
- final int r = Balancer.run(namenodes, Balancer.Parameters.DEFAULT, s.conf);
|
|
|
+ final int r = Balancer.run(namenodes, s.parameters, s.conf);
|
|
|
Assert.assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
|
|
|
|
|
|
LOG.info("BALANCER 2");
|
|
@@ -189,7 +204,7 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
balanced = true;
|
|
|
for(int d = 0; d < used.length; d++) {
|
|
|
final double p = used[d]*100.0/cap[d];
|
|
|
- balanced = p <= avg + Balancer.Parameters.DEFAULT.threshold;
|
|
|
+ balanced = p <= avg + s.parameters.threshold;
|
|
|
if (!balanced) {
|
|
|
if (i % 100 == 0) {
|
|
|
LOG.warn("datanodes " + d + " is not yet balanced: "
|
|
@@ -203,6 +218,89 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
}
|
|
|
}
|
|
|
LOG.info("BALANCER 6");
|
|
|
+ // cluster is balanced, verify that only selected blockpools were touched
|
|
|
+ Map<Integer, DatanodeStorageReport[]> postBalancerPoolUsages =
|
|
|
+ getStorageReports(s);
|
|
|
+ Assert.assertEquals(preBalancerPoolUsages.size(),
|
|
|
+ postBalancerPoolUsages.size());
|
|
|
+ for (Map.Entry<Integer, DatanodeStorageReport[]> entry
|
|
|
+ : preBalancerPoolUsages.entrySet()) {
|
|
|
+ compareTotalPoolUsage(entry.getValue(),
|
|
|
+ postBalancerPoolUsages.get(entry.getKey()));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Compare the total blockpool usage on each datanode to ensure that nothing
|
|
|
+ * was balanced.
|
|
|
+ *
|
|
|
+ * @param preReports storage reports from pre balancer run
|
|
|
+ * @param postReports storage reports from post balancer run
|
|
|
+ */
|
|
|
+ private static void compareTotalPoolUsage(DatanodeStorageReport[] preReports,
|
|
|
+ DatanodeStorageReport[] postReports) {
|
|
|
+ Assert.assertNotNull(preReports);
|
|
|
+ Assert.assertNotNull(postReports);
|
|
|
+ Assert.assertEquals(preReports.length, postReports.length);
|
|
|
+ for (DatanodeStorageReport preReport : preReports) {
|
|
|
+ String dnUuid = preReport.getDatanodeInfo().getDatanodeUuid();
|
|
|
+ for(DatanodeStorageReport postReport : postReports) {
|
|
|
+ if(postReport.getDatanodeInfo().getDatanodeUuid().equals(dnUuid)) {
|
|
|
+ Assert.assertEquals(getTotalPoolUsage(preReport),
|
|
|
+ getTotalPoolUsage(postReport));
|
|
|
+ LOG.info("Comparision of datanode pool usage pre/post balancer run. "
|
|
|
+ + "PrePoolUsage: " + getTotalPoolUsage(preReport)
|
|
|
+ + ", PostPoolUsage: " + getTotalPoolUsage(postReport));
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private static long getTotalPoolUsage(DatanodeStorageReport report) {
|
|
|
+ long usage = 0L;
|
|
|
+ for (StorageReport sr : report.getStorageReports()) {
|
|
|
+ usage += sr.getBlockPoolUsed();
|
|
|
+ }
|
|
|
+ return usage;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the storage reports for all blockpools that were not specified by the
|
|
|
+ * balancer blockpool parameters. If none were specified then the parameter
|
|
|
+ * was not set and do not return any reports.
|
|
|
+ *
|
|
|
+ * @param s suite for the test
|
|
|
+ * @return a map of storage reports where the key is the blockpool index
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private static Map<Integer,
|
|
|
+ DatanodeStorageReport[]> getStorageReports(Suite s) throws IOException {
|
|
|
+ Map<Integer, DatanodeStorageReport[]> reports =
|
|
|
+ new HashMap<Integer, DatanodeStorageReport[]>();
|
|
|
+ if (s.parameters.blockpools.size() == 0) {
|
|
|
+ // the blockpools parameter was not set, so we don't need to track any
|
|
|
+ // blockpools.
|
|
|
+ return Collections.emptyMap();
|
|
|
+ }
|
|
|
+ for (int i = 0; i < s.clients.length; i++) {
|
|
|
+ if (s.parameters.blockpools.contains(s.cluster.getNamesystem(i)
|
|
|
+ .getBlockPoolId())) {
|
|
|
+ // we want to ensure that blockpools not specified by the balancer
|
|
|
+ // parameters were left alone. Therefore, if the pool was specified,
|
|
|
+ // skip it. Note: this code assumes the clients in the suite are ordered
|
|
|
+ // the same way that they are indexed via cluster#getNamesystem(index).
|
|
|
+ continue;
|
|
|
+ } else {
|
|
|
+ LOG.info("Tracking usage of blockpool id: "
|
|
|
+ + s.cluster.getNamesystem(i).getBlockPoolId());
|
|
|
+ reports.put(i,
|
|
|
+ s.clients[i].getDatanodeStorageReport(DatanodeReportType.LIVE));
|
|
|
+ }
|
|
|
+ }
|
|
|
+ LOG.info("Tracking " + reports.size()
|
|
|
+ + " blockpool(s) for pre/post balancer usage.");
|
|
|
+ return reports;
|
|
|
}
|
|
|
|
|
|
private static void sleep(long ms) {
|
|
@@ -220,25 +318,31 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * First start a cluster and fill the cluster up to a certain size.
|
|
|
- * Then redistribute blocks according the required distribution.
|
|
|
- * Finally, balance the cluster.
|
|
|
- *
|
|
|
+ * First start a cluster and fill the cluster up to a certain size. Then
|
|
|
+ * redistribute blocks according the required distribution. Finally, balance
|
|
|
+ * the cluster.
|
|
|
+ *
|
|
|
* @param nNameNodes Number of NameNodes
|
|
|
- * @param distributionPerNN The distribution for each NameNode.
|
|
|
+ * @param nNameNodesToBalance Number of NameNodes to run the balancer on
|
|
|
+ * @param distributionPerNN The distribution for each NameNode.
|
|
|
* @param capacities Capacities of the datanodes
|
|
|
* @param racks Rack names
|
|
|
* @param conf Configuration
|
|
|
*/
|
|
|
private void unevenDistribution(final int nNameNodes,
|
|
|
- long distributionPerNN[], long capacities[], String[] racks,
|
|
|
- Configuration conf) throws Exception {
|
|
|
+ final int nNameNodesToBalance, long distributionPerNN[],
|
|
|
+ long capacities[], String[] racks, Configuration conf) throws Exception {
|
|
|
LOG.info("UNEVEN 0");
|
|
|
final int nDataNodes = distributionPerNN.length;
|
|
|
if (capacities.length != nDataNodes || racks.length != nDataNodes) {
|
|
|
throw new IllegalArgumentException("Array length is not the same");
|
|
|
}
|
|
|
|
|
|
+ if (nNameNodesToBalance > nNameNodes) {
|
|
|
+ throw new IllegalArgumentException("Number of namenodes to balance is "
|
|
|
+ + "greater than the number of namenodes.");
|
|
|
+ }
|
|
|
+
|
|
|
// calculate total space that need to be filled
|
|
|
final long usedSpacePerNN = TestBalancer.sum(distributionPerNN);
|
|
|
|
|
@@ -248,7 +352,7 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
LOG.info("UNEVEN 1");
|
|
|
final MiniDFSCluster cluster = new MiniDFSCluster
|
|
|
.Builder(new Configuration(conf))
|
|
|
- .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
|
|
|
.numDataNodes(nDataNodes)
|
|
|
.racks(racks)
|
|
|
.simulatedCapacities(capacities)
|
|
@@ -258,7 +362,7 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
cluster.waitActive();
|
|
|
DFSTestUtil.setFederatedConfiguration(cluster, conf);
|
|
|
LOG.info("UNEVEN 3");
|
|
|
- final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
|
|
|
+ final Suite s = new Suite(cluster, nNameNodes, nDataNodes, null, conf);
|
|
|
blocks = generateBlocks(s, usedSpacePerNN);
|
|
|
LOG.info("UNEVEN 4");
|
|
|
} finally {
|
|
@@ -280,7 +384,20 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
try {
|
|
|
cluster.waitActive();
|
|
|
LOG.info("UNEVEN 12");
|
|
|
- final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
|
|
|
+ Set<String> blockpools = new HashSet<String>();
|
|
|
+ for (int i = 0; i < nNameNodesToBalance; i++) {
|
|
|
+ blockpools.add(cluster.getNamesystem(i).getBlockPoolId());
|
|
|
+ }
|
|
|
+ Balancer.Parameters params =
|
|
|
+ new Balancer.Parameters(Balancer.Parameters.DEFAULT.policy,
|
|
|
+ Balancer.Parameters.DEFAULT.threshold,
|
|
|
+ Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
|
+ Balancer.Parameters.DEFAULT.excludedNodes,
|
|
|
+ Balancer.Parameters.DEFAULT.includedNodes,
|
|
|
+ Balancer.Parameters.DEFAULT.sourceNodes, blockpools,
|
|
|
+ Balancer.Parameters.DEFAULT.runDuringUpgrade);
|
|
|
+ final Suite s =
|
|
|
+ new Suite(cluster, nNameNodes, nDataNodes, params, conf);
|
|
|
for(int n = 0; n < nNameNodes; n++) {
|
|
|
// redistribute blocks
|
|
|
final Block[][] blocksDN = TestBalancer.distributeBlocks(
|
|
@@ -336,7 +453,9 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
try {
|
|
|
cluster.waitActive();
|
|
|
LOG.info("RUN_TEST 1");
|
|
|
- final Suite s = new Suite(cluster, nNameNodes, nDataNodes, conf);
|
|
|
+ final Suite s =
|
|
|
+ new Suite(cluster, nNameNodes, nDataNodes,
|
|
|
+ Balancer.Parameters.DEFAULT, conf);
|
|
|
long totalCapacity = TestBalancer.sum(capacities);
|
|
|
|
|
|
LOG.info("RUN_TEST 2");
|
|
@@ -378,10 +497,26 @@ public class TestBalancerWithMultipleNameNodes {
|
|
|
@Test
|
|
|
public void testUnevenDistribution() throws Exception {
|
|
|
final Configuration conf = createConf();
|
|
|
- unevenDistribution(2,
|
|
|
+ unevenDistribution(2, 2,
|
|
|
new long[] {30*CAPACITY/100, 5*CAPACITY/100},
|
|
|
new long[]{CAPACITY, CAPACITY},
|
|
|
new String[] {RACK0, RACK1},
|
|
|
conf);
|
|
|
}
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBalancing1OutOf2Blockpools() throws Exception {
|
|
|
+ final Configuration conf = createConf();
|
|
|
+ unevenDistribution(2, 1, new long[] { 30 * CAPACITY / 100,
|
|
|
+ 5 * CAPACITY / 100 }, new long[] { CAPACITY, CAPACITY }, new String[] {
|
|
|
+ RACK0, RACK1 }, conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testBalancing2OutOf3Blockpools() throws Exception {
|
|
|
+ final Configuration conf = createConf();
|
|
|
+ unevenDistribution(3, 2, new long[] { 30 * CAPACITY / 100,
|
|
|
+ 5 * CAPACITY / 100, 10 * CAPACITY / 100 }, new long[] { CAPACITY,
|
|
|
+ CAPACITY, CAPACITY }, new String[] { RACK0, RACK1, RACK2 }, conf);
|
|
|
+ }
|
|
|
}
|