Hadoop Common
  1. Hadoop Common
  2. HADOOP-1489

Input file get truncated for text files with \r\n

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.13.0
    • Fix Version/s: 0.14.0
    • Component/s: io
    • Labels:
      None

      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

         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 (see the callee one starts with "Caused by")

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

      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
      
      
      1. slashr33.txt
        44 kB
        Bwolen Yang
      2. MRIdentity.java
        0.6 kB
        Bwolen Yang
      3. HADOOP-1489.patch
        9 kB
        Bwolen Yang
      4. HADOOP-1489.2.patch
        7 kB
        Bwolen Yang

        Activity

        Hide
        Bwolen Yang added a comment -

        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.
        Show
        Bwolen Yang added a comment - 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.
        Hide
        Bwolen Yang added a comment -

        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.

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

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

        Running MRIdentity on this, you get

        07/06/13 17:33:28 INFO mapred.JobClient: Map input bytes=26870

        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.

        Show
        Bwolen Yang added a comment - 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. dfs.replication: 1 hadoop.tmp.dir fs.default.name dfs.name.dir mapred.job.tracker Other config settings may not break on this small input file. Running MRIdentity on this, you get 07/06/13 17:33:28 INFO mapred.JobClient: Map input bytes=26870 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.
        Hide
        dhruba borthakur added a comment -

        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.

        Show
        dhruba borthakur added a comment - 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.
        Hide
        Raghu Angadi added a comment -

        It exists in prev releases also.

        Show
        Raghu Angadi added a comment - It exists in prev releases also.
        Hide
        dhruba borthakur added a comment -

        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.

        Show
        dhruba borthakur added a comment - 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.
        Hide
        Raghu Angadi added a comment -

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

        Show
        Raghu Angadi added a comment - > 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.
        Hide
        Hairong Kuang added a comment -

        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.

        Show
        Hairong Kuang added a comment - 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.
        Hide
        Doug Cutting added a comment -

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

        Show
        Doug Cutting added a comment - > 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.
        Hide
        Raghu Angadi added a comment -

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

        Show
        Raghu Angadi added a comment - +1. I think in short term even HADOOP-1470 will not support mark().
        Hide
        Bwolen Yang added a comment -

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

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

        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

        Show
        Bwolen Yang added a comment - > 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) 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 ); } 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
        Hide
        Bwolen Yang added a comment -

        by "initial seek" I mean it's search for newline in

          LineRecordReader.readLine(fileIn, null);
        
        Show
        Bwolen Yang added a comment - by "initial seek" I mean it's search for newline in LineRecordReader.readLine(fileIn, null );
        Hide
        Bwolen Yang added a comment -

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

        Show
        Bwolen Yang added a comment - btw, if people agrees with these, I would be comfortable to submitting a patch, including a check on FSInputChecker.read()'s assumption.
        Hide
        Hairong Kuang added a comment -

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

        Show
        Hairong Kuang added a comment - +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.
        Hide
        Bwolen Yang added a comment -

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

        Show
        Bwolen Yang added a comment - > 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.
        Hide
        Raghu Angadi added a comment -

        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.

        Show
        Raghu Angadi added a comment - 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.
        Hide
        Raghu Angadi added a comment -
        bash-3.2$ printf "1 \n 2 \n 3 \r 4 \n" | wc
              3       4      14
        

        But current our readLine would count 4 lines.

        Show
        Raghu Angadi added a comment - bash-3.2$ printf "1 \n 2 \n 3 \r 4 \n" | wc 3 4 14 But current our readLine would count 4 lines.
        Hide
        Bwolen Yang added a comment -

        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.

        Show
        Bwolen Yang added a comment - 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.
        Hide
        Raghu Angadi added a comment -

        Then we should fix BufferedReader.nextLine() too.

        Show
        Raghu Angadi added a comment - Then we should fix BufferedReader.nextLine() too.
        Hide
        Bwolen Yang added a comment -

        BufferedReader came from java.io

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

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

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

        Show
        Bwolen Yang added a comment - BufferedReader came from java.io http://java.sun.com/j2se/1.3/docs/api/java/io/BufferedReader.html The semantics of treating \r without \n afterwards as a end of a line seems to be a correct semantics since \r is the carriage return character. I am only saying what BufferedReader implements basically is what Hadoop's LineRecordReader does without using mark()/reset(). Had it's nextLine() allows writes to OutputStream, we could simplify LineRecordReader to use it instead of implement yet another line reader.
        Hide
        Doug Cutting added a comment -

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

        Show
        Doug Cutting added a comment - I like the idea of using the same logic as BufferedReader. Can we simply maintain a flag instead of using mark() and reset()?
        Hide
        Raghu Angadi added a comment -

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

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

        Here is a patch for this bug. Please review.

        This patch "fixes" this bug by

        • making FSDataInputStream.Buffer not support mark()/reset()
        • make sure all read() in LineRecordReader is done thru a top level InputStreamBuffer.

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

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

        Show
        Bwolen Yang added a comment - Here is a patch for this bug. Please review. This patch "fixes" this bug by making FSDataInputStream.Buffer not support mark()/reset() make sure all read() in LineRecordReader is done thru a top level InputStreamBuffer. It also adds checks to FSInputChecker to ensure that whenever read() wants to return -1, it better be not due to the request is asking for 0 bytes. The check is somewhat aggressive to find potential violations early. Please comment. Finally, I added a unittest. I ended up reading BufferedInputStream to make this test possible. Current head of tree will fail with an assertion failure. While it is a good test for testing the existence of current bug, it is not a good test for whether the fix is correct. i.e., this test works with the patch simply because "mark()" does nothing when markSupported() is disabled (modelled after InputStream). Suggestions?
        Hide
        Raghu Angadi added a comment -

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

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

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

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

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

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

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

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

        let me know. thanks

        Show
        Bwolen Yang added a comment - > changes under hadoop/fs/, It will heavily conflict with HADOOP-1470 . are you referring to 3 changes in hadoop/fs/ChecksumFileSystem.java two in callers to readBuffer(), and one in readBuffer() itself. These 3 changes are checking for 2 assumptions: 1. read() assumes "len >= bytesPerSum" 2. readBuffer() assumes if 0 bytes read, then it must be end-of-file. The changes for 1 is 11 lines in 1 code block. The changes for 2 is 11 lines in 2 code blocks. Generally, it would be nice for unusual assumptions to be verified at run time if it doesn't have much performance impacts. Though if people think it would generate too much integration pain, I will remove the checks. Please let me know which of these 3 changes would impact integration badly. A separate question, would either of these assumptions go away for HADOOP-1470 ? If not, could the implementer please add the checks in his/her version? Alternatively, will HADOOP-1470 be ready soon? If so, I would be happy to add those checks back in after 1470 get submitted. let me know. thanks
        Hide
        Raghu Angadi added a comment -

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

        Show
        Raghu Angadi added a comment - I am fine either way. I am not opposed to having those checks in hadoop/fs. I think readBuffer() itself will go away in 1470, so both of the assumptions, I think will go way.
        Hide
        Bwolen Yang added a comment -

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

        thanks

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

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

        Show
        Raghu Angadi added a comment - +1. Changes look good. I am not sure if implementing reset() is necessary though. Default implementation is supposed to check markSupported().
        Hide
        Bwolen Yang added a comment -

        > Default implementation is supposed to check markSupported().

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

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

        Show
        Bwolen Yang added a comment - > Default implementation is supposed to check markSupported(). I just grep thru Hadoop code base, I didn't find any code that checks for this. It is better to be safer... Let me know which function's default implementation is suppose to have checked this. I will verify whether or not it does. finally, how does "submit patch" work? I click on "submit patch" link on the left, but it doesn't allow me to select which patch I am submitting. does it just assume the newest patch?
        Hide
        Raghu Angadi added a comment -

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

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

        Show
        Raghu Angadi added a comment - I was thinking of original contract of reset http://java.sun.com/j2se/1.4.2/docs/api/java/io/InputStream.html#reset( ). But what you have is fine. 'Submit patch" is just a flag, it does not actually commit anything. Usually the commiter selects the latest patch. Also in the case of Hadoop, this flag triggers automated build and runs unit tests.
        Hide
        Bwolen Yang added a comment -

        marking for sumbit.

        Show
        Bwolen Yang added a comment - marking for sumbit.
        Hide
        Bwolen Yang added a comment -

        > original contract of reset

        Looks in in the entire java source code, only one function actually bothered to check for this...

           net/URLConnection.java
           static public String guessContentTypeFromStream(InputStream is)
                                throws IOException {
                // If we can't read ahead safely, just give up on guessing
                if (!is.markSupported())
        

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

        Show
        Bwolen Yang added a comment - > original contract of reset Looks in in the entire java source code, only one function actually bothered to check for this... net/URLConnection.java static public String guessContentTypeFromStream(InputStream is) throws IOException { // If we can't read ahead safely, just give up on guessing if (!is.markSupported()) Let's play it safe just in case people forgets... thanks for the input.
        Show
        Hadoop QA added a comment - +1 http://issues.apache.org/jira/secure/attachment/12360131/HADOOP-1489.2.patch applied and successfully tested against trunk revision r548794. Test results: http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Patch/310/testReport/ Console output: http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Patch/310/console
        Hide
        Doug Cutting added a comment -

        I just committed this. Thanks, Bwolen!

        Show
        Doug Cutting added a comment - I just committed this. Thanks, Bwolen!
        Hide
        Hudson added a comment -
        Show
        Hudson added a comment - Integrated in Hadoop-Nightly #131 (See http://lucene.zones.apache.org:8080/hudson/job/Hadoop-Nightly/131/ )

          People

          • Assignee:
            Unassigned
            Reporter:
            Bwolen Yang
          • Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development