HDFS-13065. TestErasureCodingMultipleRacks#testSkewedRack3 is failing. Contributed by Gabor Bota.

This commit is contained in:
Xiao Chen 2018-01-28 22:11:08 -08:00
parent 443523f9c0
commit 6bc2f7f4b4
1 changed files with 7 additions and 1 deletions

View File

@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant;
@ -163,7 +164,8 @@ public class TestErasureCodingMultipleRacks {
// Create enough extra DNs on the 2 racks to test even placement.
// Desired placement is parityUnits replicas on the 2 racks, and 1 replica
// on the rest of the racks (which only have 1 DN)
setupCluster(dataUnits + parityUnits * 4, dataUnits - parityUnits + 2,
int numRacks = dataUnits - parityUnits + 2;
setupCluster(dataUnits + parityUnits * 4, numRacks,
dataUnits - parityUnits);
final int filesize = ecPolicy.getNumDataUnits() * ecPolicy.getCellSize();
@ -173,6 +175,10 @@ public class TestErasureCodingMultipleRacks {
final Path path = new Path("/testfile" + i);
LOG.info("Writing file " + path);
DFSTestUtil.writeFile(dfs, path, contents);
ExtendedBlock extendedBlock = DFSTestUtil.getFirstBlock(dfs, path);
// Wait for replication to finish before testing
DFSTestUtil.waitForReplication(cluster, extendedBlock, numRacks,
ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits(), 0);
BlockLocation[] blocks =
dfs.getFileBlockLocations(path, 0, Long.MAX_VALUE);
assertEquals(ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits(),