|
@@ -1580,6 +1580,85 @@ public class TestBalancer {
|
|
|
CAPACITY, RACK2, new PortNumberBasedNodes(3, 0, 1), true, true);
|
|
|
}
|
|
|
|
|
|
+
|
|
|
+ @Test(timeout = 100000)
|
|
|
+ public void testMaxIterationTime() throws Exception {
|
|
|
+ final Configuration conf = new HdfsConfiguration();
|
|
|
+ initConf(conf);
|
|
|
+ int blockSize = 10*1024*1024; // 10MB block size
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, blockSize);
|
|
|
+ // limit the worker thread count of Balancer to have only 1 queue per DN
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BALANCER_MOVERTHREADS_KEY, 1);
|
|
|
+ // limit the bandwitdh to 1 packet per sec to emulate slow block moves
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_DATANODE_BALANCE_BANDWIDTHPERSEC_KEY,
|
|
|
+ 64 * 1024);
|
|
|
+ // set client socket timeout to have an IN_PROGRESS notification back from
|
|
|
+ // the DataNode about the copy in every second.
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 2000L);
|
|
|
+ // set max iteration time to 2 seconds to timeout before moving any block
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BALANCER_MAX_ITERATION_TIME_KEY, 2000L);
|
|
|
+ // setup the cluster
|
|
|
+ final long capacity = 10L * blockSize;
|
|
|
+ final long[] dnCapacities = new long[] {capacity, capacity};
|
|
|
+ final short rep = 1;
|
|
|
+ final long seed = 0xFAFAFA;
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .numDataNodes(0)
|
|
|
+ .build();
|
|
|
+ try {
|
|
|
+ cluster.getConfiguration(0).setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
|
|
|
+ cluster.startDataNodes(conf, 1, true, null, null, dnCapacities);
|
|
|
+ cluster.waitClusterUp();
|
|
|
+ cluster.waitActive();
|
|
|
+ final Path path = new Path("/testMaxIterationTime.dat");
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
+ // fill the DN to 40%
|
|
|
+ DFSTestUtil.createFile(fs, path, 4L * blockSize, rep, seed);
|
|
|
+ // start a new DN
|
|
|
+ cluster.startDataNodes(conf, 1, true, null, null, dnCapacities);
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+ // setup Balancer and run one iteration
|
|
|
+ List<NameNodeConnector> connectors = Collections.emptyList();
|
|
|
+ try {
|
|
|
+ BalancerParameters bParams = BalancerParameters.DEFAULT;
|
|
|
+ connectors = NameNodeConnector.newNameNodeConnectors(
|
|
|
+ DFSUtil.getInternalNsRpcUris(conf), Balancer.class.getSimpleName(),
|
|
|
+ Balancer.BALANCER_ID_PATH, conf, bParams.getMaxIdleIteration());
|
|
|
+ for (NameNodeConnector nnc : connectors) {
|
|
|
+ LOG.info("NNC to work on: " + nnc);
|
|
|
+ Balancer b = new Balancer(nnc, bParams, conf);
|
|
|
+ long startTime = Time.monotonicNow();
|
|
|
+ Result r = b.runOneIteration();
|
|
|
+ long runtime = Time.monotonicNow() - startTime;
|
|
|
+ assertEquals("We expect ExitStatus.IN_PROGRESS to be reported.",
|
|
|
+ ExitStatus.IN_PROGRESS, r.exitStatus);
|
|
|
+ // accept runtime if it is under 3.5 seconds, as we need to wait for
|
|
|
+ // IN_PROGRESS report from DN, and some spare to be able to finish.
|
|
|
+ // NOTE: This can be a source of flaky tests, if the box is busy,
|
|
|
+ // assertion here is based on the following: Balancer is already set
|
|
|
+ // up, iteration gets the blocks from the NN, and makes the decision
|
|
|
+ // to move 2 blocks. After that the PendingMoves are scheduled, and
|
|
|
+ // DataNode heartbeats in for the Balancer every second, iteration is
|
|
|
+ // two seconds long. This means that it will fail if the setup and the
|
|
|
+ // heartbeat from the DataNode takes more than 500ms, as the iteration
|
|
|
+ // should end at the 3rd second from start. As the number of
|
|
|
+ // operations seems to be pretty low, and all comm happens locally, I
|
|
|
+ // think the possibility of a failure due to node busyness is low.
|
|
|
+ assertTrue("Unexpected iteration runtime: " + runtime + "ms > 3.5s",
|
|
|
+ runtime < 3500);
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ for (NameNodeConnector nnc : connectors) {
|
|
|
+ IOUtils.cleanupWithLogger(null, nnc);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown(true, true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/*
|
|
|
* Test Balancer with Ram_Disk configured
|
|
|
* One DN has two files on RAM_DISK, other DN has no files on RAM_DISK.
|