From 6bc2f7f4b4b8d4c36e92764d4c975c17f9fdd63b Mon Sep 17 00:00:00 2001 From: Xiao Chen Date: Sun, 28 Jan 2018 22:11:08 -0800 Subject: [PATCH] HDFS-13065. TestErasureCodingMultipleRacks#testSkewedRack3 is failing. Contributed by Gabor Bota. --- .../hadoop/hdfs/TestErasureCodingMultipleRacks.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java index 0689665d98..3e8725316b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingMultipleRacks.java @@ -21,6 +21,7 @@ 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 void testSkewedRack3() throws Exception { // 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 void testSkewedRack3() throws Exception { 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(),