You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by to...@apache.org on 2013/07/08 19:56:42 UTC

svn commit: r1500863 - in /hadoop/common/branches/branch-1: CHANGES.txt src/core/org/apache/hadoop/fs/BufferedFSInputStream.java src/test/org/apache/hadoop/fs/TestLocalFileSystem.java

Author: todd
Date: Mon Jul  8 17:56:42 2013
New Revision: 1500863

URL: http://svn.apache.org/r1500863
Log:
HADOOP-9307. BufferedFSInputStream.read returns wrong results after certain seeks. Contributed by Todd Lipcon.

Modified:
    hadoop/common/branches/branch-1/CHANGES.txt
    hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/BufferedFSInputStream.java
    hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java

Modified: hadoop/common/branches/branch-1/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/CHANGES.txt?rev=1500863&r1=1500862&r2=1500863&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/CHANGES.txt (original)
+++ hadoop/common/branches/branch-1/CHANGES.txt Mon Jul  8 17:56:42 2013
@@ -71,6 +71,9 @@ Release 1.3.0 - unreleased
     HADOOP-9678. TestRPC#testStopsAllThreads intermittently fails on Windows.
     (Ivan Mitic via cnauroth)
 
+    HADOOP-9307. BufferedFSInputStream.read returns wrong results after certain
+    seeks. (todd)
+
 Release 1.2.1 - 2013.07.06
 
   INCOMPATIBLE CHANGES

Modified: hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/BufferedFSInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/BufferedFSInputStream.java?rev=1500863&r1=1500862&r2=1500863&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/BufferedFSInputStream.java (original)
+++ hadoop/common/branches/branch-1/src/core/org/apache/hadoop/fs/BufferedFSInputStream.java Mon Jul  8 17:56:42 2013
@@ -63,12 +63,17 @@ implements Seekable, PositionedReadable,
     if( pos<0 ) {
       return;
     }
-    // optimize: check if the pos is in the buffer
-    long end = ((FSInputStream)in).getPos();
-    long start = end - count;
-    if( pos>=start && pos<end) {
-      this.pos = (int)(pos-start);
-      return;
+    if (this.pos != this.count) {
+      // optimize: check if the pos is in the buffer
+      // This optimization only works if pos != count -- if they are
+      // equal, it's possible that the previous reads were just
+      // longer than the total buffer size, and hence skipped the buffer.
+      long end = ((FSInputStream)in).getPos();
+      long start = end - count;
+      if( pos>=start && pos<end) {
+        this.pos = (int)(pos-start);
+        return;
+      }
     }
 
     // invalidate buffer

Modified: hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java?rev=1500863&r1=1500862&r2=1500863&view=diff
==============================================================================
--- hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java (original)
+++ hadoop/common/branches/branch-1/src/test/org/apache/hadoop/fs/TestLocalFileSystem.java Mon Jul  8 17:56:42 2013
@@ -18,7 +18,10 @@
 package org.apache.hadoop.fs;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.StringUtils;
 import java.io.*;
+import java.util.Arrays;
+import java.util.Random;
 
 import junit.framework.*;
 
@@ -28,6 +31,7 @@ import junit.framework.*;
 public class TestLocalFileSystem extends TestCase {
   private static String TEST_ROOT_DIR
     = System.getProperty("test.build.data","build/test/data/work-dir/localfs");
+  private final Path TEST_PATH = new Path(TEST_ROOT_DIR, "test-file");
 
 
   static void writeFile(FileSystem fs, Path name) throws IOException {
@@ -203,4 +207,72 @@ public class TestLocalFileSystem extends
         new RawLocalFileSystem().new LocalFSFileInputStream(path), 1024);
     assertNotNull(bis.getFileDescriptor());
   }
+
+  /**
+   * Regression test for HADOOP-9307: BufferedFSInputStream returning
+   * wrong results after certain sequences of seeks and reads.
+   */
+  public void testBufferedFSInputStream() throws IOException {
+    Configuration conf = new Configuration();
+    conf.setClass("fs.file.impl", RawLocalFileSystem.class, FileSystem.class);
+    conf.set("fs.file.impl.disable.cache", "true");
+    conf.setInt("io.file.buffer.size", 4096);
+    FileSystem fs = FileSystem.get(conf);
+    
+    byte[] buf = new byte[10*1024];
+    new Random().nextBytes(buf);
+    
+    // Write random bytes to file
+    FSDataOutputStream stream = fs.create(TEST_PATH);
+    try {
+      stream.write(buf);
+    } finally {
+      stream.close();
+    }
+    
+    Random r = new Random();
+
+    FSDataInputStream stm = fs.open(TEST_PATH);
+    // Record the sequence of seeks and reads which trigger a failure.
+    int seeks[] = new int[10];
+    int reads[] = new int[10];
+    try {
+      for (int i = 0; i < 1000; i++) {
+        int seekOff = r.nextInt(buf.length); 
+        int toRead = r.nextInt(Math.min(buf.length - seekOff, 32000));
+        
+        seeks[i % seeks.length] = seekOff;
+        reads[i % reads.length] = toRead;
+        verifyRead(stm, buf, seekOff, toRead);
+        
+      }
+    } catch (AssertionError afe) {
+      StringBuilder sb = new StringBuilder();
+      sb.append("Sequence of actions:\n");
+      for (int j = 0; j < seeks.length; j++) {
+        sb.append("seek @ ").append(seeks[j]).append("  ")
+          .append("read ").append(reads[j]).append("\n");
+      }
+      System.err.println(sb.toString());
+      throw afe;
+    } finally {
+      stm.close();
+    }
+  }
+  
+  private void verifyRead(FSDataInputStream stm, byte[] fileContents,
+       int seekOff, int toRead) throws IOException {
+    byte[] out = new byte[toRead];
+    stm.seek(seekOff);
+    stm.readFully(out);
+    byte[] expected = Arrays.copyOfRange(fileContents, seekOff, seekOff+toRead);
+    if (!Arrays.equals(out, expected)) {
+      String s ="\nExpected: " +
+          StringUtils.byteToHexString(expected) +
+          "\ngot:      " +
+          StringUtils.byteToHexString(out) + 
+          "\noff=" + seekOff + " len=" + toRead;
+      fail(s);
+    }
+  }
 }