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

classic Classic list List threaded Threaded
41 messages Options
123
Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
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.

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

Reply | Threaded
Open this post in threaded view
|

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

Markus Jelsma (Jira)
In reply to this post by Markus Jelsma (Jira)

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

123