You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Bwolen Yang (JIRA)" <ji...@apache.org> on 2007/06/13 20:24:25 UTC

[jira] Created: (HADOOP-1489) Input file get truncated for text files with \r\n

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



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 
   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));
   }

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.

-------------------------------------

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



-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504572 ] 

dhruba borthakur commented on HADOOP-1489:
------------------------------------------

So, can it be related to HADOOP-1491? In that case, distcp copies files with a buffer size specified by "copy.buf.size". It has a default value of 4K.


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504913 ] 

Doug Cutting commented on HADOOP-1489:
--------------------------------------

> An alternative quick fix is to overide markSupported() in FSDataInputStream.Buffer so that it returns false.

+1 I think we should do that in the short-term to fix this.


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Bwolen Yang updated HADOOP-1489:
--------------------------------

    Description: 
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}

  was:

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 
   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));
   }

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.

-------------------------------------

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




> 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
>
> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Hairong Kuang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504954 ] 

Hairong Kuang commented on HADOOP-1489:
---------------------------------------

+1 on the fix to the use of LineRecordReader.readLine. Shall we enforce that LineRecordReader.readLine to take BufferedInputStream as an input?

Bwolen, you are more than welcome to submit a patch.

> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504575 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

> So, can it be related to HADOOP-1491? In that case, distcp copies files with a buffer size specified by "copy.buf.size". It has a default value of 4K.

I don't think so. This bug only truncates and only occurs when user uses 'mark()' on inputstream. I don't hink distcp uses mark().  In the case of HADOOP-1491 one file (or block) seems to get overwritten.

> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505299 ] 

Doug Cutting commented on HADOOP-1489:
--------------------------------------

I like the idea of using the same logic as BufferedReader.  Can we simply maintain a flag instead of using mark() and reset()?


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506283 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

+1. Changes look good. I am not sure if implementing reset() is necessary though. Default implementation is supposed to check markSupported().


> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505013 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

{noformat}
bash-3.2$ printf "1 \n 2 \n 3 \r 4 \n" | wc
      3       4      14
{noformat}
But current our readLine would count 4 lines.


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506390 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

> original contract of reset 

Looks in in the entire java source code, only one function actually bothered to check for this...
{code}
   net/URLConnection.java
   static public String guessContentTypeFromStream(InputStream is)
                        throws IOException {
        // If we can't read ahead safely, just give up on guessing
        if (!is.markSupported())
{code}

Let's play it safe just in case people forgets...
thanks for the input.


> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Doug Cutting (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Doug Cutting updated HADOOP-1489:
---------------------------------

       Resolution: Fixed
    Fix Version/s: 0.14.0
           Status: Resolved  (was: Patch Available)

I just committed this.  Thanks, Bwolen!

> 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
>             Fix For: 0.14.0
>
>         Attachments: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504554 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

It exists in prev releases also.

> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505031 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------


BufferedReader came from java.io

{quote}
http://java.sun.com/j2se/1.3/docs/api/java/io/BufferedReader.html
{quote}

The semantics of treating \r without \n afterwards as a end of a line seems to be a correct semantics 
since \r is the carriage return character.   

I am only saying what BufferedReader implements basically is what Hadoop's LineRecordReader does
without using mark()/reset().   Had it's nextLine() allows writes to OutputStream, we could simplify 
LineRecordReader to use it instead of implement yet another line reader.



> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505023 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

BufferedReader.nextLine() will also treat \r alone (without \n afterwards) as a newline by itself.
It basically returns a line when it see a \r or a \n.
On the \r case, it sets a variable called skipLF so that on the next "nextLine()" call,
it will skip the next character if it is \n.



> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505559 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

Bwolen,
regd changes under hadoop/fs/, It will heavily conflict with HADOOP-1470. For hadoop/fs, would it be enough to implement markSupported()  and mark() as you did. 


> 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: HADOOP-1489.patch, 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Bwolen Yang updated HADOOP-1489:
--------------------------------

    Status: Patch Available  (was: Open)

marking for sumbit.

> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "dhruba borthakur (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504553 ] 

dhruba borthakur commented on HADOOP-1489:
------------------------------------------

Does this problem exist in previous release, e.g. 0.12.3? It will be good to know whether this was introduced in 0.13 release. Thanks.

> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Hudson (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506870 ] 

Hudson commented on HADOOP-1489:
--------------------------------

Integrated in Hadoop-Nightly #131 (See [http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/131/])

> 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
>             Fix For: 0.14.0
>
>         Attachments: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


Re: Opening HRegionServer/HMaster/HClient for extension

Posted by James Kennedy <ja...@troove.net>.
Hi Michael,

I'll create a Jira task and fix the patch spacing.  I can't really talk 
too much about the HRegionServer/HClient extension i'm developing but I 
do think that there could be a general purpose need.  For example, HBase 
let's you filter a scan by rowkey and column key. But what about actual 
data values?  An extension could be an HRegionServer with a scanner that 
can filters rows by column values given some WHERE criteria.  Or maybe 
that's a bad example cause that should be built directly into HBase?   
Another would be implementing distributed joins between tables...

I havn't had a chance to re-profile yet. I'd modified the HBase code so 
I could extend and so part of the motivation of this patch was so that I 
could revert, update, add the patch you suggested, and then re-apply the 
extension patch.

I've done that and will hopefully get back to profiling this afternoon.

Thanks,
James

Michael Stack wrote:
> The patch looks like an improvement to me.  Whats the rationale for 
> needing to extend client/server?  Do you think it of general 
> applicability?
> I'd suggest making an issue and attaching a patch (file against hbase 
> component and it looks like your tabs are not the hadoop two spaces 
> convention going by the below).  We can continue discussion therein.  
> I offer to vote for it after review and trying it local.
>
> St.Ack
> P.S. Did HADOOP-1498, applied yesterday, change the profiling 
> characteristics you wrote about a few days ago?
>
>
> James Kennedy wrote:
>> For what i'm doing I found it necessary to extend 
>> HRegionServer/HRegion/HClient for some custom functionality.
>>
>> Following good Java practice I see that the HBase code as been 
>> programmed defensively, keeping stuff private as much as possible.
>>
>> However, for extensibility it would be nice if the servers/client 
>> were easy to extend.
>>
>> Attached is a patch that makes several methods protected instead of 
>> private, adds getters to fields of inner classes, and some other 
>> modifications i found were useful for some simple extension code.
>>
>> I didn't make this a Jira task because I wasn't sure if you guys 
>> approved of opening up the code like this but hopefully someone will 
>> find this useful.
>>
>> - James K
>> ------------------------------------------------------------------------
>>
>> Index: 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java 
>>
>> ===================================================================
>> --- 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java    
>> (revision 549130)
>> +++ 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java    
>> (working copy)
>> @@ -62,7 +62,7 @@
>>    /*
>>     * Data structure that holds current location for a region and its 
>> info.
>>     */
>> -  static class RegionLocation {
>> +  protected static class RegionLocation {
>>      HRegionInfo regionInfo;
>>      HServerAddress serverAddress;
>>  
>> @@ -76,6 +76,22 @@
>>        return "address: " + this.serverAddress.toString() + ", 
>> regioninfo: " +
>>          this.regionInfo;
>>      }
>> +
>> +    public HRegionInfo getRegionInfo() {
>> +        return regionInfo;
>> +    }
>> +
>> +    public void setRegionInfo(HRegionInfo regionInfo) {
>> +        this.regionInfo = regionInfo;
>> +    }
>> +
>> +    public HServerAddress getServerAddress() {
>> +        return serverAddress;
>> +    }
>> +
>> +    public void setServerAddress(HServerAddress serverAddress) {
>> +        this.serverAddress = serverAddress;
>> +    }
>>    }
>>       // Map tableName -> (Map startRow -> (HRegionInfo, HServerAddress)
>> @@ -116,7 +132,7 @@
>>      this.rand = new Random();
>>    }
>>    -  private void handleRemoteException(RemoteException e) throws 
>> IOException {
>> +  protected void handleRemoteException(RemoteException e) throws 
>> IOException {
>>      String msg = e.getMessage();
>>      
>> if(e.getClassName().equals("org.apache.hadoop.hbase.InvalidColumnNameException")) 
>> {
>>        throw new InvalidColumnNameException(msg);
>> @@ -143,7 +159,7 @@
>>       /* Find the address of the master and connect to it
>>     */
>> -  private void checkMaster() throws MasterNotRunningException {
>> +  protected void checkMaster() throws MasterNotRunningException {
>>      if (this.master != null) {
>>        return;
>>      }
>> @@ -531,7 +547,7 @@
>>     * @param tableName - the table name to be checked
>>     * @throws IllegalArgumentException - if the table name is reserved
>>     */
>> -  private void checkReservedTableName(Text tableName) {
>> +  protected void checkReservedTableName(Text tableName) {
>>      if(tableName.equals(ROOT_TABLE_NAME)
>>          || tableName.equals(META_TABLE_NAME)) {
>>        @@ -547,7 +563,7 @@
>>    
>> ////////////////////////////////////////////////////////////////////////////// 
>>
>>    // Client API
>>    
>> ////////////////////////////////////////////////////////////////////////////// 
>>
>> -
>> +     /**
>>     * Loads information so that a table can be manipulated.
>>     * @@ -558,8 +574,21 @@
>>      if(tableName == null || tableName.getLength() == 0) {
>>        throw new IllegalArgumentException("table name cannot be null 
>> or zero length");
>>      }
>> -    this.tableServers = tablesToServers.get(tableName);
>> -    if (this.tableServers == null ) {
>> +    this.tableServers = getTableServers(tableName);
>> +  }
>> +  +  /**
>> +   * Gets the servers of the given table.
>> +   * +   * @param tableName - the table to be located
>> +   * @throws IOException - if the table can not be located after 
>> retrying
>> +   */
>> +  protected synchronized SortedMap<Text, RegionLocation> 
>> getTableServers(Text tableName) throws IOException {
>> +    if(tableName == null || tableName.getLength() == 0) {
>> +      throw new IllegalArgumentException("table name cannot be null 
>> or zero length");
>> +    }
>> +    SortedMap<Text, RegionLocation> serverResult  = 
>> tablesToServers.get(tableName);
>> +    if (serverResult == null ) {
>>        if (LOG.isDebugEnabled()) {
>>          LOG.debug("No servers for " + tableName + ". Doing a find...");
>>        }
>> @@ -565,8 +594,9 @@
>>        }
>>        // We don't know where the table is.
>>        // Load the information from meta.
>> -      this.tableServers = findServersForTable(tableName);
>> +      serverResult = findServersForTable(tableName);
>>      }
>> +    return serverResult;
>>    }
>>  
>>    /*
>> @@ -836,7 +866,7 @@
>>     * @param regionServer - the server to connect to
>>     * @throws IOException
>>     */
>> -  synchronized HRegionInterface getHRegionConnection(HServerAddress 
>> regionServer)
>> +  protected synchronized HRegionInterface 
>> getHRegionConnection(HServerAddress regionServer)
>>        throws IOException {
>>  
>>        // See if we already have a connection
>> @@ -916,7 +946,7 @@
>>     * @param row Row to find.
>>     * @return Location of row.
>>     */
>> -  synchronized RegionLocation getRegionLocation(Text row) {
>> +  protected synchronized RegionLocation getRegionLocation(Text row) {
>>      if(row == null || row.getLength() == 0) {
>>        throw new IllegalArgumentException("row key cannot be null or 
>> zero length");
>>      }
>> @@ -1554,6 +1584,20 @@
>>      }
>>           return errCode;
>> +  }  +
>> +  /**
>> +   * @return the map of opened servers
>> +   */
>> +  protected TreeMap<String, HRegionInterface> getOpenServers(){
>> +    return servers;
>> +  }
>> +
>> +  /**
>> +   * @return the configuration for this server
>> +   */
>> +  public Configuration getConf(){
>> +    return conf;
>>    }
>>       /**
>> @@ -1565,4 +1609,5 @@
>>      int errCode = (new HClient(c)).doCommandLine(args);
>>      System.exit(errCode);
>>    }
>> +
>>  }
>> Index: 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java 
>>
>> ===================================================================
>> --- 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java    
>> (revision 549130)
>> +++ 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java    
>> (working copy)
>> @@ -55,7 +55,7 @@
>>   * regionName is a unique identifier for this HRegion. (startKey, 
>> endKey]
>>   * defines the keyspace for this HRegion.
>>   */
>> -class HRegion implements HConstants {
>> +public class HRegion implements HConstants {
>>    static String SPLITDIR = "splits";
>>    static String MERGEDIR = "merges";
>>    static String TMPREGION_PREFIX = "tmpregion_";
>> @@ -298,7 +298,7 @@
>>     *     * @throws IOException
>>     */
>> -  HRegion(Path rootDir, HLog log, FileSystem fs, Configuration conf, 
>> +  public HRegion(Path rootDir, HLog log, FileSystem fs, 
>> Configuration conf,        HRegionInfo regionInfo, Path initialFiles)
>>    throws IOException {
>>      @@ -386,7 +386,7 @@
>>     * This method could take some time to execute, so don't call it 
>> from a     * time-sensitive thread.
>>     */
>> -  Vector<HStoreFile> close() throws IOException {
>> +  public Vector<HStoreFile> close() throws IOException {
>>      lock.obtainWriteLock();
>>      try {
>>        boolean shouldClose = false;
>> @@ -548,43 +548,43 @@
>>    // HRegion accessors
>>    
>> ////////////////////////////////////////////////////////////////////////////// 
>>
>>  
>> -  Text getStartKey() {
>> +  public Text getStartKey() {
>>      return regionInfo.startKey;
>>    }
>>    -  Text getEndKey() {
>> +  public Text getEndKey() {
>>      return regionInfo.endKey;
>>    }
>>    -  long getRegionId() {
>> +  public long getRegionId() {
>>      return regionInfo.regionId;
>>    }
>>  
>> -  Text getRegionName() {
>> +  public Text getRegionName() {
>>      return regionInfo.regionName;
>>    }
>>    -  Path getRootDir() {
>> +  public Path getRootDir() {
>>      return rootDir;
>>    }
>>   -  HTableDescriptor getTableDesc() {
>> +  public HTableDescriptor getTableDesc() {
>>      return regionInfo.tableDesc;
>>    }
>>    -  HLog getLog() {
>> +  public HLog getLog() {
>>      return log;
>>    }
>>    -  Configuration getConf() {
>> +  public Configuration getConf() {
>>      return conf;
>>    }
>>    -  Path getRegionDir() {
>> +  public Path getRegionDir() {
>>      return regiondir;
>>    }
>>    -  FileSystem getFilesystem() {
>> +  public FileSystem getFilesystem() {
>>      return fs;
>>    }
>>  
>> @@ -973,7 +973,7 @@
>>     * Return an iterator that scans over the HRegion, returning the 
>> indicated     * columns.  This Iterator must be closed by the caller.
>>     */
>> -  HInternalScannerInterface getScanner(Text[] cols, Text firstRow)
>> +  public HInternalScannerInterface getScanner(Text[] cols, Text 
>> firstRow)
>>    throws IOException {
>>      lock.obtainReadLock();
>>      try {
>> @@ -1011,7 +1011,7 @@
>>     * @return lockid
>>     * @see #put(long, Text, BytesWritable)
>>     */
>> -  long startUpdate(Text row) throws IOException {
>> +  public long startUpdate(Text row) throws IOException {
>>      // We obtain a per-row lock, so other clients will block while 
>> one client
>>      // performs an update.  The read lock is released by the client 
>> calling
>>      // #commit or #abort or if the HRegionServer lease on the lock 
>> expires.
>> @@ -1029,7 +1029,7 @@
>>     * This method really just tests the input, then calls an internal 
>> localput()     * method.
>>     */
>> -  void put(long lockid, Text targetCol, byte [] val) throws 
>> IOException {
>> +  public void put(long lockid, Text targetCol, byte [] val) throws 
>> IOException {
>>      if (DELETE_BYTES.compareTo(val) == 0) {
>>        throw new IOException("Cannot insert value: " + val);
>>      }
>> @@ -1039,7 +1039,7 @@
>>    /**
>>     * Delete a value or write a value. This is a just a convenience 
>> method for put().
>>     */
>> -  void delete(long lockid, Text targetCol) throws IOException {
>> +  public void delete(long lockid, Text targetCol) throws IOException {
>>      localput(lockid, targetCol, DELETE_BYTES.get());
>>    }
>>  
>> @@ -1055,7 +1055,7 @@
>>     * @param val Value to enter into cell
>>     * @throws IOException
>>     */
>> -  void localput(final long lockid, final Text targetCol,
>> +  public void localput(final long lockid, final Text targetCol,
>>      final byte [] val)
>>    throws IOException {
>>      checkColumn(targetCol);
>> @@ -1090,7 +1090,7 @@
>>     * writes associated with the given row-lock.  These values have 
>> not yet
>>     * been placed in memcache or written to the log.
>>     */
>> -  void abort(long lockid) throws IOException {
>> +  public void abort(long lockid) throws IOException {
>>      Text row = getRowFromLock(lockid);
>>      if(row == null) {
>>        throw new LockException("No write lock for lockid " + lockid);
>> @@ -1124,7 +1124,7 @@
>>     * @param lockid Lock for row we're to commit.
>>     * @throws IOException
>>     */
>> -  void commit(final long lockid) throws IOException {
>> +  public void commit(final long lockid) throws IOException {
>>      // Remove the row from the pendingWrites list so      // that 
>> repeated executions won't screw this up.
>>      Text row = getRowFromLock(lockid);
>> Index: 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java 
>>
>> ===================================================================
>> --- 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java    
>> (revision 549130)
>> +++ 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java    
>> (working copy)
>> @@ -139,6 +139,76 @@
>>      this.regionName.readFields(in);
>>      this.offLine = in.readBoolean();
>>    }
>> +  +  /**
>> +   * @return the endKey
>> +   */
>> +  public Text getEndKey(){
>> +    return endKey;
>> +  }
>> +
>> +  /**
>> +   * @param endKey the endKey to set
>> +   */
>> +  public void setEndKey(Text endKey){
>> +    this.endKey = endKey;
>> +  }
>> +
>> +  /**
>> +   * @return the regionId
>> +   */
>> +  public long getRegionId(){
>> +    return regionId;
>> +  }
>> +
>> +  /**
>> +   * @param regionId the regionId to set
>> +   */
>> +  public void setRegionId(long regionId){
>> +    this.regionId = regionId;
>> +  }
>> +
>> +  /**
>> +   * @return the regionName
>> +   */
>> +  public Text getRegionName(){
>> +    return regionName;
>> +  }
>> +
>> +  /**
>> +   * @param regionName the regionName to set
>> +   */
>> +  public void setRegionName(Text regionName){
>> +    this.regionName = regionName;
>> +  }
>> +
>> +  /**
>> +   * @return the startKey
>> +   */
>> +  public Text getStartKey(){
>> +    return startKey;
>> +  }
>> +
>> +  /**
>> +   * @param startKey the startKey to set
>> +   */
>> +  public void setStartKey(Text startKey){
>> +    this.startKey = startKey;
>> +  }
>> +
>> +  /**
>> +   * @return the tableDesc
>> +   */
>> +  public HTableDescriptor getTableDesc(){
>> +    return tableDesc;
>> +  }
>> +
>> +  /**
>> +   * @param tableDesc the tableDesc to set
>> +   */
>> +  public void setTableDesc(HTableDescriptor tableDesc){
>> +    this.tableDesc = tableDesc;
>> +  }
>>  
>>    
>> ////////////////////////////////////////////////////////////////////////////// 
>>
>>    // Comparable
>> @@ -162,4 +232,6 @@
>>      // Compare end keys.
>>      return this.endKey.compareTo(other.endKey);
>>    }
>> +
>> +  }
>> Index: 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java 
>>
>> ===================================================================
>> --- 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java    
>> (revision 549130)
>> +++ 
>> /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java    
>> (working copy)
>> @@ -468,7 +468,7 @@
>>     * Sets a flag that will cause all the HRegionServer threads to 
>> shut down
>>     * in an orderly fashion.
>>     */
>> -  synchronized void stop() {
>> +  public synchronized void stop() {
>>      stopRequested = true;
>>      notifyAll();                        // Wakes run() if it is 
>> sleeping
>>    }
>> @@ -1079,7 +1079,7 @@
>>    }
>>  
>>    /** -   * Private utility method for safely obtaining an HRegion 
>> handle.
>> +   * Protected utility method for safely obtaining an HRegion handle.
>>     * @param regionName Name of online {@link HRegion} to return
>>     * @return {@link HRegion} for <code>regionName</code>
>>     * @throws NotServingRegionException
>> @@ -1084,7 +1084,7 @@
>>     * @return {@link HRegion} for <code>regionName</code>
>>     * @throws NotServingRegionException
>>     */
>> -  private HRegion getRegion(final Text regionName)
>> +  protected HRegion getRegion(final Text regionName)
>>    throws NotServingRegionException {
>>      return getRegion(regionName, false);
>>    }
>> @@ -1090,7 +1090,7 @@
>>    }
>>       /** -   * Private utility method for safely obtaining an 
>> HRegion handle.
>> +   * Protected utility method for safely obtaining an HRegion handle.
>>     * @param regionName Name of online {@link HRegion} to return
>>     * @param checkRetiringRegions Set true if we're to check retiring 
>> regions
>>     * as well as online regions.
>> @@ -1097,7 +1097,7 @@
>>     * @return {@link HRegion} for <code>regionName</code>
>>     * @throws NotServingRegionException
>>     */
>> -  private HRegion getRegion(final Text regionName,
>> +  protected HRegion getRegion(final Text regionName,
>>        final boolean checkRetiringRegions)
>>    throws NotServingRegionException {
>>      HRegion region = null;
>>   
>


Re: Opening HRegionServer/HMaster/HClient for extension

Posted by Michael Stack <st...@duboce.net>.
The patch looks like an improvement to me.  Whats the rationale for 
needing to extend client/server?  Do you think it of general 
applicability? 

I'd suggest making an issue and attaching a patch (file against hbase 
component and it looks like your tabs are not the hadoop two spaces 
convention going by the below).  We can continue discussion therein.  I 
offer to vote for it after review and trying it local.

St.Ack
P.S. Did HADOOP-1498, applied yesterday, change the profiling 
characteristics you wrote about a few days ago?


James Kennedy wrote:
> For what i'm doing I found it necessary to extend 
> HRegionServer/HRegion/HClient for some custom functionality.
>
> Following good Java practice I see that the HBase code as been 
> programmed defensively, keeping stuff private as much as possible.
>
> However, for extensibility it would be nice if the servers/client were 
> easy to extend.
>
> Attached is a patch that makes several methods protected instead of 
> private, adds getters to fields of inner classes, and some other 
> modifications i found were useful for some simple extension code.
>
> I didn't make this a Jira task because I wasn't sure if you guys 
> approved of opening up the code like this but hopefully someone will 
> find this useful.
>
> - James K
> ------------------------------------------------------------------------
>
> Index: /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java
> ===================================================================
> --- /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java	(revision 549130)
> +++ /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HClient.java	(working copy)
> @@ -62,7 +62,7 @@
>    /*
>     * Data structure that holds current location for a region and its info.
>     */
> -  static class RegionLocation {
> +  protected static class RegionLocation {
>      HRegionInfo regionInfo;
>      HServerAddress serverAddress;
>  
> @@ -76,6 +76,22 @@
>        return "address: " + this.serverAddress.toString() + ", regioninfo: " +
>          this.regionInfo;
>      }
> +
> +	public HRegionInfo getRegionInfo() {
> +		return regionInfo;
> +	}
> +
> +	public void setRegionInfo(HRegionInfo regionInfo) {
> +		this.regionInfo = regionInfo;
> +	}
> +
> +	public HServerAddress getServerAddress() {
> +		return serverAddress;
> +	}
> +
> +	public void setServerAddress(HServerAddress serverAddress) {
> +		this.serverAddress = serverAddress;
> +	}
>    }
>    
>    // Map tableName -> (Map startRow -> (HRegionInfo, HServerAddress)
> @@ -116,7 +132,7 @@
>      this.rand = new Random();
>    }
>    
> -  private void handleRemoteException(RemoteException e) throws IOException {
> +  protected void handleRemoteException(RemoteException e) throws IOException {
>      String msg = e.getMessage();
>      if(e.getClassName().equals("org.apache.hadoop.hbase.InvalidColumnNameException")) {
>        throw new InvalidColumnNameException(msg);
> @@ -143,7 +159,7 @@
>    
>    /* Find the address of the master and connect to it
>     */
> -  private void checkMaster() throws MasterNotRunningException {
> +  protected void checkMaster() throws MasterNotRunningException {
>      if (this.master != null) {
>        return;
>      }
> @@ -531,7 +547,7 @@
>     * @param tableName - the table name to be checked
>     * @throws IllegalArgumentException - if the table name is reserved
>     */
> -  private void checkReservedTableName(Text tableName) {
> +  protected void checkReservedTableName(Text tableName) {
>      if(tableName.equals(ROOT_TABLE_NAME)
>          || tableName.equals(META_TABLE_NAME)) {
>        
> @@ -547,7 +563,7 @@
>    //////////////////////////////////////////////////////////////////////////////
>    // Client API
>    //////////////////////////////////////////////////////////////////////////////
> -
> +  
>    /**
>     * Loads information so that a table can be manipulated.
>     * 
> @@ -558,8 +574,21 @@
>      if(tableName == null || tableName.getLength() == 0) {
>        throw new IllegalArgumentException("table name cannot be null or zero length");
>      }
> -    this.tableServers = tablesToServers.get(tableName);
> -    if (this.tableServers == null ) {
> +    this.tableServers = getTableServers(tableName);
> +  }
> +  
> +  /**
> +   * Gets the servers of the given table.
> +   * 
> +   * @param tableName - the table to be located
> +   * @throws IOException - if the table can not be located after retrying
> +   */
> +  protected synchronized SortedMap<Text, RegionLocation> getTableServers(Text tableName) throws IOException {
> +    if(tableName == null || tableName.getLength() == 0) {
> +      throw new IllegalArgumentException("table name cannot be null or zero length");
> +    }
> +    SortedMap<Text, RegionLocation> serverResult  = tablesToServers.get(tableName);
> +    if (serverResult == null ) {
>        if (LOG.isDebugEnabled()) {
>          LOG.debug("No servers for " + tableName + ". Doing a find...");
>        }
> @@ -565,8 +594,9 @@
>        }
>        // We don't know where the table is.
>        // Load the information from meta.
> -      this.tableServers = findServersForTable(tableName);
> +      serverResult = findServersForTable(tableName);
>      }
> +    return serverResult;
>    }
>  
>    /*
> @@ -836,7 +866,7 @@
>     * @param regionServer - the server to connect to
>     * @throws IOException
>     */
> -  synchronized HRegionInterface getHRegionConnection(HServerAddress regionServer)
> +  protected synchronized HRegionInterface getHRegionConnection(HServerAddress regionServer)
>        throws IOException {
>  
>        // See if we already have a connection
> @@ -916,7 +946,7 @@
>     * @param row Row to find.
>     * @return Location of row.
>     */
> -  synchronized RegionLocation getRegionLocation(Text row) {
> +  protected synchronized RegionLocation getRegionLocation(Text row) {
>      if(row == null || row.getLength() == 0) {
>        throw new IllegalArgumentException("row key cannot be null or zero length");
>      }
> @@ -1554,6 +1584,20 @@
>      }
>      
>      return errCode;
> +  }  
> +
> +  /**
> +   * @return the map of opened servers
> +   */
> +  protected TreeMap<String, HRegionInterface> getOpenServers(){
> +    return servers;
> +  }
> +
> +  /**
> +   * @return the configuration for this server
> +   */
> +  public Configuration getConf(){
> +    return conf;
>    }
>    
>    /**
> @@ -1565,4 +1609,5 @@
>      int errCode = (new HClient(c)).doCommandLine(args);
>      System.exit(errCode);
>    }
> +
>  }
> Index: /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java
> ===================================================================
> --- /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java	(revision 549130)
> +++ /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegion.java	(working copy)
> @@ -55,7 +55,7 @@
>   * regionName is a unique identifier for this HRegion. (startKey, endKey]
>   * defines the keyspace for this HRegion.
>   */
> -class HRegion implements HConstants {
> +public class HRegion implements HConstants {
>    static String SPLITDIR = "splits";
>    static String MERGEDIR = "merges";
>    static String TMPREGION_PREFIX = "tmpregion_";
> @@ -298,7 +298,7 @@
>     * 
>     * @throws IOException
>     */
> -  HRegion(Path rootDir, HLog log, FileSystem fs, Configuration conf, 
> +  public HRegion(Path rootDir, HLog log, FileSystem fs, Configuration conf, 
>        HRegionInfo regionInfo, Path initialFiles)
>    throws IOException {
>      
> @@ -386,7 +386,7 @@
>     * This method could take some time to execute, so don't call it from a 
>     * time-sensitive thread.
>     */
> -  Vector<HStoreFile> close() throws IOException {
> +  public Vector<HStoreFile> close() throws IOException {
>      lock.obtainWriteLock();
>      try {
>        boolean shouldClose = false;
> @@ -548,43 +548,43 @@
>    // HRegion accessors
>    //////////////////////////////////////////////////////////////////////////////
>  
> -  Text getStartKey() {
> +  public Text getStartKey() {
>      return regionInfo.startKey;
>    }
>    
> -  Text getEndKey() {
> +  public Text getEndKey() {
>      return regionInfo.endKey;
>    }
>    
> -  long getRegionId() {
> +  public long getRegionId() {
>      return regionInfo.regionId;
>    }
>  
> -  Text getRegionName() {
> +  public Text getRegionName() {
>      return regionInfo.regionName;
>    }
>    
> -  Path getRootDir() {
> +  public Path getRootDir() {
>      return rootDir;
>    }
>   
> -  HTableDescriptor getTableDesc() {
> +  public HTableDescriptor getTableDesc() {
>      return regionInfo.tableDesc;
>    }
>    
> -  HLog getLog() {
> +  public HLog getLog() {
>      return log;
>    }
>    
> -  Configuration getConf() {
> +  public Configuration getConf() {
>      return conf;
>    }
>    
> -  Path getRegionDir() {
> +  public Path getRegionDir() {
>      return regiondir;
>    }
>    
> -  FileSystem getFilesystem() {
> +  public FileSystem getFilesystem() {
>      return fs;
>    }
>  
> @@ -973,7 +973,7 @@
>     * Return an iterator that scans over the HRegion, returning the indicated 
>     * columns.  This Iterator must be closed by the caller.
>     */
> -  HInternalScannerInterface getScanner(Text[] cols, Text firstRow)
> +  public HInternalScannerInterface getScanner(Text[] cols, Text firstRow)
>    throws IOException {
>      lock.obtainReadLock();
>      try {
> @@ -1011,7 +1011,7 @@
>     * @return lockid
>     * @see #put(long, Text, BytesWritable)
>     */
> -  long startUpdate(Text row) throws IOException {
> +  public long startUpdate(Text row) throws IOException {
>      // We obtain a per-row lock, so other clients will block while one client
>      // performs an update.  The read lock is released by the client calling
>      // #commit or #abort or if the HRegionServer lease on the lock expires.
> @@ -1029,7 +1029,7 @@
>     * This method really just tests the input, then calls an internal localput() 
>     * method.
>     */
> -  void put(long lockid, Text targetCol, byte [] val) throws IOException {
> +  public void put(long lockid, Text targetCol, byte [] val) throws IOException {
>      if (DELETE_BYTES.compareTo(val) == 0) {
>        throw new IOException("Cannot insert value: " + val);
>      }
> @@ -1039,7 +1039,7 @@
>    /**
>     * Delete a value or write a value. This is a just a convenience method for put().
>     */
> -  void delete(long lockid, Text targetCol) throws IOException {
> +  public void delete(long lockid, Text targetCol) throws IOException {
>      localput(lockid, targetCol, DELETE_BYTES.get());
>    }
>  
> @@ -1055,7 +1055,7 @@
>     * @param val Value to enter into cell
>     * @throws IOException
>     */
> -  void localput(final long lockid, final Text targetCol,
> +  public void localput(final long lockid, final Text targetCol,
>      final byte [] val)
>    throws IOException {
>      checkColumn(targetCol);
> @@ -1090,7 +1090,7 @@
>     * writes associated with the given row-lock.  These values have not yet
>     * been placed in memcache or written to the log.
>     */
> -  void abort(long lockid) throws IOException {
> +  public void abort(long lockid) throws IOException {
>      Text row = getRowFromLock(lockid);
>      if(row == null) {
>        throw new LockException("No write lock for lockid " + lockid);
> @@ -1124,7 +1124,7 @@
>     * @param lockid Lock for row we're to commit.
>     * @throws IOException
>     */
> -  void commit(final long lockid) throws IOException {
> +  public void commit(final long lockid) throws IOException {
>      // Remove the row from the pendingWrites list so 
>      // that repeated executions won't screw this up.
>      Text row = getRowFromLock(lockid);
> Index: /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java
> ===================================================================
> --- /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java	(revision 549130)
> +++ /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionInfo.java	(working copy)
> @@ -139,6 +139,76 @@
>      this.regionName.readFields(in);
>      this.offLine = in.readBoolean();
>    }
> +  
> +  /**
> +   * @return the endKey
> +   */
> +  public Text getEndKey(){
> +    return endKey;
> +  }
> +
> +  /**
> +   * @param endKey the endKey to set
> +   */
> +  public void setEndKey(Text endKey){
> +    this.endKey = endKey;
> +  }
> +
> +  /**
> +   * @return the regionId
> +   */
> +  public long getRegionId(){
> +    return regionId;
> +  }
> +
> +  /**
> +   * @param regionId the regionId to set
> +   */
> +  public void setRegionId(long regionId){
> +    this.regionId = regionId;
> +  }
> +
> +  /**
> +   * @return the regionName
> +   */
> +  public Text getRegionName(){
> +    return regionName;
> +  }
> +
> +  /**
> +   * @param regionName the regionName to set
> +   */
> +  public void setRegionName(Text regionName){
> +    this.regionName = regionName;
> +  }
> +
> +  /**
> +   * @return the startKey
> +   */
> +  public Text getStartKey(){
> +    return startKey;
> +  }
> +
> +  /**
> +   * @param startKey the startKey to set
> +   */
> +  public void setStartKey(Text startKey){
> +    this.startKey = startKey;
> +  }
> +
> +  /**
> +   * @return the tableDesc
> +   */
> +  public HTableDescriptor getTableDesc(){
> +    return tableDesc;
> +  }
> +
> +  /**
> +   * @param tableDesc the tableDesc to set
> +   */
> +  public void setTableDesc(HTableDescriptor tableDesc){
> +    this.tableDesc = tableDesc;
> +  }
>  
>    //////////////////////////////////////////////////////////////////////////////
>    // Comparable
> @@ -162,4 +232,6 @@
>      // Compare end keys.
>      return this.endKey.compareTo(other.endKey);
>    }
> +
> + 
>  }
> Index: /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java
> ===================================================================
> --- /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java	(revision 549130)
> +++ /opt/eclipse/sandbox2/Hadoop/src/contrib/hbase/src/java/org/apache/hadoop/hbase/HRegionServer.java	(working copy)
> @@ -468,7 +468,7 @@
>     * Sets a flag that will cause all the HRegionServer threads to shut down
>     * in an orderly fashion.
>     */
> -  synchronized void stop() {
> +  public synchronized void stop() {
>      stopRequested = true;
>      notifyAll();                        // Wakes run() if it is sleeping
>    }
> @@ -1079,7 +1079,7 @@
>    }
>  
>    /** 
> -   * Private utility method for safely obtaining an HRegion handle.
> +   * Protected utility method for safely obtaining an HRegion handle.
>     * @param regionName Name of online {@link HRegion} to return
>     * @return {@link HRegion} for <code>regionName</code>
>     * @throws NotServingRegionException
> @@ -1084,7 +1084,7 @@
>     * @return {@link HRegion} for <code>regionName</code>
>     * @throws NotServingRegionException
>     */
> -  private HRegion getRegion(final Text regionName)
> +  protected HRegion getRegion(final Text regionName)
>    throws NotServingRegionException {
>      return getRegion(regionName, false);
>    }
> @@ -1090,7 +1090,7 @@
>    }
>    
>    /** 
> -   * Private utility method for safely obtaining an HRegion handle.
> +   * Protected utility method for safely obtaining an HRegion handle.
>     * @param regionName Name of online {@link HRegion} to return
>     * @param checkRetiringRegions Set true if we're to check retiring regions
>     * as well as online regions.
> @@ -1097,7 +1097,7 @@
>     * @return {@link HRegion} for <code>regionName</code>
>     * @throws NotServingRegionException
>     */
> -  private HRegion getRegion(final Text regionName,
> +  protected HRegion getRegion(final Text regionName,
>        final boolean checkRetiringRegions)
>    throws NotServingRegionException {
>      HRegion region = null;
>   


Opening HRegionServer/HMaster/HClient for extension

Posted by James Kennedy <ja...@troove.net>.
For what i'm doing I found it necessary to extend 
HRegionServer/HRegion/HClient for some custom functionality.

Following good Java practice I see that the HBase code as been 
programmed defensively, keeping stuff private as much as possible.

However, for extensibility it would be nice if the servers/client were 
easy to extend.

Attached is a patch that makes several methods protected instead of 
private, adds getters to fields of inner classes, and some other 
modifications i found were useful for some simple extension code.

I didn't make this a Jira task because I wasn't sure if you guys 
approved of opening up the code like this but hopefully someone will 
find this useful.

- James K

Re: [jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by Doug Cutting <cu...@apache.org>.
Michael Stack wrote:
> If your patch fails, set the issue status to 'Resume Progress', upload a 
> patch with fix and then set the status to 'Submit Patch' again.

Keep in mind, however, that the primary purpose of submitting patches is 
to get a committer to review and commit them, not to run Hudson tests. 
Folks should run 'ant clean test javadoc checkstyle' before submitting 
patches.  Tests should pass.  Javadoc should report no errors. 
Checkstyle's error count should not exceed that listed at:

http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/lastSuccessfulBuild/artifact/trunk/build/test/checkstyle-errors.html

Hudson's tests should only double-check things, and not be used as a 
primary tester, which would create too much noise on this list and in 
Jira.  Submitting patches that fail Hudson testing is frowned on, 
(unless the failure is not actually due to the patch).

Doug

Re: [jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by Michael Stack <st...@duboce.net>.
Bwolen Yang (JIRA) wrote:
> finally, how does "submit patch" work?  I click on "submit patch" link on the left, but it doesn't allow me to select which patch I am submitting.  does it just assume the newest patch?
>   
It will the last submitted patch.

If your patch fails, set the issue status to 'Resume Progress', upload a 
patch with fix and then set the status to 'Submit Patch' again.

As to how the patch queue works, here is a bit of an explanation: 
http://www.nabble.com/-jira--Created%3A-%28HADOOP-1465%29-Add-cluster-stop-start-scripts-for-hbase-tf3874827.html#a10979523

St.Ack

[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506359 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

> Default implementation is supposed to check markSupported().

I just grep thru Hadoop code base, I didn't find any code that checks for this.  It is better to be safer... 
Let me know which function's default implementation is suppose to have checked this.  I will verify whether or not it does.

finally, how does "submit patch" work?  I click on "submit patch" link on the left, but it doesn't allow me to select which patch I am submitting.  does it just assume the newest patch?


> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506363 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

I was thinking of original contract of reset http://java.sun.com/j2se/1.4.2/docs/api/java/io/InputStream.html#reset(). But what you have is fine.

'Submit patch" is just a flag, it does not actually commit anything. Usually the commiter selects the latest patch. Also in the case of Hadoop, this flag triggers automated build and runs unit tests.


> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504918 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

> An alternative quick fix is to overide markSupported() in FSDataInputStream.Buffer so that it returns false.

If you do this, please also fix LineRecordReader().  Otherwise, the initial seek will fail.
My current work around (before i understood what was happening) was delaying
the initial seek till BufferedInputStream() is created.  (The orig code directly reads
using FSDataInputStream)

{code}
    boolean skipFirstLine = false; 
    ...
    } else if (start != 0) { 
      --start; 
      fileIn.seek(start); 
      skipFirstLine = true; 
    } 
 
    this.in = new BufferedInputStream(in); 
    if (skipFirstLine) { 
      start += LineRecordReader.readLine(this.in, null); 
    } 
{code}

Having an extra buffer on top means that the external buffer being used by the FSInputChecker will never be called with mark() or reset().

bwolen


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505027 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

Then we should fix BufferedReader.nextLine() too.


> 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Bwolen Yang updated HADOOP-1489:
--------------------------------

    Attachment: MRIdentity.java

This is a simple map-reduce that uses the default identity mapper / reducer.   To run this, you need to give it two input arguments
   - input file/directory
   - output directory
Next attachment slashr33.txt contains the input file to run on.  I'll document more on the skipping behavior there.

> 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
>
>
> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505012 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

We need to mark only because readLine tries to let '\r' alone to be a valid end of line character. I think most other software don't define that way. We could require a '\n' to end a line. Of course readLine should still swallow a '\r' if it immediately followed by '\n'. Then we don't need to mark or 'put back' a byte.


> 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Bwolen Yang updated HADOOP-1489:
--------------------------------

    Attachment: slashr33.txt


It turned out not to be trivial to generate a small input file that breaks both the 0.13.0 release and head of the source tree.   It probably is related to buffer size settings...etc.  The only input file I got that consistently breaks all the config setups, is the original 52MB file.  This is not so ideal.   So, here is a 45k input file that breaks 0.13.0 release with the following hadoop-site.xml settings.

{quote}
dfs.replication:  1 
hadoop.tmp.dir 
fs.default.name 
dfs.name.dir 
mapred.job.tracker 
 {quote}

Other config settings may not break on this small input file.

Running MRIdentity on this, you get
{quote}
07/06/13 17:33:28 INFO mapred.JobClient:     Map input bytes=26870
{quote}

Since the file is 45KB,  about 20k bytes didn't get read.

I want to emphasize that different buffersize setup in the config may result in different outcome.     Please let me know if there difficulties reproducing this.



> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Hadoop QA (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12506398 ] 

Hadoop QA commented on HADOOP-1489:
-----------------------------------

+1

http://issues.apache.org/jira/secure/attachment/12360131/HADOOP-1489.2.patch applied and successfully tested against trunk revision r548794.

Test results:   http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Patch/310/testReport/
Console output: http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Patch/310/console

> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504915 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

+1. I think in short term even HADOOP-1470 will not support mark().


> 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Bwolen Yang updated HADOOP-1489:
--------------------------------

    Attachment: HADOOP-1489.patch

Here is a patch for this bug.   Please review.

This patch "fixes" this bug by
- making FSDataInputStream.Buffer not support mark()/reset()
- make sure all read() in LineRecordReader is done thru a top level InputStreamBuffer.

It also adds checks to FSInputChecker to ensure that whenever read() wants to return -1, it better be not due to the request is asking for 0 bytes.   The check is somewhat aggressive to find potential violations early.   Please comment.

Finally, I added a unittest.  I ended up reading BufferedInputStream to make this test possible.  Current head of tree will fail with an assertion failure.   While it is a good test for testing the existence of current bug, it is not a good test for whether the fix is correct.  i.e., 
this test works with the patch simply because "mark()" does nothing when markSupported() is disabled (modelled after InputStream).    Suggestions?

> 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: HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505323 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

This function is a stateless static function. Bwolen's work around to create a BufferedInputSream() by before calling ReadLine() should work ok now ( because the caller anyway created BufferedInputStream()) or using BufferedReader.readLine() will also work I think.


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504923 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

btw, if people agrees with these, I would be comfortable to submitting a patch, including a check on FSInputChecker.read()'s assumption.


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505634 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------


> changes under hadoop/fs/, It will heavily conflict with HADOOP-1470. 

are you referring to  3 changes in 
   hadoop/fs/ChecksumFileSystem.java
two in callers to readBuffer(), and one in readBuffer() itself.

These 3 changes are checking for 2 assumptions:
1. read() assumes "len >= bytesPerSum"
2. readBuffer() assumes if 0 bytes read, then it must be end-of-file.

- The changes for 1 is 11 lines in 1 code block.
- The changes for 2 is 11 lines in 2 code blocks.

Generally, it would be nice for unusual assumptions to be verified at run time if it doesn't have much performance impacts.
Though if people think it would generate too much integration pain, I will remove the checks.   Please let me know which
of these 3 changes would impact integration badly. 

A separate question, would either of these assumptions go away for HADOOP-1470?
If not, could the implementer please add the checks in his/her version?
Alternatively, will HADOOP-1470 be ready soon?  If so, I would be happy to add those checks back in after 1470 get submitted.

let me know.  thanks


> 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: HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504999 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

> Shall we enforce that LineRecordReader.readLine to take BufferedInputStream as an input?

would It be too restrictive to force BufferedInputStream on future extensions of
LineRecordReader?

The only requirement is that InputStream needs to support mark/reset.  
This would be easy to check if a LineReader class got broken out of LineRecordReader.
(though maybe an overkill till someone else needs a LineReader outside of LineRecordReader).

Another thing is that reading BufferedReader.nextLine, its implementation does not depend on mark/reset as it remembers seeing \r means that when people request to read the next line, and if the first character is \n, skip it.   It is too bad BufferedReader.nextLine returns a String instead of allowing people to pass it an OutputStream to write to....   Would have been nice to use it to simlify code and avoid a copy.


> 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Raghu Angadi (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12505870 ] 

Raghu Angadi commented on HADOOP-1489:
--------------------------------------

I am fine either way. I am not opposed to having those checks in hadoop/fs. I think readBuffer() itself will go away in 1470, so both of the assumptions, I think will go way.


> 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: HADOOP-1489.patch, 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.


[jira] Commented: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12504919 ] 

Bwolen Yang commented on HADOOP-1489:
-------------------------------------

by "initial seek" I mean it's search for newline in
{code}
  LineRecordReader.readLine(fileIn, null);
{code}


> 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.


[jira] Updated: (HADOOP-1489) Input file get truncated for text files with \r\n

Posted by "Bwolen Yang (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HADOOP-1489?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Bwolen Yang updated HADOOP-1489:
--------------------------------

    Attachment: HADOOP-1489.2.patch

ok.  let's deal with the checks when HADOOP-1470 is done.
Here is a HADOOP-1489.2.patch without those 3 checks.
Are there other issues I should consider before submitting this patch for review?
also, currently the build is broken by hbase.  does patch submission need to wait for the build to be fixed first?

thanks

> 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: HADOOP-1489.2.patch, HADOOP-1489.patch, 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.