|
@@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertTrue;
|
|
import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
import java.io.DataOutputStream;
|
|
import java.io.DataOutputStream;
|
|
|
|
+import java.io.EOFException;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.Random;
|
|
import java.util.Random;
|
|
@@ -29,7 +30,7 @@ import java.util.concurrent.ExecutionException;
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.ExecutorService;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Executors;
|
|
import java.util.concurrent.Future;
|
|
import java.util.concurrent.Future;
|
|
-
|
|
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.ChecksumException;
|
|
import org.apache.hadoop.fs.ChecksumException;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
import org.apache.log4j.Level;
|
|
import org.apache.log4j.Level;
|
|
|
|
+import org.junit.Assert;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
import org.mockito.Mockito;
|
|
import org.mockito.Mockito;
|
|
@@ -494,6 +496,54 @@ public class TestPread {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testTruncateWhileReading() throws Exception {
|
|
|
|
+ Path path = new Path("/testfile");
|
|
|
|
+ final int blockSize = 512;
|
|
|
|
+
|
|
|
|
+ // prevent initial pre-fetch of multiple block locations
|
|
|
|
+ Configuration conf = new Configuration();
|
|
|
|
+ conf.setLong(HdfsClientConfigKeys.Read.PREFETCH_SIZE_KEY, blockSize);
|
|
|
|
+
|
|
|
|
+ MiniDFSCluster cluster =
|
|
|
|
+ new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
|
|
|
+ try {
|
|
|
|
+ DistributedFileSystem fs = cluster.getFileSystem();
|
|
|
|
+ // create multi-block file
|
|
|
|
+ FSDataOutputStream dos =
|
|
|
|
+ fs.create(path, true, blockSize, (short)1, blockSize);
|
|
|
|
+ dos.write(new byte[blockSize*3]);
|
|
|
|
+ dos.close();
|
|
|
|
+ // truncate a file while it's open
|
|
|
|
+ final FSDataInputStream dis = fs.open(path);
|
|
|
|
+ while (!fs.truncate(path, 10)) {
|
|
|
|
+ Thread.sleep(10);
|
|
|
|
+ }
|
|
|
|
+ // verify that reading bytes outside the initial pre-fetch do
|
|
|
|
+ // not send the client into an infinite loop querying locations.
|
|
|
|
+ ExecutorService executor = Executors.newFixedThreadPool(1);
|
|
|
|
+ Future<?> future = executor.submit(new Callable<Void>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Void call() throws IOException {
|
|
|
|
+ // read from 2nd block.
|
|
|
|
+ dis.readFully(blockSize, new byte[4]);
|
|
|
|
+ return null;
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
+ try {
|
|
|
|
+ future.get(4, TimeUnit.SECONDS);
|
|
|
|
+ Assert.fail();
|
|
|
|
+ } catch (ExecutionException ee) {
|
|
|
|
+ assertTrue(ee.toString(), ee.getCause() instanceof EOFException);
|
|
|
|
+ } finally {
|
|
|
|
+ future.cancel(true);
|
|
|
|
+ executor.shutdown();
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ cluster.shutdown();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
public static void main(String[] args) throws Exception {
|
|
public static void main(String[] args) throws Exception {
|
|
new TestPread().testPreadDFS();
|
|
new TestPread().testPreadDFS();
|
|
}
|
|
}
|