|
@@ -28,6 +28,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
@@ -160,6 +161,62 @@ public class TestDiskBalancer {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testDiskBalancerWithFederatedCluster() throws Exception {
|
|
|
+
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_DISK_BALANCER_ENABLED, true);
|
|
|
+ final int blockCount = 100;
|
|
|
+ final int blockSize = 1024;
|
|
|
+ final int diskCount = 2;
|
|
|
+ final int dataNodeCount = 1;
|
|
|
+ final int dataNodeIndex = 0;
|
|
|
+ final int sourceDiskIndex = 0;
|
|
|
+ final long cap = blockSize * 3L * blockCount;
|
|
|
+
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY, blockSize);
|
|
|
+
|
|
|
+ final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
|
|
|
+ .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(2))
|
|
|
+ .numDataNodes(dataNodeCount)
|
|
|
+ .storagesPerDatanode(diskCount)
|
|
|
+ .storageCapacities(new long[] {cap, cap})
|
|
|
+ .build();
|
|
|
+ cluster.waitActive();
|
|
|
+
|
|
|
+ DFSTestUtil.setFederatedConfiguration(cluster, conf);
|
|
|
+
|
|
|
+ final String fileName = "/tmp.txt";
|
|
|
+ final Path filePath = new Path(fileName);
|
|
|
+ long fileLen = blockCount * blockSize;
|
|
|
+
|
|
|
+
|
|
|
+ FileSystem fs = cluster.getFileSystem(0);
|
|
|
+ TestBalancer.createFile(cluster, filePath, fileLen, (short) 1,
|
|
|
+ 0);
|
|
|
+ DFSTestUtil.waitReplication(fs, filePath, (short) 1);
|
|
|
+
|
|
|
+ fs = cluster.getFileSystem(1);
|
|
|
+ TestBalancer.createFile(cluster, filePath, fileLen, (short) 1,
|
|
|
+ 1);
|
|
|
+ DFSTestUtil.waitReplication(fs, filePath, (short) 1);
|
|
|
+
|
|
|
+ try {
|
|
|
+ DataMover dataMover = new DataMover(cluster, dataNodeIndex,
|
|
|
+ sourceDiskIndex, conf, blockSize, blockCount);
|
|
|
+ dataMover.moveDataToSourceDisk();
|
|
|
+ NodePlan plan = dataMover.generatePlan();
|
|
|
+ dataMover.executePlan(plan);
|
|
|
+ dataMover.verifyPlanExectionDone();
|
|
|
+ dataMover.verifyAllVolumesHaveData();
|
|
|
+ dataMover.verifyTolerance(plan, 0, sourceDiskIndex, 10);
|
|
|
+ } finally {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testBalanceDataBetweenMultiplePairsOfVolumes()
|
|
|
throws Exception {
|
|
@@ -599,9 +656,9 @@ public class TestDiskBalancer {
|
|
|
try (FsDatasetSpi.FsVolumeReferences refs =
|
|
|
node.getFSDataset().getFsVolumeReferences()) {
|
|
|
for (FsVolumeSpi volume : refs) {
|
|
|
- assertTrue(DiskBalancerTestUtil.getBlockCount(volume) > 0);
|
|
|
- LOG.info(refs.toString() + " : Block Count : {}",
|
|
|
- DiskBalancerTestUtil.getBlockCount(volume));
|
|
|
+ assertTrue(DiskBalancerTestUtil.getBlockCount(volume, true) > 0);
|
|
|
+ LOG.info("{} : Block Count : {}", refs, DiskBalancerTestUtil
|
|
|
+ .getBlockCount(volume, true));
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -618,12 +675,11 @@ public class TestDiskBalancer {
|
|
|
try (FsDatasetSpi.FsVolumeReferences refs =
|
|
|
node.getFSDataset().getFsVolumeReferences()) {
|
|
|
volume = (FsVolumeImpl) refs.get(sourceDiskIndex);
|
|
|
- assertTrue(DiskBalancerTestUtil.getBlockCount(volume) > 0);
|
|
|
+ assertTrue(DiskBalancerTestUtil.getBlockCount(volume, true) > 0);
|
|
|
|
|
|
- assertTrue(
|
|
|
- (DiskBalancerTestUtil.getBlockCount(volume) *
|
|
|
- (blockSize + delta)) >=
|
|
|
- plan.getVolumeSetPlans().get(0).getBytesToMove());
|
|
|
+ assertTrue((DiskBalancerTestUtil.getBlockCount(volume, true) *
|
|
|
+ (blockSize + delta)) >= plan.getVolumeSetPlans().get(0)
|
|
|
+ .getBytesToMove());
|
|
|
}
|
|
|
}
|
|
|
}
|