|
@@ -33,6 +33,7 @@ import static org.junit.Assume.assumeTrue;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
+import java.io.OutputStream;
|
|
import java.io.PrintWriter;
|
|
import java.io.PrintWriter;
|
|
import java.net.InetAddress;
|
|
import java.net.InetAddress;
|
|
import java.net.InetSocketAddress;
|
|
import java.net.InetSocketAddress;
|
|
@@ -397,11 +398,11 @@ public class TestBalancer {
|
|
long timeout = TIMEOUT;
|
|
long timeout = TIMEOUT;
|
|
long failtime = (timeout <= 0L) ? Long.MAX_VALUE
|
|
long failtime = (timeout <= 0L) ? Long.MAX_VALUE
|
|
: Time.monotonicNow() + timeout;
|
|
: Time.monotonicNow() + timeout;
|
|
- if (!p.nodesToBeIncluded.isEmpty()) {
|
|
|
|
- totalCapacity = p.nodesToBeIncluded.size() * CAPACITY;
|
|
|
|
|
|
+ if (!p.includedNodes.isEmpty()) {
|
|
|
|
+ totalCapacity = p.includedNodes.size() * CAPACITY;
|
|
}
|
|
}
|
|
- if (!p.nodesToBeExcluded.isEmpty()) {
|
|
|
|
- totalCapacity -= p.nodesToBeExcluded.size() * CAPACITY;
|
|
|
|
|
|
+ if (!p.excludedNodes.isEmpty()) {
|
|
|
|
+ totalCapacity -= p.excludedNodes.size() * CAPACITY;
|
|
}
|
|
}
|
|
final double avgUtilization = ((double)totalUsedSpace) / totalCapacity;
|
|
final double avgUtilization = ((double)totalUsedSpace) / totalCapacity;
|
|
boolean balanced;
|
|
boolean balanced;
|
|
@@ -414,12 +415,12 @@ public class TestBalancer {
|
|
for (DatanodeInfo datanode : datanodeReport) {
|
|
for (DatanodeInfo datanode : datanodeReport) {
|
|
double nodeUtilization = ((double)datanode.getDfsUsed())
|
|
double nodeUtilization = ((double)datanode.getDfsUsed())
|
|
/ datanode.getCapacity();
|
|
/ datanode.getCapacity();
|
|
- if (Dispatcher.Util.isExcluded(p.nodesToBeExcluded, datanode)) {
|
|
|
|
|
|
+ if (Dispatcher.Util.isExcluded(p.excludedNodes, datanode)) {
|
|
assertTrue(nodeUtilization == 0);
|
|
assertTrue(nodeUtilization == 0);
|
|
actualExcludedNodeCount++;
|
|
actualExcludedNodeCount++;
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
- if (!Dispatcher.Util.isIncluded(p.nodesToBeIncluded, datanode)) {
|
|
|
|
|
|
+ if (!Dispatcher.Util.isIncluded(p.includedNodes, datanode)) {
|
|
assertTrue(nodeUtilization == 0);
|
|
assertTrue(nodeUtilization == 0);
|
|
actualExcludedNodeCount++;
|
|
actualExcludedNodeCount++;
|
|
continue;
|
|
continue;
|
|
@@ -642,6 +643,7 @@ public class TestBalancer {
|
|
Balancer.Parameters.DEFAULT.threshold,
|
|
Balancer.Parameters.DEFAULT.threshold,
|
|
Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
nodes.getNodesToBeExcluded(), nodes.getNodesToBeIncluded(),
|
|
nodes.getNodesToBeExcluded(), nodes.getNodesToBeIncluded(),
|
|
|
|
+ Balancer.Parameters.DEFAULT.sourceNodes,
|
|
false);
|
|
false);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -754,36 +756,36 @@ public class TestBalancer {
|
|
args.add("datanode");
|
|
args.add("datanode");
|
|
|
|
|
|
File excludeHostsFile = null;
|
|
File excludeHostsFile = null;
|
|
- if (!p.nodesToBeExcluded.isEmpty()) {
|
|
|
|
|
|
+ if (!p.excludedNodes.isEmpty()) {
|
|
args.add("-exclude");
|
|
args.add("-exclude");
|
|
if (useFile) {
|
|
if (useFile) {
|
|
excludeHostsFile = new File ("exclude-hosts-file");
|
|
excludeHostsFile = new File ("exclude-hosts-file");
|
|
PrintWriter pw = new PrintWriter(excludeHostsFile);
|
|
PrintWriter pw = new PrintWriter(excludeHostsFile);
|
|
- for (String host: p.nodesToBeExcluded) {
|
|
|
|
|
|
+ for (String host: p.excludedNodes) {
|
|
pw.write( host + "\n");
|
|
pw.write( host + "\n");
|
|
}
|
|
}
|
|
pw.close();
|
|
pw.close();
|
|
args.add("-f");
|
|
args.add("-f");
|
|
args.add("exclude-hosts-file");
|
|
args.add("exclude-hosts-file");
|
|
} else {
|
|
} else {
|
|
- args.add(StringUtils.join(p.nodesToBeExcluded, ','));
|
|
|
|
|
|
+ args.add(StringUtils.join(p.excludedNodes, ','));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
File includeHostsFile = null;
|
|
File includeHostsFile = null;
|
|
- if (!p.nodesToBeIncluded.isEmpty()) {
|
|
|
|
|
|
+ if (!p.includedNodes.isEmpty()) {
|
|
args.add("-include");
|
|
args.add("-include");
|
|
if (useFile) {
|
|
if (useFile) {
|
|
includeHostsFile = new File ("include-hosts-file");
|
|
includeHostsFile = new File ("include-hosts-file");
|
|
PrintWriter pw = new PrintWriter(includeHostsFile);
|
|
PrintWriter pw = new PrintWriter(includeHostsFile);
|
|
- for (String host: p.nodesToBeIncluded){
|
|
|
|
|
|
+ for (String host: p.includedNodes){
|
|
pw.write( host + "\n");
|
|
pw.write( host + "\n");
|
|
}
|
|
}
|
|
pw.close();
|
|
pw.close();
|
|
args.add("-f");
|
|
args.add("-f");
|
|
args.add("include-hosts-file");
|
|
args.add("include-hosts-file");
|
|
} else {
|
|
} else {
|
|
- args.add(StringUtils.join(p.nodesToBeIncluded, ','));
|
|
|
|
|
|
+ args.add(StringUtils.join(p.includedNodes, ','));
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -881,7 +883,8 @@ public class TestBalancer {
|
|
Balancer.Parameters.DEFAULT.policy,
|
|
Balancer.Parameters.DEFAULT.policy,
|
|
Balancer.Parameters.DEFAULT.threshold,
|
|
Balancer.Parameters.DEFAULT.threshold,
|
|
Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
- datanodes, Balancer.Parameters.DEFAULT.nodesToBeIncluded,
|
|
|
|
|
|
+ datanodes, Balancer.Parameters.DEFAULT.includedNodes,
|
|
|
|
+ Balancer.Parameters.DEFAULT.sourceNodes,
|
|
false);
|
|
false);
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
|
|
assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
|
|
@@ -1094,7 +1097,7 @@ public class TestBalancer {
|
|
excludeHosts.add( "datanodeZ");
|
|
excludeHosts.add( "datanodeZ");
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
- excludeHosts, Parameters.DEFAULT.nodesToBeIncluded), false, false);
|
|
|
|
|
|
+ excludeHosts, Parameters.DEFAULT.includedNodes), false, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1124,7 +1127,7 @@ public class TestBalancer {
|
|
excludeHosts.add( "datanodeZ");
|
|
excludeHosts.add( "datanodeZ");
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"}, excludeHosts,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"}, excludeHosts,
|
|
- Parameters.DEFAULT.nodesToBeIncluded), true, false);
|
|
|
|
|
|
+ Parameters.DEFAULT.includedNodes), true, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1154,7 +1157,7 @@ public class TestBalancer {
|
|
excludeHosts.add( "datanodeZ");
|
|
excludeHosts.add( "datanodeZ");
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
- excludeHosts, Parameters.DEFAULT.nodesToBeIncluded), true, true);
|
|
|
|
|
|
+ excludeHosts, Parameters.DEFAULT.includedNodes), true, true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1183,7 +1186,7 @@ public class TestBalancer {
|
|
includeHosts.add( "datanodeY");
|
|
includeHosts.add( "datanodeY");
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
- Parameters.DEFAULT.nodesToBeExcluded, includeHosts), false, false);
|
|
|
|
|
|
+ Parameters.DEFAULT.excludedNodes, includeHosts), false, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1212,7 +1215,7 @@ public class TestBalancer {
|
|
includeHosts.add( "datanodeY");
|
|
includeHosts.add( "datanodeY");
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
- Parameters.DEFAULT.nodesToBeExcluded, includeHosts), true, false);
|
|
|
|
|
|
+ Parameters.DEFAULT.excludedNodes, includeHosts), true, false);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1241,7 +1244,7 @@ public class TestBalancer {
|
|
includeHosts.add( "datanodeY");
|
|
includeHosts.add( "datanodeY");
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
- Parameters.DEFAULT.nodesToBeExcluded, includeHosts), true, true);
|
|
|
|
|
|
+ Parameters.DEFAULT.excludedNodes, includeHosts), true, true);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1381,8 +1384,9 @@ public class TestBalancer {
|
|
new Balancer.Parameters(Parameters.DEFAULT.policy,
|
|
new Balancer.Parameters(Parameters.DEFAULT.policy,
|
|
Parameters.DEFAULT.threshold,
|
|
Parameters.DEFAULT.threshold,
|
|
Parameters.DEFAULT.maxIdleIteration,
|
|
Parameters.DEFAULT.maxIdleIteration,
|
|
- Parameters.DEFAULT.nodesToBeExcluded,
|
|
|
|
- Parameters.DEFAULT.nodesToBeIncluded,
|
|
|
|
|
|
+ Parameters.DEFAULT.excludedNodes,
|
|
|
|
+ Parameters.DEFAULT.includedNodes,
|
|
|
|
+ Parameters.DEFAULT.sourceNodes,
|
|
true);
|
|
true);
|
|
assertEquals(ExitStatus.SUCCESS.getExitCode(),
|
|
assertEquals(ExitStatus.SUCCESS.getExitCode(),
|
|
Balancer.run(namenodes, runDuringUpgrade, conf));
|
|
Balancer.run(namenodes, runDuringUpgrade, conf));
|
|
@@ -1538,6 +1542,116 @@ public class TestBalancer {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /** Balancer should not move blocks with size < minBlockSize. */
|
|
|
|
+ @Test(timeout=60000)
|
|
|
|
+ public void testMinBlockSizeAndSourceNodes() throws Exception {
|
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
|
+ initConf(conf);
|
|
|
|
+
|
|
|
|
+ final short replication = 3;
|
|
|
|
+ final long[] lengths = {10, 10, 10, 10};
|
|
|
|
+ final long[] capacities = new long[replication];
|
|
|
|
+ final long totalUsed = capacities.length * sum(lengths);
|
|
|
|
+ Arrays.fill(capacities, 1000);
|
|
|
|
+
|
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
|
+ .numDataNodes(capacities.length)
|
|
|
|
+ .simulatedCapacities(capacities)
|
|
|
|
+ .build();
|
|
|
|
+ final DistributedFileSystem dfs = cluster.getFileSystem();
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ cluster.waitActive();
|
|
|
|
+ client = NameNodeProxies.createProxy(conf, dfs.getUri(),
|
|
|
|
+ ClientProtocol.class).getProxy();
|
|
|
|
+
|
|
|
|
+ // fill up the cluster to be 80% full
|
|
|
|
+ for(int i = 0; i < lengths.length; i++) {
|
|
|
|
+ final long size = lengths[i];
|
|
|
|
+ final Path p = new Path("/file" + i + "_size" + size);
|
|
|
|
+ try(final OutputStream out = dfs.create(p)) {
|
|
|
|
+ for(int j = 0; j < size; j++) {
|
|
|
|
+ out.write(j);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // start up an empty node with the same capacity
|
|
|
|
+ cluster.startDataNodes(conf, capacities.length, true, null, null, capacities);
|
|
|
|
+ LOG.info("capacities = " + Arrays.toString(capacities));
|
|
|
|
+ LOG.info("totalUsedSpace= " + totalUsed);
|
|
|
|
+ LOG.info("lengths = " + Arrays.toString(lengths) + ", #=" + lengths.length);
|
|
|
|
+ waitForHeartBeat(totalUsed, 2*capacities[0]*capacities.length, client, cluster);
|
|
|
|
+
|
|
|
|
+ final Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
|
+
|
|
|
|
+ { // run Balancer with min-block-size=50
|
|
|
|
+ final Parameters p = new Parameters(
|
|
|
|
+ BalancingPolicy.Node.INSTANCE, 1,
|
|
|
|
+ NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
|
|
|
|
+ Collections.<String> emptySet(), Collections.<String> emptySet(),
|
|
|
|
+ Collections.<String> emptySet(), false);
|
|
|
|
+
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
|
|
|
|
+ final int r = Balancer.run(namenodes, p, conf);
|
|
|
|
+ assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
|
|
|
|
+
|
|
|
|
+ { // run Balancer with empty nodes as source nodes
|
|
|
|
+ final Set<String> sourceNodes = new HashSet<>();
|
|
|
|
+ final List<DataNode> datanodes = cluster.getDataNodes();
|
|
|
|
+ for(int i = capacities.length; i < datanodes.size(); i++) {
|
|
|
|
+ sourceNodes.add(datanodes.get(i).getDisplayName());
|
|
|
|
+ }
|
|
|
|
+ final Parameters p = new Parameters(
|
|
|
|
+ BalancingPolicy.Node.INSTANCE, 1,
|
|
|
|
+ NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
|
|
|
|
+ Collections.<String> emptySet(), Collections.<String> emptySet(),
|
|
|
|
+ sourceNodes, false);
|
|
|
|
+
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
|
|
|
|
+ final int r = Balancer.run(namenodes, p, conf);
|
|
|
|
+ assertEquals(ExitStatus.NO_MOVE_BLOCK.getExitCode(), r);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ { // run Balancer with a filled node as a source node
|
|
|
|
+ final Set<String> sourceNodes = new HashSet<>();
|
|
|
|
+ final List<DataNode> datanodes = cluster.getDataNodes();
|
|
|
|
+ sourceNodes.add(datanodes.get(0).getDisplayName());
|
|
|
|
+ final Parameters p = new Parameters(
|
|
|
|
+ BalancingPolicy.Node.INSTANCE, 1,
|
|
|
|
+ NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
|
|
|
|
+ Collections.<String> emptySet(), Collections.<String> emptySet(),
|
|
|
|
+ sourceNodes, false);
|
|
|
|
+
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
|
|
|
|
+ final int r = Balancer.run(namenodes, p, conf);
|
|
|
|
+ assertEquals(ExitStatus.NO_MOVE_BLOCK.getExitCode(), r);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ { // run Balancer with all filled node as source nodes
|
|
|
|
+ final Set<String> sourceNodes = new HashSet<>();
|
|
|
|
+ final List<DataNode> datanodes = cluster.getDataNodes();
|
|
|
|
+ for(int i = 0; i < capacities.length; i++) {
|
|
|
|
+ sourceNodes.add(datanodes.get(i).getDisplayName());
|
|
|
|
+ }
|
|
|
|
+ final Parameters p = new Parameters(
|
|
|
|
+ BalancingPolicy.Node.INSTANCE, 1,
|
|
|
|
+ NameNodeConnector.DEFAULT_MAX_IDLE_ITERATIONS,
|
|
|
|
+ Collections.<String> emptySet(), Collections.<String> emptySet(),
|
|
|
|
+ sourceNodes, false);
|
|
|
|
+
|
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
|
|
|
|
+ final int r = Balancer.run(namenodes, p, conf);
|
|
|
|
+ assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* @param args
|
|
* @param args
|
|
*/
|
|
*/
|