HDFS-14021. TestReconstructStripedBlocksWithRackAwareness#testReconstructForNotEnoughRacks fails intermittently. Contributed by Xiao Chen.

This commit is contained in:
Inigo Goiri 2018-10-24 09:20:38 -07:00
parent bbc6dcd3d0
commit c1874046e2
1 changed files with 8 additions and 3 deletions

View File

@ -162,7 +162,9 @@ public void testReconstructForNotEnoughRacks() throws Exception {
// the file's block is in 9 dn but 5 racks
DFSTestUtil.createFile(fs, file,
cellSize * dataBlocks * 2, (short) 1, 0L);
Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
GenericTestUtils.waitFor(() ->
bm.numOfUnderReplicatedBlocks() == 0, 100, 30000);
LOG.info("Created file {}", file);
final INodeFile fileNode = fsn.getFSDirectory()
.getINode4Write(file.toString()).asFile();
@ -173,7 +175,8 @@ public void testReconstructForNotEnoughRacks() throws Exception {
for (DatanodeStorageInfo storage : blockInfo.storages) {
rackSet.add(storage.getDatanodeDescriptor().getNetworkLocation());
}
Assert.assertEquals(dataBlocks - 1, rackSet.size());
Assert.assertEquals("rackSet size is wrong: " + rackSet, dataBlocks - 1,
rackSet.size());
// restart the stopped datanode
cluster.restartDataNode(lastHost);
@ -181,6 +184,7 @@ public void testReconstructForNotEnoughRacks() throws Exception {
// make sure we have 6 racks again
NetworkTopology topology = bm.getDatanodeManager().getNetworkTopology();
LOG.info("topology is: {}", topology);
Assert.assertEquals(hosts.length, topology.getNumOfLeaves());
Assert.assertEquals(dataBlocks, topology.getNumOfRacks());
@ -202,7 +206,8 @@ public void testReconstructForNotEnoughRacks() throws Exception {
for (DatanodeStorageInfo storage : blockInfo.storages) {
if (storage != null) {
DatanodeDescriptor dn = storage.getDatanodeDescriptor();
Assert.assertEquals(0, dn.getNumberOfBlocksToBeErasureCoded());
Assert.assertEquals("Block to be erasure coded is wrong for datanode:"
+ dn, 0, dn.getNumberOfBlocksToBeErasureCoded());
if (dn.getNumberOfBlocksToBeReplicated() == 1) {
scheduled = true;
}