|
@@ -75,8 +75,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
|
|
|
-import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
|
|
|
import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
|
|
|
+import org.apache.hadoop.hdfs.server.balancer.BalancerParameters;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
|
|
@@ -319,7 +319,7 @@ public class TestBalancer {
|
|
|
* @throws TimeoutException
|
|
|
*/
|
|
|
static void waitForBalancer(long totalUsedSpace, long totalCapacity,
|
|
|
- ClientProtocol client, MiniDFSCluster cluster, Balancer.Parameters p)
|
|
|
+ ClientProtocol client, MiniDFSCluster cluster, BalancerParameters p)
|
|
|
throws IOException, TimeoutException {
|
|
|
waitForBalancer(totalUsedSpace, totalCapacity, client, cluster, p, 0);
|
|
|
}
|
|
@@ -377,7 +377,7 @@ public class TestBalancer {
|
|
|
|
|
|
// start rebalancing
|
|
|
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
- int r = Balancer.run(namenodes, Balancer.Parameters.DEFAULT, conf);
|
|
|
+ int r = Balancer.run(namenodes, BalancerParameters.DEFAULT, conf);
|
|
|
assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
|
|
|
|
|
|
} finally {
|
|
@@ -393,16 +393,16 @@ public class TestBalancer {
|
|
|
* @throws TimeoutException
|
|
|
*/
|
|
|
static void waitForBalancer(long totalUsedSpace, long totalCapacity,
|
|
|
- ClientProtocol client, MiniDFSCluster cluster, Balancer.Parameters p,
|
|
|
+ ClientProtocol client, MiniDFSCluster cluster, BalancerParameters p,
|
|
|
int expectedExcludedNodes) throws IOException, TimeoutException {
|
|
|
long timeout = TIMEOUT;
|
|
|
long failtime = (timeout <= 0L) ? Long.MAX_VALUE
|
|
|
: Time.monotonicNow() + timeout;
|
|
|
- if (!p.includedNodes.isEmpty()) {
|
|
|
- totalCapacity = p.includedNodes.size() * CAPACITY;
|
|
|
+ if (!p.getIncludedNodes().isEmpty()) {
|
|
|
+ totalCapacity = p.getIncludedNodes().size() * CAPACITY;
|
|
|
}
|
|
|
- if (!p.excludedNodes.isEmpty()) {
|
|
|
- totalCapacity -= p.excludedNodes.size() * CAPACITY;
|
|
|
+ if (!p.getExcludedNodes().isEmpty()) {
|
|
|
+ totalCapacity -= p.getExcludedNodes().size() * CAPACITY;
|
|
|
}
|
|
|
final double avgUtilization = ((double)totalUsedSpace) / totalCapacity;
|
|
|
boolean balanced;
|
|
@@ -415,12 +415,12 @@ public class TestBalancer {
|
|
|
for (DatanodeInfo datanode : datanodeReport) {
|
|
|
double nodeUtilization = ((double)datanode.getDfsUsed())
|
|
|
/ datanode.getCapacity();
|
|
|
- if (Dispatcher.Util.isExcluded(p.excludedNodes, datanode)) {
|
|
|
+ if (Dispatcher.Util.isExcluded(p.getExcludedNodes(), datanode)) {
|
|
|
assertTrue(nodeUtilization == 0);
|
|
|
actualExcludedNodeCount++;
|
|
|
continue;
|
|
|
}
|
|
|
- if (!Dispatcher.Util.isIncluded(p.includedNodes, datanode)) {
|
|
|
+ if (!Dispatcher.Util.isIncluded(p.getIncludedNodes(), datanode)) {
|
|
|
assertTrue(nodeUtilization == 0);
|
|
|
actualExcludedNodeCount++;
|
|
|
continue;
|
|
@@ -636,16 +636,14 @@ public class TestBalancer {
|
|
|
}
|
|
|
}
|
|
|
// run balancer and validate results
|
|
|
- Balancer.Parameters p = Balancer.Parameters.DEFAULT;
|
|
|
+ BalancerParameters.Builder pBuilder =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
if (nodes != null) {
|
|
|
- p = new Balancer.Parameters(
|
|
|
- Balancer.Parameters.DEFAULT.policy,
|
|
|
- Balancer.Parameters.DEFAULT.threshold,
|
|
|
- Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
|
- nodes.getNodesToBeExcluded(), nodes.getNodesToBeIncluded(),
|
|
|
- Balancer.Parameters.DEFAULT.sourceNodes,
|
|
|
- Balancer.Parameters.DEFAULT.blockpools, false);
|
|
|
+ pBuilder.setExcludedNodes(nodes.getNodesToBeExcluded());
|
|
|
+ pBuilder.setIncludedNodes(nodes.getNodesToBeIncluded());
|
|
|
+ pBuilder.setRunDuringUpgrade(false);
|
|
|
}
|
|
|
+ BalancerParameters p = pBuilder.build();
|
|
|
|
|
|
int expectedExcludedNodes = 0;
|
|
|
if (nodes != null) {
|
|
@@ -668,14 +666,15 @@ public class TestBalancer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void runBalancer(Configuration conf,
|
|
|
- long totalUsedSpace, long totalCapacity) throws Exception {
|
|
|
- runBalancer(conf, totalUsedSpace, totalCapacity, Balancer.Parameters.DEFAULT, 0);
|
|
|
+ private void runBalancer(Configuration conf, long totalUsedSpace,
|
|
|
+ long totalCapacity) throws Exception {
|
|
|
+ runBalancer(conf, totalUsedSpace, totalCapacity,
|
|
|
+ BalancerParameters.DEFAULT, 0);
|
|
|
}
|
|
|
|
|
|
- private void runBalancer(Configuration conf,
|
|
|
- long totalUsedSpace, long totalCapacity, Balancer.Parameters p,
|
|
|
- int excludedNodes) throws Exception {
|
|
|
+ private void runBalancer(Configuration conf, long totalUsedSpace,
|
|
|
+ long totalCapacity, BalancerParameters p, int excludedNodes)
|
|
|
+ throws Exception {
|
|
|
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
|
|
|
|
|
|
// start rebalancing
|
|
@@ -693,7 +692,8 @@ public class TestBalancer {
|
|
|
waitForBalancer(totalUsedSpace, totalCapacity, client, cluster, p, excludedNodes);
|
|
|
}
|
|
|
|
|
|
- private static int runBalancer(Collection<URI> namenodes, final Parameters p,
|
|
|
+ private static int runBalancer(Collection<URI> namenodes,
|
|
|
+ final BalancerParameters p,
|
|
|
Configuration conf) throws IOException, InterruptedException {
|
|
|
final long sleeptime =
|
|
|
conf.getLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
|
|
@@ -710,8 +710,8 @@ public class TestBalancer {
|
|
|
try {
|
|
|
connectors = NameNodeConnector.newNameNodeConnectors(namenodes,
|
|
|
Balancer.class.getSimpleName(), Balancer.BALANCER_ID_PATH, conf,
|
|
|
- Balancer.Parameters.DEFAULT.maxIdleIteration);
|
|
|
-
|
|
|
+ BalancerParameters.DEFAULT.getMaxIdleIteration());
|
|
|
+
|
|
|
boolean done = false;
|
|
|
for(int iteration = 0; !done; iteration++) {
|
|
|
done = true;
|
|
@@ -747,45 +747,45 @@ public class TestBalancer {
|
|
|
return ExitStatus.SUCCESS.getExitCode();
|
|
|
}
|
|
|
|
|
|
- private void runBalancerCli(Configuration conf,
|
|
|
- long totalUsedSpace, long totalCapacity,
|
|
|
- Balancer.Parameters p, boolean useFile, int expectedExcludedNodes) throws Exception {
|
|
|
+ private void runBalancerCli(Configuration conf, long totalUsedSpace,
|
|
|
+ long totalCapacity, BalancerParameters p, boolean useFile,
|
|
|
+ int expectedExcludedNodes) throws Exception {
|
|
|
waitForHeartBeat(totalUsedSpace, totalCapacity, client, cluster);
|
|
|
List <String> args = new ArrayList<String>();
|
|
|
args.add("-policy");
|
|
|
args.add("datanode");
|
|
|
|
|
|
File excludeHostsFile = null;
|
|
|
- if (!p.excludedNodes.isEmpty()) {
|
|
|
+ if (!p.getExcludedNodes().isEmpty()) {
|
|
|
args.add("-exclude");
|
|
|
if (useFile) {
|
|
|
excludeHostsFile = new File ("exclude-hosts-file");
|
|
|
PrintWriter pw = new PrintWriter(excludeHostsFile);
|
|
|
- for (String host: p.excludedNodes) {
|
|
|
+ for (String host : p.getExcludedNodes()) {
|
|
|
pw.write( host + "\n");
|
|
|
}
|
|
|
pw.close();
|
|
|
args.add("-f");
|
|
|
args.add("exclude-hosts-file");
|
|
|
} else {
|
|
|
- args.add(StringUtils.join(p.excludedNodes, ','));
|
|
|
+ args.add(StringUtils.join(p.getExcludedNodes(), ','));
|
|
|
}
|
|
|
}
|
|
|
|
|
|
File includeHostsFile = null;
|
|
|
- if (!p.includedNodes.isEmpty()) {
|
|
|
+ if (!p.getIncludedNodes().isEmpty()) {
|
|
|
args.add("-include");
|
|
|
if (useFile) {
|
|
|
includeHostsFile = new File ("include-hosts-file");
|
|
|
PrintWriter pw = new PrintWriter(includeHostsFile);
|
|
|
- for (String host: p.includedNodes){
|
|
|
+ for (String host : p.getIncludedNodes()) {
|
|
|
pw.write( host + "\n");
|
|
|
}
|
|
|
pw.close();
|
|
|
args.add("-f");
|
|
|
args.add("include-hosts-file");
|
|
|
} else {
|
|
|
- args.add(StringUtils.join(p.includedNodes, ','));
|
|
|
+ args.add(StringUtils.join(p.getIncludedNodes(), ','));
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -879,14 +879,11 @@ public class TestBalancer {
|
|
|
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
Set<String> datanodes = new HashSet<String>();
|
|
|
datanodes.add(cluster.getDataNodes().get(0).getDatanodeId().getHostName());
|
|
|
- Balancer.Parameters p = new Balancer.Parameters(
|
|
|
- Balancer.Parameters.DEFAULT.policy,
|
|
|
- Balancer.Parameters.DEFAULT.threshold,
|
|
|
- Balancer.Parameters.DEFAULT.maxIdleIteration,
|
|
|
- datanodes, Balancer.Parameters.DEFAULT.includedNodes,
|
|
|
- Balancer.Parameters.DEFAULT.sourceNodes,
|
|
|
- Balancer.Parameters.DEFAULT.blockpools, false);
|
|
|
- final int r = Balancer.run(namenodes, p, conf);
|
|
|
+ BalancerParameters.Builder pBuilder =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
+ pBuilder.setExcludedNodes(datanodes);
|
|
|
+ pBuilder.setRunDuringUpgrade(false);
|
|
|
+ final int r = Balancer.run(namenodes, pBuilder.build(), conf);
|
|
|
assertEquals(ExitStatus.SUCCESS.getExitCode(), r);
|
|
|
} finally {
|
|
|
cluster.shutdown();
|
|
@@ -1081,20 +1078,20 @@ public class TestBalancer {
|
|
|
@Test
|
|
|
public void testBalancerCliParseBlockpools() {
|
|
|
String[] parameters = new String[] { "-blockpools", "bp-1,bp-2,bp-3" };
|
|
|
- Balancer.Parameters p = Balancer.Cli.parse(parameters);
|
|
|
- assertEquals(3, p.blockpools.size());
|
|
|
+ BalancerParameters p = Balancer.Cli.parse(parameters);
|
|
|
+ assertEquals(3, p.getBlockPools().size());
|
|
|
|
|
|
parameters = new String[] { "-blockpools", "bp-1" };
|
|
|
p = Balancer.Cli.parse(parameters);
|
|
|
- assertEquals(1, p.blockpools.size());
|
|
|
+ assertEquals(1, p.getBlockPools().size());
|
|
|
|
|
|
parameters = new String[] { "-blockpools", "bp-1,,bp-2" };
|
|
|
p = Balancer.Cli.parse(parameters);
|
|
|
- assertEquals(3, p.blockpools.size());
|
|
|
+ assertEquals(3, p.getBlockPools().size());
|
|
|
|
|
|
parameters = new String[] { "-blockpools", "bp-1," };
|
|
|
p = Balancer.Cli.parse(parameters);
|
|
|
- assertEquals(1, p.blockpools.size());
|
|
|
+ assertEquals(1, p.getBlockPools().size());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1123,7 +1120,8 @@ public class TestBalancer {
|
|
|
excludeHosts.add( "datanodeZ");
|
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
|
- excludeHosts, Parameters.DEFAULT.includedNodes), false, false);
|
|
|
+ excludeHosts, BalancerParameters.DEFAULT.getIncludedNodes()),
|
|
|
+ false, false);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1151,9 +1149,11 @@ public class TestBalancer {
|
|
|
Set<String> excludeHosts = new HashSet<String>();
|
|
|
excludeHosts.add( "datanodeY");
|
|
|
excludeHosts.add( "datanodeZ");
|
|
|
- doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
|
- new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"}, excludeHosts,
|
|
|
- Parameters.DEFAULT.includedNodes), true, false);
|
|
|
+ doTest(conf, new long[] { CAPACITY, CAPACITY },
|
|
|
+ new String[] { RACK0, RACK1 }, CAPACITY, RACK2, new HostNameBasedNodes(
|
|
|
+ new String[] { "datanodeX", "datanodeY", "datanodeZ" },
|
|
|
+ excludeHosts, BalancerParameters.DEFAULT.getIncludedNodes()), true,
|
|
|
+ false);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1183,7 +1183,8 @@ public class TestBalancer {
|
|
|
excludeHosts.add( "datanodeZ");
|
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
|
- excludeHosts, Parameters.DEFAULT.includedNodes), true, true);
|
|
|
+ excludeHosts, BalancerParameters.DEFAULT.getIncludedNodes()), true,
|
|
|
+ true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1212,7 +1213,8 @@ public class TestBalancer {
|
|
|
includeHosts.add( "datanodeY");
|
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
|
- Parameters.DEFAULT.excludedNodes, includeHosts), false, false);
|
|
|
+ BalancerParameters.DEFAULT.getExcludedNodes(), includeHosts),
|
|
|
+ false, false);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1241,7 +1243,8 @@ public class TestBalancer {
|
|
|
includeHosts.add( "datanodeY");
|
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
|
- Parameters.DEFAULT.excludedNodes, includeHosts), true, false);
|
|
|
+ BalancerParameters.DEFAULT.getExcludedNodes(), includeHosts), true,
|
|
|
+ false);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1270,7 +1273,8 @@ public class TestBalancer {
|
|
|
includeHosts.add( "datanodeY");
|
|
|
doTest(conf, new long[]{CAPACITY, CAPACITY}, new String[]{RACK0, RACK1}, CAPACITY, RACK2,
|
|
|
new HostNameBasedNodes(new String[] {"datanodeX", "datanodeY", "datanodeZ"},
|
|
|
- Parameters.DEFAULT.excludedNodes, includeHosts), true, true);
|
|
|
+ BalancerParameters.DEFAULT.getExcludedNodes(), includeHosts), true,
|
|
|
+ true);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1343,7 +1347,7 @@ public class TestBalancer {
|
|
|
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
|
|
|
// Run Balancer
|
|
|
- final Balancer.Parameters p = Parameters.DEFAULT;
|
|
|
+ final BalancerParameters p = BalancerParameters.DEFAULT;
|
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
|
|
|
|
// Validate no RAM_DISK block should be moved
|
|
@@ -1395,7 +1399,7 @@ public class TestBalancer {
|
|
|
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
|
|
|
// Run balancer
|
|
|
- final Balancer.Parameters p = Parameters.DEFAULT;
|
|
|
+ final BalancerParameters p = BalancerParameters.DEFAULT;
|
|
|
|
|
|
fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
|
|
|
fs.rollingUpgrade(HdfsConstants.RollingUpgradeAction.PREPARE);
|
|
@@ -1406,14 +1410,10 @@ public class TestBalancer {
|
|
|
Balancer.run(namenodes, p, conf));
|
|
|
|
|
|
// Should work with the -runDuringUpgrade flag.
|
|
|
- final Balancer.Parameters runDuringUpgrade =
|
|
|
- new Balancer.Parameters(Parameters.DEFAULT.policy,
|
|
|
- Parameters.DEFAULT.threshold,
|
|
|
- Parameters.DEFAULT.maxIdleIteration,
|
|
|
- Parameters.DEFAULT.excludedNodes,
|
|
|
- Parameters.DEFAULT.includedNodes,
|
|
|
- Parameters.DEFAULT.sourceNodes,
|
|
|
- Balancer.Parameters.DEFAULT.blockpools, true);
|
|
|
+ BalancerParameters.Builder b =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
+ b.setRunDuringUpgrade(true);
|
|
|
+ final BalancerParameters runDuringUpgrade = b.build();
|
|
|
assertEquals(ExitStatus.SUCCESS.getExitCode(),
|
|
|
Balancer.run(namenodes, runDuringUpgrade, conf));
|
|
|
|
|
@@ -1480,7 +1480,7 @@ public class TestBalancer {
|
|
|
// update space info
|
|
|
cluster.triggerHeartbeats();
|
|
|
|
|
|
- Balancer.Parameters p = Balancer.Parameters.DEFAULT;
|
|
|
+ BalancerParameters p = BalancerParameters.DEFAULT;
|
|
|
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
|
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
|
|
|
@@ -1612,12 +1612,11 @@ public class TestBalancer {
|
|
|
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(),
|
|
|
- Balancer.Parameters.DEFAULT.blockpools, false);
|
|
|
+ BalancerParameters.Builder b =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
+ b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
|
|
|
+ b.setThreshold(1);
|
|
|
+ final BalancerParameters p = b.build();
|
|
|
|
|
|
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
|
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
@@ -1632,11 +1631,12 @@ public class TestBalancer {
|
|
|
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, Balancer.Parameters.DEFAULT.blockpools, false);
|
|
|
+ BalancerParameters.Builder b =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
+ b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
|
|
|
+ b.setThreshold(1);
|
|
|
+ b.setSourceNodes(sourceNodes);
|
|
|
+ final BalancerParameters p = b.build();
|
|
|
|
|
|
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 50);
|
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
@@ -1647,11 +1647,12 @@ public class TestBalancer {
|
|
|
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, Balancer.Parameters.DEFAULT.blockpools, false);
|
|
|
+ BalancerParameters.Builder b =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
+ b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
|
|
|
+ b.setThreshold(1);
|
|
|
+ b.setSourceNodes(sourceNodes);
|
|
|
+ final BalancerParameters p = b.build();
|
|
|
|
|
|
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
|
|
|
final int r = Balancer.run(namenodes, p, conf);
|
|
@@ -1664,11 +1665,12 @@ public class TestBalancer {
|
|
|
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, Balancer.Parameters.DEFAULT.blockpools, false);
|
|
|
+ BalancerParameters.Builder b =
|
|
|
+ new BalancerParameters.Builder();
|
|
|
+ b.setBalancingPolicy(BalancingPolicy.Node.INSTANCE);
|
|
|
+ b.setThreshold(1);
|
|
|
+ b.setSourceNodes(sourceNodes);
|
|
|
+ final BalancerParameters p = b.build();
|
|
|
|
|
|
conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1);
|
|
|
final int r = Balancer.run(namenodes, p, conf);
|