HDFS-13662. TestBlockReaderLocal#testStatisticsForErasureCodingRead is flaky

This commit is contained in:
Xiao Chen 2018-10-16 19:32:12 -07:00
parent e3342a1aba
commit 533138718c
1 changed files with 7 additions and 3 deletions

View File

@ -28,6 +28,7 @@ import java.nio.ByteBuffer;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.ChecksumException; import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
@ -827,9 +828,12 @@ public class TestBlockReaderLocal {
Path ecFile = new Path(ecDir, "file2"); Path ecFile = new Path(ecDir, "file2");
DFSTestUtil.createFile(fs, ecFile, length, repl, randomSeed); DFSTestUtil.createFile(fs, ecFile, length, repl, randomSeed);
// Shutdown one DataNode so that erasure coding decoding process can kick
// in. // Shutdown a DataNode that holds a data block, to trigger EC decoding.
cluster.shutdownDataNode(0); final BlockLocation[] locs = fs.getFileBlockLocations(ecFile, 0, length);
final String[] nodes = locs[0].getNames();
cluster.stopDataNode(nodes[0]);
try (HdfsDataInputStream in = (HdfsDataInputStream) fs.open(ecFile)) { try (HdfsDataInputStream in = (HdfsDataInputStream) fs.open(ecFile)) {
IOUtils.readFully(in, buf, 0, length); IOUtils.readFully(in, buf, 0, length);