|
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
@@ -54,6 +55,8 @@ import org.mockito.stubbing.Answer;
|
|
|
public class TestPread {
|
|
|
static final long seed = 0xDEADBEEFL;
|
|
|
static final int blockSize = 4096;
|
|
|
+ static final int numBlocksPerFile = 12;
|
|
|
+ static final int fileSize = numBlocksPerFile * blockSize;
|
|
|
boolean simulatedStorage;
|
|
|
boolean isHedgedRead;
|
|
|
|
|
@@ -66,10 +69,10 @@ public class TestPread {
|
|
|
private void writeFile(FileSystem fileSys, Path name) throws IOException {
|
|
|
int replication = 3;// We need > 1 blocks to test out the hedged reads.
|
|
|
// test empty file open and read
|
|
|
- DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 0,
|
|
|
+ DFSTestUtil.createFile(fileSys, name, fileSize, 0,
|
|
|
blockSize, (short)replication, seed);
|
|
|
FSDataInputStream in = fileSys.open(name);
|
|
|
- byte[] buffer = new byte[12 * blockSize];
|
|
|
+ byte[] buffer = new byte[fileSize];
|
|
|
in.readFully(0, buffer, 0, 0);
|
|
|
IOException res = null;
|
|
|
try { // read beyond the end of the file
|
|
@@ -84,7 +87,7 @@ public class TestPread {
|
|
|
assertTrue("Cannot delete file", false);
|
|
|
|
|
|
// now create the real file
|
|
|
- DFSTestUtil.createFile(fileSys, name, 12 * blockSize, 12 * blockSize,
|
|
|
+ DFSTestUtil.createFile(fileSys, name, fileSize, fileSize,
|
|
|
blockSize, (short) replication, seed);
|
|
|
}
|
|
|
|
|
@@ -128,11 +131,13 @@ public class TestPread {
|
|
|
|
|
|
private void pReadFile(FileSystem fileSys, Path name) throws IOException {
|
|
|
FSDataInputStream stm = fileSys.open(name);
|
|
|
- byte[] expected = new byte[12 * blockSize];
|
|
|
+ byte[] expected = new byte[fileSize];
|
|
|
if (simulatedStorage) {
|
|
|
- for (int i= 0; i < expected.length; i++) {
|
|
|
- expected[i] = SimulatedFSDataset.DEFAULT_DATABYTE;
|
|
|
- }
|
|
|
+ assert fileSys instanceof DistributedFileSystem;
|
|
|
+ DistributedFileSystem dfs = (DistributedFileSystem) fileSys;
|
|
|
+ LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(name.toString(),
|
|
|
+ 0, fileSize);
|
|
|
+ DFSTestUtil.fillExpectedBuf(lbs, expected);
|
|
|
} else {
|
|
|
Random rand = new Random(seed);
|
|
|
rand.nextBytes(expected);
|
|
@@ -447,7 +452,7 @@ public class TestPread {
|
|
|
FileSystem fileSys = cluster.getFileSystem();
|
|
|
fileSys.setVerifyChecksum(verifyChecksum);
|
|
|
try {
|
|
|
- Path file1 = new Path("preadtest.dat");
|
|
|
+ Path file1 = new Path("/preadtest.dat");
|
|
|
writeFile(fileSys, file1);
|
|
|
pReadFile(fileSys, file1);
|
|
|
datanodeRestartTest(cluster, fileSys, file1);
|