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 cu...@apache.org on 2007/06/20 23:49:21 UTC

svn commit: r549257 - in /lucene/hadoop/trunk: CHANGES.txt src/java/org/apache/hadoop/fs/FSDataInputStream.java src/java/org/apache/hadoop/mapred/LineRecordReader.java src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java

Author: cutting
Date: Wed Jun 20 14:49:21 2007
New Revision: 549257

URL: http://svn.apache.org/viewvc?view=rev&rev=549257
Log:
HADOOP-1489.  Fix text input truncation bug due to mark/reset.  Contributed by Bwolen Yang.

Added:
    lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java
Modified:
    lucene/hadoop/trunk/CHANGES.txt
    lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
    lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LineRecordReader.java

Modified: lucene/hadoop/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=549257&r1=549256&r2=549257
==============================================================================
--- lucene/hadoop/trunk/CHANGES.txt (original)
+++ lucene/hadoop/trunk/CHANGES.txt Wed Jun 20 14:49:21 2007
@@ -217,6 +217,9 @@
      when opening files, reducing the namenode load somewhat.
      (Raghu Angadi via cutting)
 
+ 66. HADOOP-1489.  Fix text input truncation bug due to mark/reset.
+     Add a unittest. (Bwolen Yang via cutting)
+
 
 Release 0.13.0 - 2007-06-08
 

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java?view=diff&rev=549257&r1=549256&r2=549257
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FSDataInputStream.java Wed Jun 20 14:49:21 2007
@@ -103,6 +103,15 @@
       throws IOException {
       ((PositionCache)in).readFully(position, buffer, offset, length);
     }
+
+    // Disable marking, as its use can cause the BufferedInputStream superclass
+    // to read a smaller amount of data than bytesPerChecksum, which breaks
+    // ChecksumFileSystem.
+    public boolean markSupported() {
+      return false;
+    }
+
+    public synchronized void mark(int readlimit) {}
   }
 
   protected FSInputStream inStream;

Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LineRecordReader.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LineRecordReader.java?view=diff&rev=549257&r1=549256&r2=549257
==============================================================================
--- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LineRecordReader.java (original)
+++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/LineRecordReader.java Wed Jun 20 14:49:21 2007
@@ -56,16 +56,20 @@
     FileSystem fs = FileSystem.get(job);
     FSDataInputStream fileIn = fs.open(split.getPath());
     InputStream in = fileIn;
+    boolean skipFirstLine = false;
     if (codec != null) {
       in = codec.createInputStream(fileIn);
       end = Long.MAX_VALUE;
     } else if (start != 0) {
-      fileIn.seek(start - 1);
-      LineRecordReader.readLine(fileIn, null);
-      start = fileIn.getPos();
+      skipFirstLine = true;  // wait till BufferedInputStream to skip
+      --start;
+      fileIn.seek(start);
     }
 
     this.in = new BufferedInputStream(in);
+    if (skipFirstLine) {  // skip first line and re-establish "start".
+      start += LineRecordReader.readLine(this.in, null);
+    }
     this.start = start;
     this.pos = start;
     this.end = end;

Added: lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java
URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java?view=auto&rev=549257
==============================================================================
--- lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java (added)
+++ lucene/hadoop/trunk/src/test/org/apache/hadoop/fs/TestTruncatedInputBug.java Wed Jun 20 14:49:21 2007
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import junit.framework.TestCase;
+import java.io.*;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.util.StringUtils;
+
+/**
+ * test for the input truncation bug when mark/reset is used.
+ * HADOOP-1489
+ * @author Bwolen Yang
+ */
+public class TestTruncatedInputBug extends TestCase {
+  private static String TEST_ROOT_DIR =
+    new Path(System.getProperty("test.build.data","/tmp"))
+    .toString().replace(' ', '+');
+  
+  private void writeFile(FileSystem fileSys, 
+                         Path name, int nBytesToWrite) 
+    throws IOException {
+    DataOutputStream out = fileSys.create(name);
+    for (int i = 0; i < nBytesToWrite; ++i) {
+      out.writeByte(0);
+    }
+    out.close();
+  }
+  
+  /**
+   * When mark() is used on BufferedInputStream, the request
+   * size on the checksum file system can be small.  However,
+   * checksum file system currently depends on the request size
+   * >= bytesPerSum to work properly.
+   */
+  public void testTruncatedInputBug() throws IOException {
+    final int ioBufSize = 512;
+    final int fileSize = ioBufSize*4;
+    int filePos = 0;
+
+    Configuration conf = new Configuration();
+    conf.setInt("io.file.buffer.size", ioBufSize);
+    FileSystem fileSys = FileSystem.getLocal(conf);
+
+    try {
+      // First create a test input file.
+      Path testFile = new Path(TEST_ROOT_DIR, "HADOOP-1489");
+      writeFile(fileSys, testFile, fileSize);
+      assertTrue(fileSys.exists(testFile));
+      assertTrue(fileSys.getLength(testFile) == fileSize);
+
+      // Now read the file for ioBufSize bytes
+      FSDataInputStream in = fileSys.open(testFile, ioBufSize);
+      // seek beyond data buffered by open
+      filePos += ioBufSize * 2 + (ioBufSize - 10);  
+      in.seek(filePos);
+
+      // read 4 more bytes before marking
+      for (int i = 0; i < 4; ++i) {  
+        if (in.read() == -1) {
+          break;
+        }
+        ++filePos;
+      }
+
+      // Now set mark() to trigger the bug
+      // NOTE: in the fixed code, mark() does nothing (not supported) and
+      //   hence won't trigger this bug.
+      in.mark(1);
+      System.out.println("MARKED");
+      
+      // Try to read the rest
+      while (filePos < fileSize) {
+        if (in.read() == -1) {
+          break;
+        }
+        ++filePos;
+      }
+      in.close();
+
+      System.out.println("Read " + filePos + " bytes."
+                         + " file size=" + fileSize);
+      assertTrue(filePos == fileSize);
+
+    } finally {
+      try {
+        fileSys.close();
+      } catch (Exception e) {
+        // noop
+      }
+    }
+  }  // end testTruncatedInputBug
+}