[ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504908 ]
Hairong Kuang commented on HADOOP-1489: --------------------------------------- This is a good one and very subtle too. The problem is caused by using an external buffer for checksum verification. So FSInputChecker requires the amount of data read should be at least bytesPerCheccksum, which makes sure that no checksum verified data are read. We use Java's BufferedInputStream to implement the external buffer and guranteens that the buffer size is at least bytePerChecksum. However, in order to support mark() in BufferedInputStream, BufferedInputStream sometimes may expand its buffer size and issue read that asks for bytes which are less than bytesPerChecksum. Therefore, we get the problem described in this jira. The proposal in HADOOP-1470 is to verify checksum using an internal buffer. So the buffer size restriction will no longer be required. Hence a patch to HADOOP-1470 will also resolve this issue. An alternative quick fix is to overide markSupported() in FSDataInputStream.Buffer so that it returns false. > Input file get truncated for text files with \r\n > ------------------------------------------------- > > Key: HADOOP-1489 > URL: https://issues.apache.org/jira/browse/HADOOP-1489 > Project: Hadoop > Issue Type: Bug > Components: io > Affects Versions: 0.13.0 > Reporter: Bwolen Yang > Attachments: MRIdentity.java, slashr33.txt > > > When input file has \r\n, LineRecordReader uses mark()/reset() to read one > byte ahead to check if \r is followed by \n. This probably caused the > BufferedInputStream to issue a small read request (e.g., 127 bytes). The > ChecksumFileSystem.FSInputChecker.read() code > {code} > public int read(byte b[], int off, int len) throws IOException { > // make sure that it ends at a checksum boundary > long curPos = getPos(); > long endPos = len+curPos/bytesPerSum*bytesPerSum; > return readBuffer(b, off, (int)(endPos-curPos)); > } > {code} > tries to truncate "len" to checksum boundary. For DFS, bytesPerSum is 512. > So for small reads, the truncated length become negative (i.e., endPos - > curPos is < 0). The underlying DFS read returns 0 when length is negative. > However, readBuffer changes it to -1 assuming end-of-file has been reached. > This means effectively, the rest of the input file did not get read. In my > case, only 8MB of a 52MB file is actually read. Two sample stacks are > appended. > One related issue, if there are assumptions (such as len >= bytesPerSum) in > FSInputChecker's read(), would it be ok to add a check that throws an > exception when the assumption is violated? This assumption is a bit unusal > and as code changes (both Hadoop and Java's implementation of > BufferedInputStream), the assumption may get violated. This silently > dropping large part of input seems really difficult for people to notice (and > debug) when people start to deal with terabytes of data. Also, I suspect > the performance impact for such a check would not be noticed. > bwolen > Here are two sample stacks. (i have readbuffer throw when it gets 0 bytes, > and have inputchecker catches the exception and rethrow both. This way, I > catch the values from both caller and callee (see the callee one starts with > "Caused by") > ------------------------------------- > {code} > java.lang.RuntimeException: end of read() > in=org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker len=127 > pos=45223932 res=-999999 > at > org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:50) > at java.io.BufferedInputStream.fill(BufferedInputStream.java:218) > at java.io.BufferedInputStream.read(BufferedInputStream.java:237) > at > org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:116) > at java.io.FilterInputStream.read(FilterInputStream.java:66) > at > org.apache.hadoop.mapred.LineRecordReader.readLine(LineRecordReader.java:132) > at > org.apache.hadoop.mapred.LineRecordReader.readLine(LineRecordReader.java:124) > at > org.apache.hadoop.mapred.LineRecordReader.next(LineRecordReader.java:108) > at org.apache.hadoop.mapred.MapTask$1.next(MapTask.java:168) > at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:44) > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:186) > at > org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1720) > Caused by: java.lang.RuntimeException: end of read() > datas=org.apache.hadoop.dfs.DFSClient$DFSDataInputStream pos=45223932 > len=-381 bytesPerSum=512 eof=false read=0 > at > org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:200) > at > org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:175) > at > org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:47) > ... 11 more > --------------- > java.lang.RuntimeException: end of read() > in=org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker len=400 pos=4503 > res=-999999 > at > org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:50) > at java.io.BufferedInputStream.fill(BufferedInputStream.java:218) > at java.io.BufferedInputStream.read(BufferedInputStream.java:237) > at > org.apache.hadoop.fs.FSDataInputStream$Buffer.read(FSDataInputStream.java:116) > at java.io.FilterInputStream.read(FilterInputStream.java:66) > at > org.apache.hadoop.mapred.LineRecordReader.readLine(LineRecordReader.java:132) > at > org.apache.hadoop.mapred.LineRecordReader.readLine(LineRecordReader.java:124) > at > org.apache.hadoop.mapred.LineRecordReader.next(LineRecordReader.java:108) > at org.apache.hadoop.mapred.MapTask$1.next(MapTask.java:168) > at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:44) > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:186) > at > org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:1720) > Caused by: java.lang.RuntimeException: end of read() > datas=org.apache.hadoop.dfs.DFSClient$DFSDataInputStream pos=4503 len=-7 > bytesPerSum=512 eof=false read=0 > at > org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.readBuffer(ChecksumFileSystem.java:200) > at > org.apache.hadoop.fs.ChecksumFileSystem$FSInputChecker.read(ChecksumFileSystem.java:175) > at > org.apache.hadoop.fs.FSDataInputStream$PositionCache.read(FSDataInputStream.java:47) > ... 11 more > {code} -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.