|
@@ -0,0 +1,104 @@
|
|
|
+package org.apache.hadoop.dfs;
|
|
|
+
|
|
|
+import javax.swing.filechooser.FileSystemView;
|
|
|
+import junit.framework.TestCase;
|
|
|
+import java.io.*;
|
|
|
+import java.util.Random;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
+import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
+import org.apache.hadoop.fs.FSInputStream;
|
|
|
+import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
+
|
|
|
+/**
|
|
|
+ * This class tests the presence of seek bug as described
|
|
|
+ * in HADOOP-508
|
|
|
+ * @author Milind Bhandarkar
|
|
|
+ */
|
|
|
+public class TestSeekBug extends TestCase {
|
|
|
+ static final long seed = 0xDEADBEEFL;
|
|
|
+ static final int ONEMB = 1 << 20;
|
|
|
+
|
|
|
+ private void writeFile(FileSystem fileSys, Path name) throws IOException {
|
|
|
+ // create and write a file that contains 1MB
|
|
|
+ DataOutputStream stm = fileSys.create(name);
|
|
|
+ byte[] buffer = new byte[ONEMB];
|
|
|
+ Random rand = new Random(seed);
|
|
|
+ rand.nextBytes(buffer);
|
|
|
+ stm.write(buffer);
|
|
|
+ stm.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void checkAndEraseData(byte[] actual, int from, byte[] expected, String message) {
|
|
|
+ for (int idx = 0; idx < actual.length; idx++) {
|
|
|
+ this.assertEquals(message+" byte "+(from+idx)+" differs. expected "+
|
|
|
+ expected[from+idx]+" actual "+actual[idx],
|
|
|
+ actual[idx], expected[from+idx]);
|
|
|
+ actual[idx] = 0;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private void seekReadFile(FileSystem fileSys, Path name) throws IOException {
|
|
|
+ FSInputStream stmRaw = fileSys.openRaw(name);
|
|
|
+ FSDataInputStream stm = new FSDataInputStream(stmRaw, 4096);
|
|
|
+ byte[] expected = new byte[ONEMB];
|
|
|
+ Random rand = new Random(seed);
|
|
|
+ rand.nextBytes(expected);
|
|
|
+
|
|
|
+ // First read 128 bytes to set count in BufferedInputStream
|
|
|
+ byte[] actual = new byte[128];
|
|
|
+ stm.read(actual, 0, actual.length);
|
|
|
+ // Now read a byte array that is bigger than the internal buffer
|
|
|
+ actual = new byte[100000];
|
|
|
+ stm.read(actual, 0, actual.length);
|
|
|
+ checkAndEraseData(actual, 128, expected, "First Read Test");
|
|
|
+ // now do a small seek, within the range that is already read
|
|
|
+ stm.seek(96036); // 4 byte seek
|
|
|
+ actual = new byte[128];
|
|
|
+ stm.read(actual, 0, actual.length);
|
|
|
+ checkAndEraseData(actual, 96036, expected, "Seek Bug");
|
|
|
+ // all done
|
|
|
+ stm.close();
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cleanupFile(FileSystem fileSys, Path name) throws IOException {
|
|
|
+ assertTrue(fileSys.exists(name));
|
|
|
+ fileSys.delete(name);
|
|
|
+ assertTrue(!fileSys.exists(name));
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test if the seek bug exists in FSDataInputStream in DFS.
|
|
|
+ */
|
|
|
+ public void testSeekBugDFS() throws IOException {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster(65312, conf, false);
|
|
|
+ FileSystem fileSys = cluster.getFileSystem();
|
|
|
+ try {
|
|
|
+ Path file1 = new Path("seektest.dat");
|
|
|
+ writeFile(fileSys, file1);
|
|
|
+ seekReadFile(fileSys, file1);
|
|
|
+ cleanupFile(fileSys, file1);
|
|
|
+ } finally {
|
|
|
+ fileSys.close();
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Tests if the seek bug exists in FSDataInputStream in LocalFS.
|
|
|
+ */
|
|
|
+ public void testSeekBugLocalFS() throws IOException {
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ FileSystem fileSys = FileSystem.getNamed("local", conf);
|
|
|
+ try {
|
|
|
+ Path file1 = new Path("build/test/data", "seektest.dat");
|
|
|
+ writeFile(fileSys, file1);
|
|
|
+ seekReadFile(fileSys, file1);
|
|
|
+ cleanupFile(fileSys, file1);
|
|
|
+ } finally {
|
|
|
+ fileSys.close();
|
|
|
+ }
|
|
|
+ }
|
|
|
+}
|