Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Blocker Blocker
    • Resolution: Fixed
    • Affects Version/s: 0.20.2
    • Fix Version/s: 0.20.2
    • Component/s: hdfs-client
    • Labels:
      None
    • Hadoop Flags:
      Reviewed

      Description

      We recently started to use hadoop-0.20.1 in our production environment (less than 2 weeks ago) and already had 3 instances of truncated files, more than we had for months using hadoop-0.18.3.
      Writing is done using libhdfs, although it rather seems to be a problem on the server side.

      I will post some relevant logs (they are too large to be put into the description)

      1. h732_20091028_0.20.patch
        0.6 kB
        Tsz Wo Nicholas Sze

        Issue Links

          Activity

          Hide
          Tsz Wo Nicholas Sze added a comment -

          > ... I will have to pull it into pur 0.19 hadoop cluster as well.

          Hi Dhruba, h732_20091028_0.20.patch is also applied to 0.19. You may want to use it.

          > ... I will file new issues for TestFsck and TestReduceFetch.

          Filed HDFS-745 and MAPREDUCE-1172 for TestFsck and TestReduceFetch, respectively.

          Show
          Tsz Wo Nicholas Sze added a comment - > ... I will have to pull it into pur 0.19 hadoop cluster as well. Hi Dhruba, h732_20091028_0.20.patch is also applied to 0.19. You may want to use it. > ... I will file new issues for TestFsck and TestReduceFetch. Filed HDFS-745 and MAPREDUCE-1172 for TestFsck and TestReduceFetch, respectively.
          Tsz Wo Nicholas Sze made changes -
          Status Open [ 1 ] Resolved [ 5 ]
          Assignee Tsz Wo (Nicholas), SZE [ szetszwo ]
          Resolution Fixed [ 1 ]
          Hide
          Tsz Wo Nicholas Sze added a comment -

          I have committed this to 0.20 only.

          Show
          Tsz Wo Nicholas Sze added a comment - I have committed this to 0.20 only.
          Hide
          Tsz Wo Nicholas Sze added a comment -

          Tested on 0.20:

               [exec] -1 overall.  
               [exec] 
               [exec]     +1 @author.  The patch does not contain any @author tags.
               [exec] 
               [exec]     -1 tests included.  The patch doesn't appear to include any new or modified tests.
               [exec]                         Please justify why no tests are needed for this patch.
               [exec] 
               [exec]     +1 javadoc.  The javadoc tool did not generate any warning messages.
               [exec] 
               [exec]     +1 javac.  The applied patch does not increase the total number of javac compiler warnings.
               [exec] 
               [exec]     +1 findbugs.  The patch does not introduce any new Findbugs warnings.
               [exec] 
               [exec]     +1 Eclipse classpath. The patch retains Eclipse classpath integrity.
          

          All unit tests passed except TestDatanodeBlockScanner, TestFsck and TestReduceFetch. The failures are not related to the patch. These three tests also failed on a clean 0.20 trunk in my machine. See HDFS-734 for TestDatanodeBlockScanner. I will file new issues for TestFsck and TestReduceFetch.

          No new test is added since the change is very simple.

          I am going to commit the patch to 0.20 only since we don't have plan to release 0.19.3.

          Show
          Tsz Wo Nicholas Sze added a comment - Tested on 0.20: [exec] -1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] -1 tests included. The patch doesn't appear to include any new or modified tests. [exec] Please justify why no tests are needed for this patch. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings. [exec] [exec] +1 Eclipse classpath. The patch retains Eclipse classpath integrity. All unit tests passed except TestDatanodeBlockScanner, TestFsck and TestReduceFetch. The failures are not related to the patch. These three tests also failed on a clean 0.20 trunk in my machine. See HDFS-734 for TestDatanodeBlockScanner. I will file new issues for TestFsck and TestReduceFetch. No new test is added since the change is very simple. I am going to commit the patch to 0.20 only since we don't have plan to release 0.19.3.
          Hide
          dhruba borthakur added a comment -

          > Should we brother fixing 0.19?

          Either way is fine. I will have to pull it into pur 0.19 hadoop cluster as well.

          Show
          dhruba borthakur added a comment - > Should we brother fixing 0.19? Either way is fine. I will have to pull it into pur 0.19 hadoop cluster as well.
          Tsz Wo Nicholas Sze made changes -
          Hadoop Flags [Reviewed]
          Fix Version/s 0.20.2 [ 12314204 ]
          Affects Version/s 0.20.2 [ 12314204 ]
          Affects Version/s 0.20.1 [ 12314048 ]
          Priority Major [ 3 ] Blocker [ 1 ]
          Hide
          Tsz Wo Nicholas Sze added a comment -

          > Hi nicholas, can we put this patch in 0.20.3 release? Thanks.

          I believe this is qualified to be a blocker of 0.20.

          Should we brother fixing 0.19?

          Show
          Tsz Wo Nicholas Sze added a comment - > Hi nicholas, can we put this patch in 0.20.3 release? Thanks. I believe this is qualified to be a blocker of 0.20. Should we brother fixing 0.19?
          Hide
          dhruba borthakur added a comment -

          Hi nicholas, can we put this patch in 0.20.3 release? Thanks.

          Show
          dhruba borthakur added a comment - Hi nicholas, can we put this patch in 0.20.3 release? Thanks.
          Hide
          Hairong Kuang added a comment -

          +1. The patch looks good.

          Show
          Hairong Kuang added a comment - +1. The patch looks good.
          Tsz Wo Nicholas Sze made changes -
          Attachment h732_20091028_0.20.patch [ 12423497 ]
          Hide
          Tsz Wo Nicholas Sze added a comment -

          h732_20091028_0.20.patch: fix 0.20 by the codes in trunk.

          Show
          Tsz Wo Nicholas Sze added a comment - h732_20091028_0.20.patch: fix 0.20 by the codes in trunk.
          Hide
          Tsz Wo Nicholas Sze added a comment -

          > This idea actually was implemented by HADOOP-5859, which was committed to 0.20. So this bug does not exist in 0.20 and above. Only 0.19 and 0.20 have such problem.

          TYPO: I mean HADOOP-5859 was committed to 0.21. So this bug does not exist in 0.21 and above.

          Show
          Tsz Wo Nicholas Sze added a comment - > This idea actually was implemented by HADOOP-5859 , which was committed to 0.20. So this bug does not exist in 0.20 and above. Only 0.19 and 0.20 have such problem. TYPO: I mean HADOOP-5859 was committed to 0.21 . So this bug does not exist in 0.21 and above.
          Hide
          Tsz Wo Nicholas Sze added a comment -

          > If a DFSOutputStraem.close() returned error, then the next call to DFSOutputStream.close() should also return an error. ...
          +1

          This idea actually was implemented by HADOOP-5859, which was committed to 0.20. So this bug does not exist in 0.20 and above. Only 0.19 and 0.20 have such problem.

          Show
          Tsz Wo Nicholas Sze added a comment - > If a DFSOutputStraem.close() returned error, then the next call to DFSOutputStream.close() should also return an error. ... +1 This idea actually was implemented by HADOOP-5859 , which was committed to 0.20. So this bug does not exist in 0.20 and above. Only 0.19 and 0.20 have such problem.
          Tsz Wo Nicholas Sze made changes -
          Link This issue relates to HADOOP-5859 [ HADOOP-5859 ]
          Hide
          Hairong Kuang added a comment -

          I do not think the change made in HADOOP-4760 was right.

          > If the stream is already closed then invoking this method has no effect.
          should be interpreted as: if the stream is successfully closed then invoking this method has no effect.

          Show
          Hairong Kuang added a comment - I do not think the change made in HADOOP-4760 was right. > If the stream is already closed then invoking this method has no effect. should be interpreted as: if the stream is successfully closed then invoking this method has no effect.
          Tsz Wo Nicholas Sze made changes -
          Component/s hdfs client [ 12312928 ]
          Component/s name-node [ 12312926 ]
          Hide
          Tsz Wo Nicholas Sze added a comment -

          Since HADOOP-4760 was committed to 0.19 and above, so 0.18 does not have such problem.

          Updated component to "hdfs client".

          Show
          Tsz Wo Nicholas Sze added a comment - Since HADOOP-4760 was committed to 0.19 and above, so 0.18 does not have such problem. Updated component to "hdfs client".
          Hide
          Tsz Wo Nicholas Sze added a comment -

          The behavior of DFSOutputStream.close() was changed by HADOOP-4760.

          Show
          Tsz Wo Nicholas Sze added a comment - The behavior of DFSOutputStream.close() was changed by HADOOP-4760 .
          Tsz Wo Nicholas Sze made changes -
          Field Original Value New Value
          Link This issue relates to HADOOP-4760 [ HADOOP-4760 ]
          Hide
          dhruba borthakur added a comment -

          If a DFSOutputStraem.close() returned error, then the next call to DFSOutputStream.close() should also return an error. Christian has pointed out that this the code is such that this is not the case. It looks like a bug.

          Show
          dhruba borthakur added a comment - If a DFSOutputStraem.close() returned error, then the next call to DFSOutputStream.close() should also return an error. Christian has pointed out that this the code is such that this is not the case. It looks like a bug.
          Hide
          Tsz Wo Nicholas Sze added a comment -

          > But from our client application's point of view, when it called hdfsCloseFile in libhdfs again after a first failure of close, the call was successful. ...

          This is definitely a bug if the second call of close() returns successfully, given that the first call failed.

          Show
          Tsz Wo Nicholas Sze added a comment - > But from our client application's point of view, when it called hdfsCloseFile in libhdfs again after a first failure of close, the call was successful. ... This is definitely a bug if the second call of close() returns successfully, given that the first call failed.
          Hide
          Christian Kunz added a comment -

          Hairong, of course, you are right in technical terms.
          But from our client application's point of view, when it called hdfsCloseFile in libhdfs again after a first failure of close, the call was successful. This behavior of close in DFSOutputStream changed in hadoop-0.20.1 compared to hadoop-0.18 and made the application think that the files was successfully closed.
          As mentioned in the previous comment, we are changing our application, assuming the OutputStream should not be used after a call to close.

          Show
          Christian Kunz added a comment - Hairong, of course, you are right in technical terms. But from our client application's point of view, when it called hdfsCloseFile in libhdfs again after a first failure of close, the call was successful. This behavior of close in DFSOutputStream changed in hadoop-0.20.1 compared to hadoop-0.18 and made the application think that the files was successfully closed. As mentioned in the previous comment, we are changing our application, assuming the OutputStream should not be used after a call to close.
          Hide
          Hairong Kuang added a comment -

          > First attempt to close the file was unsuccessful, but second attempt was successful (but with truncated size).
          I do not think this is true. In most of cases, a dfs client is not able to close a file if it failed to push data to datanodes because all replicas are left in under construction state.

          As I said in my yesterday's comment, it is the NameNode that closed the file. The following NameNode log

          2009-10-23 21:16:00,397 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(blk_6703874482275767879_76840999) successful
          2009-10-23 22:16:02,159 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* blk_6703874482275767879_76840999 recovery
          started, primary=xxx.yyy.zzz.44:uuu10
          2009-10-23 22:16:02,925 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(lastblock=blk_6703874482275767879_76840999, newgenerationstamp=76888761, newlength=17825792,
          newtargets=[xxx.yyy.zzz.44:uuu10], closeFile=true, deleteBlock=false)
          

          shows that after the dfs client died (around 21:16), its lease expired after 1 hour (around 22:16). So NameNode initiated recovery and then closed the file.

          Show
          Hairong Kuang added a comment - > First attempt to close the file was unsuccessful, but second attempt was successful (but with truncated size). I do not think this is true. In most of cases, a dfs client is not able to close a file if it failed to push data to datanodes because all replicas are left in under construction state. As I said in my yesterday's comment, it is the NameNode that closed the file. The following NameNode log 2009-10-23 21:16:00,397 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(blk_6703874482275767879_76840999) successful 2009-10-23 22:16:02,159 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* blk_6703874482275767879_76840999 recovery started, primary=xxx.yyy.zzz.44:uuu10 2009-10-23 22:16:02,925 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(lastblock=blk_6703874482275767879_76840999, newgenerationstamp=76888761, newlength=17825792, newtargets=[xxx.yyy.zzz.44:uuu10], closeFile=true, deleteBlock=false) shows that after the dfs client died (around 21:16), its lease expired after 1 hour (around 22:16). So NameNode initiated recovery and then closed the file.
          Hide
          Christian Kunz added a comment -

          From what I see in the code, 'closed = true' is set in processDatanodeError after any exception, and in the finally clause of closeInternal.

          Seemingly multiple tries of close must have been introduced in the past for some early version of hadoop (maybe hadoop-0.15 or earlier) when it reduced the number of task failures, and since then we did not have problems with it (although without any beneficial effects) till hadoop-0.20.1.

          Okay, we will change our applications to not retry a close, assuming that the output stream should not be used after a call to close, whether successful or not.

          Maybe, hdfsCloseFile in libhdfs should be changed as well to deallocate resources not only when it is successful.

          Show
          Christian Kunz added a comment - From what I see in the code, 'closed = true' is set in processDatanodeError after any exception, and in the finally clause of closeInternal. Seemingly multiple tries of close must have been introduced in the past for some early version of hadoop (maybe hadoop-0.15 or earlier) when it reduced the number of task failures, and since then we did not have problems with it (although without any beneficial effects) till hadoop-0.20.1. Okay, we will change our applications to not retry a close, assuming that the output stream should not be used after a call to close, whether successful or not. Maybe, hdfsCloseFile in libhdfs should be changed as well to deallocate resources not only when it is successful.
          Hide
          Raghu Angadi added a comment -

          0.20 seems to be setting 'closed' to true inside a finally. It would be better to fix the behaviour to be equivalent to 0.21.

          That said, I think contract of close is not the real issue here. Why isn't error from first close() not treated as hard error?

          Even when the close call fails, DFS client does not go by itself and has to continue to provide consistent results.

          Do you mean DFS client does not serve other streams properly after this error?

          Besides that, what is the purpose of recovering a file aborted during close? What is a use case for that?

          This changed quite some time back. This is the normal expected behaviour of most filesystems. A user's process or machine might die in the middle of writing and there is no use of throwing the data that is already written away.

          Christian, do you expect the actual error on datanodes while writing? I would be concerned about pipeline error detection whenever I see failure on all the three datanodes. Multiple bugs were fixed in this area. Please include any stacktrace around the messages in datanode logs (third datanode log would be very useful, but looks like you were not able to recover it).

          partial data recovered after such a failure is as expected. I agree, it would be better to make second invocation of close() return error as well and it would be good practice for app not to ignore error from the first close().

          Show
          Raghu Angadi added a comment - 0.20 seems to be setting 'closed' to true inside a finally. It would be better to fix the behaviour to be equivalent to 0.21. That said, I think contract of close is not the real issue here. Why isn't error from first close() not treated as hard error? Even when the close call fails, DFS client does not go by itself and has to continue to provide consistent results. Do you mean DFS client does not serve other streams properly after this error? Besides that, what is the purpose of recovering a file aborted during close? What is a use case for that? This changed quite some time back. This is the normal expected behaviour of most filesystems. A user's process or machine might die in the middle of writing and there is no use of throwing the data that is already written away. Christian, do you expect the actual error on datanodes while writing? I would be concerned about pipeline error detection whenever I see failure on all the three datanodes. Multiple bugs were fixed in this area. Please include any stacktrace around the messages in datanode logs (third datanode log would be very useful, but looks like you were not able to recover it). partial data recovered after such a failure is as expected. I agree, it would be better to make second invocation of close() return error as well and it would be good practice for app not to ignore error from the first close().
          Hide
          dhruba borthakur added a comment -

          hi christian, the 0.20 code looks correct. DFSOutputStream.closed is initialized to false and is set to true if and only if a call to DFSOutputStream.close() successfully completed. In this case, if the app invoked DFSOutputStream.close() again, this method will return success.

          Show
          dhruba borthakur added a comment - hi christian, the 0.20 code looks correct. DFSOutputStream.closed is initialized to false and is set to true if and only if a call to DFSOutputStream.close() successfully completed. In this case, if the app invoked DFSOutputStream.close() again, this method will return success.
          Hide
          Christian Kunz added a comment -

          I am still not convinced that everything is okay. Even when the close call fails the DFS client does not go by itself and has to continue to provide consistent results.

          Our client application called hdfsCloseFile of libhdfs in 2 attempts, the second one was successful, as mentioned at the end of the first comment.

          When looking at source code of hadoop-0.18.3, hadoop-0.20.1, and trunk, I see different behaviors of the close function in DFSOutputStream:

          hadoop-0.18.3:
          close() calls closeInternal() which throws an exception when aborted previously.

          hadoop-0.20.1:
          if(closed) return;
          always returns okay when closed, even when aborted previously.

          trunk:
          if (closed)

          { IOException e = lastException; if (e == null) return; else throw e;}

          hadoop-0.18.3 and trunk are acceptable, but in hadoop-0.20.1, when a client tries to close a file twice, it will always be successful on the second attempt, even when aborted previously. This is inconsistent.

          Besides that, what is the purpose of recovering a file aborted during close? What is a use case for that?

          Show
          Christian Kunz added a comment - I am still not convinced that everything is okay. Even when the close call fails the DFS client does not go by itself and has to continue to provide consistent results. Our client application called hdfsCloseFile of libhdfs in 2 attempts, the second one was successful, as mentioned at the end of the first comment. When looking at source code of hadoop-0.18.3, hadoop-0.20.1, and trunk, I see different behaviors of the close function in DFSOutputStream: hadoop-0.18.3: close() calls closeInternal() which throws an exception when aborted previously. hadoop-0.20.1: if(closed) return; always returns okay when closed, even when aborted previously. trunk: if (closed) { IOException e = lastException; if (e == null) return; else throw e;} hadoop-0.18.3 and trunk are acceptable, but in hadoop-0.20.1, when a client tries to close a file twice, it will always be successful on the second attempt, even when aborted previously. This is inconsistent. Besides that, what is the purpose of recovering a file aborted during close? What is a use case for that?
          Hide
          Hairong Kuang added a comment -

          First of all, dfs supports non-blocking writes. Although an application may successful writes 18654752 byte and returns, the bytes may still be buffered at the client side and have not been pushed the datanodes in the pipeline yet.

          From the logs that you provided, it seemed to me that the packet that starts with the byte 17825793 was not successfully pushed to all datanodes. For some reason, three datanodes failed in a row. The dfs client tried to resend the packet twice as a result the generation stamp of the block was bumped from 76799972 to 76840998, then to 76840999 and replica length was truncated to 17825792. Eventually the dfs client failed with an error: "All datanodes xxx.yyy.zzz.44:uuu10 are bad. Aborting..". Afterwards NN tried to recover this un-closed file. Since the only valid replica at xxx.yyy.zzz.44:uuu10 had 17825792 bytes, that's why the block ended up with 17825792 bytes.

          Basically dfs does not provide any guarantee on the file length if a dfs client goes away and the file is left unclosed. But in 0.21, if an application calls hflush(), then dfs guarantee
          that hflush bytes will not be truncated on error recovery.

          Show
          Hairong Kuang added a comment - First of all, dfs supports non-blocking writes. Although an application may successful writes 18654752 byte and returns, the bytes may still be buffered at the client side and have not been pushed the datanodes in the pipeline yet. From the logs that you provided, it seemed to me that the packet that starts with the byte 17825793 was not successfully pushed to all datanodes. For some reason, three datanodes failed in a row. The dfs client tried to resend the packet twice as a result the generation stamp of the block was bumped from 76799972 to 76840998, then to 76840999 and replica length was truncated to 17825792. Eventually the dfs client failed with an error: "All datanodes xxx.yyy.zzz.44:uuu10 are bad. Aborting..". Afterwards NN tried to recover this un-closed file. Since the only valid replica at xxx.yyy.zzz.44:uuu10 had 17825792 bytes, that's why the block ended up with 17825792 bytes. Basically dfs does not provide any guarantee on the file length if a dfs client goes away and the file is left unclosed. But in 0.21, if an application calls hflush(), then dfs guarantee that hflush bytes will not be truncated on error recovery.
          Hide
          Christian Kunz added a comment -

          Concerning the file in the first comment I found logs of 2 datanodes showing that indeed the block size shrunk from 18153472 to 17825792. 18153472 is not the correct size, but it is larger than 17825792, and I would argue, that a block should never be recovered by a block of smaller size.

          Logs from datanode receiving original block:

          2009-10-23 19:46:47,934 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving block blk_6703874482275767879_76799972 src: /xxx.yyy.zzz.43:34608 dest: /xxx.yyy.zzz.43:uuu10
          2009-10-23 21:15:59,694 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(xxx.yyy.zzz.43:uuu10, storageID=DS-243564233-xxx.yyy.zzz.43-uuu10-1254870555871, infoPort=50075, ipcPort=8020):Exception writing block blk_6703874482275767879_76799972 to mirror xxx.yyy.zzz.56:uuu10
          2009-10-23 21:15:59,694 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_6703874482275767879_76799972 java.io.IOException: Connection reset by peer
          2009-10-23 21:15:59,711 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 2 Exception java.io.EOFException
          2009-10-23 21:15:59,713 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 2 : Thread is interrupted.
          2009-10-23 21:15:59,713 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 2 for block blk_6703874482275767879_76799972 terminating
          2009-10-23 21:15:59,713 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_6703874482275767879_76799972 received exception java.io.IOException: Connection reset by peer

          Logs from datanode next in the pipeline:

          2009-10-23 19:46:48,174 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving block blk_6703874482275767879_76799972 src: /xxx.yyy.zzz.43:34609 dest: /xxx.yyy.zzz.56:uuu10
          2009-10-23 21:15:59,661 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(xxx.yyy.zzz.56:uuu10, storageID=DS-807595239-72.30.217.12-50010-1203107050520, infoPort=50075, ipcPort=8020):Exception writing block blk_6703874482275767879_76799972 to mirror xxx.yyy.zzz.44:uuu10
          2009-10-23 21:15:59,661 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_6703874482275767879_76799972 java.io.IOException: Connection reset by peer
          2009-10-23 21:15:59,680 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 1 Exception java.io.EOFException
          2009-10-23 21:15:59,681 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 1 : Thread is interrupted.
          2009-10-23 21:15:59,681 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 1 for block blk_6703874482275767879_76799972 terminating
          2009-10-23 21:15:59,681 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_6703874482275767879_76799972 received exception java.io.IOException: Connection reset by peer
          2009-10-23 21:16:00,069 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: oldblock=blk_6703874482275767879_76799972(length=18153472), newblock=blk_6703874482275767879_76840998(length=17825792), datanode=xxx.yyy.zzz.56:uuu10
          2009-10-23 21:16:00,134 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving block blk_6703874482275767879_76840998 src: /xxx.yyy.zzz.43:36067 dest: /xxx.yyy.zzz.56:uuu10
          2009-10-23 21:16:00,134 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Reopen already-open Block for append blk_6703874482275767879_76840998
          2009-10-23 21:16:00,154 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Changing block file offset of block blk_6703874482275767879_76840998 from 0 to 17825792 meta file offset to 17415
          2009-10-23 21:16:00,171 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(xxx.yyy.zzz.56:uuu10, storageID=DS-807595239-72.30.217.12-50010-1203107050520, infoPort=50075, ipcPort=8020):Exception writing block blk_6703874482275767879_76840998 to mirror xxx.yyy.zzz.44:uuu10
          2009-10-23 21:16:00,171 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_6703874482275767879_76840998 java.io.IOException: Connection reset by peer
          2009-10-23 21:16:00,400 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76840998 1 Exception java.nio.channels.ClosedByInterruptException
          2009-10-23 21:16:00,417 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76840998 1 : Thread is interrupted.
          2009-10-23 21:16:00,417 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 1 for block blk_6703874482275767879_76840998 terminating
          2009-10-23 21:16:00,417 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_6703874482275767879_76840998 received exception java.io.IOException: Connection reset by peer

          Show
          Christian Kunz added a comment - Concerning the file in the first comment I found logs of 2 datanodes showing that indeed the block size shrunk from 18153472 to 17825792. 18153472 is not the correct size, but it is larger than 17825792, and I would argue, that a block should never be recovered by a block of smaller size. Logs from datanode receiving original block: 2009-10-23 19:46:47,934 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving block blk_6703874482275767879_76799972 src: /xxx.yyy.zzz.43:34608 dest: /xxx.yyy.zzz.43:uuu10 2009-10-23 21:15:59,694 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(xxx.yyy.zzz.43:uuu10, storageID=DS-243564233-xxx.yyy.zzz.43-uuu10-1254870555871, infoPort=50075, ipcPort=8020):Exception writing block blk_6703874482275767879_76799972 to mirror xxx.yyy.zzz.56:uuu10 2009-10-23 21:15:59,694 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_6703874482275767879_76799972 java.io.IOException: Connection reset by peer 2009-10-23 21:15:59,711 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 2 Exception java.io.EOFException 2009-10-23 21:15:59,713 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 2 : Thread is interrupted. 2009-10-23 21:15:59,713 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 2 for block blk_6703874482275767879_76799972 terminating 2009-10-23 21:15:59,713 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_6703874482275767879_76799972 received exception java.io.IOException: Connection reset by peer Logs from datanode next in the pipeline: 2009-10-23 19:46:48,174 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving block blk_6703874482275767879_76799972 src: /xxx.yyy.zzz.43:34609 dest: /xxx.yyy.zzz.56:uuu10 2009-10-23 21:15:59,661 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(xxx.yyy.zzz.56:uuu10, storageID=DS-807595239-72.30.217.12-50010-1203107050520, infoPort=50075, ipcPort=8020):Exception writing block blk_6703874482275767879_76799972 to mirror xxx.yyy.zzz.44:uuu10 2009-10-23 21:15:59,661 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_6703874482275767879_76799972 java.io.IOException: Connection reset by peer 2009-10-23 21:15:59,680 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 1 Exception java.io.EOFException 2009-10-23 21:15:59,681 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76799972 1 : Thread is interrupted. 2009-10-23 21:15:59,681 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 1 for block blk_6703874482275767879_76799972 terminating 2009-10-23 21:15:59,681 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_6703874482275767879_76799972 received exception java.io.IOException: Connection reset by peer 2009-10-23 21:16:00,069 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: oldblock=blk_6703874482275767879_76799972(length=18153472), newblock=blk_6703874482275767879_76840998(length=17825792), datanode=xxx.yyy.zzz.56:uuu10 2009-10-23 21:16:00,134 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving block blk_6703874482275767879_76840998 src: /xxx.yyy.zzz.43:36067 dest: /xxx.yyy.zzz.56:uuu10 2009-10-23 21:16:00,134 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Reopen already-open Block for append blk_6703874482275767879_76840998 2009-10-23 21:16:00,154 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Changing block file offset of block blk_6703874482275767879_76840998 from 0 to 17825792 meta file offset to 17415 2009-10-23 21:16:00,171 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(xxx.yyy.zzz.56:uuu10, storageID=DS-807595239-72.30.217.12-50010-1203107050520, infoPort=50075, ipcPort=8020):Exception writing block blk_6703874482275767879_76840998 to mirror xxx.yyy.zzz.44:uuu10 2009-10-23 21:16:00,171 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Exception in receiveBlock for block blk_6703874482275767879_76840998 java.io.IOException: Connection reset by peer 2009-10-23 21:16:00,400 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76840998 1 Exception java.nio.channels.ClosedByInterruptException 2009-10-23 21:16:00,417 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder blk_6703874482275767879_76840998 1 : Thread is interrupted. 2009-10-23 21:16:00,417 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: PacketResponder 1 for block blk_6703874482275767879_76840998 terminating 2009-10-23 21:16:00,417 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock blk_6703874482275767879_76840998 received exception java.io.IOException: Connection reset by peer
          Hide
          Christian Kunz added a comment -

          Here I post a message I sent to common-dev on September 4, about an incidence where
          commitBlockSynchronization replaced a block with a block of smaller size.
          This happened on hadoop-0.18.3, but it seemed to be very rare. Question is whether this is the same problem as the one in the previous comment, and if so, whether something changed in hadoop-0.20.1 to increase the rate of such incidences.

          Anyhow, are there situations where block sizes can shrink? If not, this should <b>absolutely</b> not be allowed.

          Here are the relevant log messages of the namenode, showing the block size went from 273592 to 262144.

          log.2009-09-01:2009-09-01 20:52:45,451 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          NameSystem.allocateBlock:
          fileName.blk_2101888387690035515_63606967

          log.2009-09-01:2009-09-01 20:57:43,535 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.234:54610 is added to blk_2101888387690035515_63606967 size 273592

          log.2009-09-01:2009-09-01 20:57:47,847 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.250:54610 is added to blk_2101888387690035515_63606967 size 273592

          log.2009-09-01:2009-09-01 20:58:01,480 INFO org.apache.hadoop.fs.FSNamesystem:
          commitBlockSynchronization(lastblock=blk_2101888387690035515_63606967, newgenerationstamp=63637360, newlength=262144,
          newtargets=[xxx.yyy.zzz.234:54610])

          log.2009-09-01:2009-09-01 20:58:01,482 INFO org.apache.hadoop.fs.FSNamesystem:
          commitBlockSynchronization(blk_2101888387690035515_63637360) successful

          log.2009-09-01:2009-09-01 21:01:07,508 INFO org.apache.hadoop.fs.FSNamesystem:
          commitBlockSynchronization(lastblock=blk_2101888387690035515_63637360, newgenerationstamp=63639267, newlength=262144,
          newtargets=[xxx.yyy.zzz.234:54610])

          log.2009-09-01:2009-09-01 21:01:07,512 INFO org.apache.hadoop.fs.FSNamesystem:
          commitBlockSynchronization(blk_2101888387690035515_63639267) successful

          log.2009-09-01:2009-09-01 21:01:08,946 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          ask xxx.yyy.zzz.234:54610 to replicate blk_2101888387690035515_63639267 to datanode(s) xxx.yyy.zzz.230:54610 xxx.yyy.zzz.115:54610

          log.2009-09-01:2009-09-01 21:01:12,061 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.230:54610 is added to blk_2101888387690035515_63639267 size 262144

          log.2009-09-01:2009-09-01 21:01:12,097 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.115:54610 is added to blk_2101888387690035515_63639267 size 262144

          log.2009-09-01:2009-09-01 21:35:03,362 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          NameSystem.processReport: block blk_2101888387690035515_63637360 on xxx.yyy.zzz.250:54610 size 273592 does not belong to any file.

          log.2009-09-01:2009-09-01 21:35:06,344 INFO org.apache.hadoop.dfs.StateChange: BLOCK*
          ask xxx.yyy.zzz.250:54610 to delete blk_2101888387690035515_63637360

          Show
          Christian Kunz added a comment - Here I post a message I sent to common-dev on September 4, about an incidence where commitBlockSynchronization replaced a block with a block of smaller size. This happened on hadoop-0.18.3, but it seemed to be very rare. Question is whether this is the same problem as the one in the previous comment, and if so, whether something changed in hadoop-0.20.1 to increase the rate of such incidences. Anyhow, are there situations where block sizes can shrink? If not, this should <b>absolutely</b> not be allowed. Here are the relevant log messages of the namenode, showing the block size went from 273592 to 262144. log.2009-09-01:2009-09-01 20:52:45,451 INFO org.apache.hadoop.dfs.StateChange: BLOCK* NameSystem.allocateBlock: fileName.blk_2101888387690035515_63606967 log.2009-09-01:2009-09-01 20:57:43,535 INFO org.apache.hadoop.dfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.234:54610 is added to blk_2101888387690035515_63606967 size 273592 log.2009-09-01:2009-09-01 20:57:47,847 INFO org.apache.hadoop.dfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.250:54610 is added to blk_2101888387690035515_63606967 size 273592 log.2009-09-01:2009-09-01 20:58:01,480 INFO org.apache.hadoop.fs.FSNamesystem: commitBlockSynchronization(lastblock=blk_2101888387690035515_63606967, newgenerationstamp=63637360, newlength=262144, newtargets= [xxx.yyy.zzz.234:54610] ) log.2009-09-01:2009-09-01 20:58:01,482 INFO org.apache.hadoop.fs.FSNamesystem: commitBlockSynchronization(blk_2101888387690035515_63637360) successful log.2009-09-01:2009-09-01 21:01:07,508 INFO org.apache.hadoop.fs.FSNamesystem: commitBlockSynchronization(lastblock=blk_2101888387690035515_63637360, newgenerationstamp=63639267, newlength=262144, newtargets= [xxx.yyy.zzz.234:54610] ) log.2009-09-01:2009-09-01 21:01:07,512 INFO org.apache.hadoop.fs.FSNamesystem: commitBlockSynchronization(blk_2101888387690035515_63639267) successful log.2009-09-01:2009-09-01 21:01:08,946 INFO org.apache.hadoop.dfs.StateChange: BLOCK* ask xxx.yyy.zzz.234:54610 to replicate blk_2101888387690035515_63639267 to datanode(s) xxx.yyy.zzz.230:54610 xxx.yyy.zzz.115:54610 log.2009-09-01:2009-09-01 21:01:12,061 INFO org.apache.hadoop.dfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.230:54610 is added to blk_2101888387690035515_63639267 size 262144 log.2009-09-01:2009-09-01 21:01:12,097 INFO org.apache.hadoop.dfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.115:54610 is added to blk_2101888387690035515_63639267 size 262144 log.2009-09-01:2009-09-01 21:35:03,362 INFO org.apache.hadoop.dfs.StateChange: BLOCK* NameSystem.processReport: block blk_2101888387690035515_63637360 on xxx.yyy.zzz.250:54610 size 273592 does not belong to any file. log.2009-09-01:2009-09-01 21:35:06,344 INFO org.apache.hadoop.dfs.StateChange: BLOCK* ask xxx.yyy.zzz.250:54610 to delete blk_2101888387690035515_63637360
          Hide
          Christian Kunz added a comment -

          The file size of the file should have been 18654752, computed by adding up all the bytes successfully written. But it ended up having only 17825792. I verified that the missing data was at the end, i.e. the block got somehow recovered from an incomplete replica.

          Here are relevant log messages from the namenode:

          grep blk_6703874482275767879 hadoop-user-namenode-host.log.2009-10-23

          2009-10-23 19:46:47,815 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.allocateBlock:
          dir/output/app/_temporary/_attempt_200910221954_0001_r_009110_0/9110/filename.
          blk_6703874482275767879_76799972
          2009-10-23 21:16:00,028 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(lastblock=blk_6703874482275767879_76799972, newgenerationstamp=76840998, newlength=17825792,
          newtargets=[xxx.yyy.zzz.56:uuu10, xxx.yyy.zzz.44:uuu10], closeFile=false, deleteBlock=false)
          2009-10-23 21:16:00,028 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(blk_6703874482275767879_76840998) successful
          2009-10-23 21:16:00,397 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(lastblock=blk_6703874482275767879_76840998, newgenerationstamp=76840999, newlength=17825792,
          newtargets=[xxx.yyy.zzz.44:uuu10], closeFile=false, deleteBlock=false)
          2009-10-23 21:16:00,397 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(blk_6703874482275767879_76840999) successful
          2009-10-23 22:16:02,159 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* blk_6703874482275767879_76840999 recovery
          started, primary=xxx.yyy.zzz.44:uuu10
          2009-10-23 22:16:02,925 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: addStoredBlock
          request received for blk_6703874482275767879_76888761 on xxx.yyy.zzz.44:uuu10 size 17825792 But it does not belong to any
          file.
          2009-10-23 22:16:02,925 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(lastblock=blk_6703874482275767879_76840999, newgenerationstamp=76888761, newlength=17825792,
          newtargets=[xxx.yyy.zzz.44:uuu10], closeFile=true, deleteBlock=false)
          2009-10-23 22:16:02,931 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem:
          commitBlockSynchronization(newblock=blk_6703874482275767879_76888761,
          file=dir/output/app/9110/filename,
          newgenerationstamp=76888761, newlength=17825792, newtargets=[xxx.yyy.zzz.44:uuu10]) successful
          2009-10-23 22:16:05,112 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* ask xxx.yyy.zzz.44:uuu10 to replicate
          blk_6703874482275767879_76888761 to datanode(s) xxx.yyy.zzz.51:uuu10 xxx.yyy.zzz.237:uuu10
          2009-10-23 22:16:06,310 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated:
          xxx.yyy.zzz.51:uuu10 is added to blk_6703874482275767879_76888761 size 17825792
          2009-10-23 22:16:06,312 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated:
          xxx.yyy.zzz.237:uuu10 is added to blk_6703874482275767879_76888761 size 17825792

          Relevant log messages of the task (the block went through recovery):

          09/10/23 21:15:59 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block
          blk_6703874482275767879_76799972java.io.IOException: Connection reset by peer
          09/10/23 21:15:59 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76799972 bad datanode[0]
          xxx.yyy.zzz.43:uuu10
          09/10/23 21:15:59 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76799972 in pipeline
          xxx.yyy.zzz.43:uuu10, xxx.yyy.zzz.56:uuu10, xxx.yyy.zzz.44:uuu10: bad datanode xxx.yyy.zzz.43:uuu10
          09/10/23 21:16:00 WARN hdfs.DFSClient: DataStreamer Exception: java.io.IOException: Connection reset by peer
          09/10/23 21:16:00 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76840998 bad datanode[0]
          xxx.yyy.zzz.56:uuu10
          09/10/23 21:16:00 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76840998 in pipeline
          xxx.yyy.zzz.56:uuu10, xxx.yyy.zzz.44:uuu10: bad datanode xxx.yyy.zzz.56:uuu10
          09/10/23 21:16:00 WARN hdfs.DFSClient: DataStreamer Exception: java.io.IOException: Connection reset by peer
          09/10/23 21:16:00 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76840999 bad datanode[0]
          xxx.yyy.zzz.44:uuu10
          Exception in thread "main" java.io.IOException: All datanodes xxx.yyy.zzz.44:uuu10 are bad. Aborting..

          First attempt to close the file was unsuccessful, but second attempt was successful (but with truncated size).

          Show
          Christian Kunz added a comment - The file size of the file should have been 18654752, computed by adding up all the bytes successfully written. But it ended up having only 17825792. I verified that the missing data was at the end, i.e. the block got somehow recovered from an incomplete replica. Here are relevant log messages from the namenode: grep blk_6703874482275767879 hadoop-user-namenode-host.log.2009-10-23 2009-10-23 19:46:47,815 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.allocateBlock: dir/output/app/_temporary/_attempt_200910221954_0001_r_009110_0/9110/filename. blk_6703874482275767879_76799972 2009-10-23 21:16:00,028 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(lastblock=blk_6703874482275767879_76799972, newgenerationstamp=76840998, newlength=17825792, newtargets= [xxx.yyy.zzz.56:uuu10, xxx.yyy.zzz.44:uuu10] , closeFile=false, deleteBlock=false) 2009-10-23 21:16:00,028 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(blk_6703874482275767879_76840998) successful 2009-10-23 21:16:00,397 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(lastblock=blk_6703874482275767879_76840998, newgenerationstamp=76840999, newlength=17825792, newtargets= [xxx.yyy.zzz.44:uuu10] , closeFile=false, deleteBlock=false) 2009-10-23 21:16:00,397 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(blk_6703874482275767879_76840999) successful 2009-10-23 22:16:02,159 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* blk_6703874482275767879_76840999 recovery started, primary=xxx.yyy.zzz.44:uuu10 2009-10-23 22:16:02,925 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: addStoredBlock request received for blk_6703874482275767879_76888761 on xxx.yyy.zzz.44:uuu10 size 17825792 But it does not belong to any file. 2009-10-23 22:16:02,925 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(lastblock=blk_6703874482275767879_76840999, newgenerationstamp=76888761, newlength=17825792, newtargets= [xxx.yyy.zzz.44:uuu10] , closeFile=true, deleteBlock=false) 2009-10-23 22:16:02,931 INFO org.apache.hadoop.hdfs.server.namenode.FSNamesystem: commitBlockSynchronization(newblock=blk_6703874482275767879_76888761, file=dir/output/app/9110/filename, newgenerationstamp=76888761, newlength=17825792, newtargets= [xxx.yyy.zzz.44:uuu10] ) successful 2009-10-23 22:16:05,112 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* ask xxx.yyy.zzz.44:uuu10 to replicate blk_6703874482275767879_76888761 to datanode(s) xxx.yyy.zzz.51:uuu10 xxx.yyy.zzz.237:uuu10 2009-10-23 22:16:06,310 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.51:uuu10 is added to blk_6703874482275767879_76888761 size 17825792 2009-10-23 22:16:06,312 INFO org.apache.hadoop.hdfs.StateChange: BLOCK* NameSystem.addStoredBlock: blockMap updated: xxx.yyy.zzz.237:uuu10 is added to blk_6703874482275767879_76888761 size 17825792 Relevant log messages of the task (the block went through recovery): 09/10/23 21:15:59 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block blk_6703874482275767879_76799972java.io.IOException: Connection reset by peer 09/10/23 21:15:59 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76799972 bad datanode [0] xxx.yyy.zzz.43:uuu10 09/10/23 21:15:59 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76799972 in pipeline xxx.yyy.zzz.43:uuu10, xxx.yyy.zzz.56:uuu10, xxx.yyy.zzz.44:uuu10: bad datanode xxx.yyy.zzz.43:uuu10 09/10/23 21:16:00 WARN hdfs.DFSClient: DataStreamer Exception: java.io.IOException: Connection reset by peer 09/10/23 21:16:00 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76840998 bad datanode [0] xxx.yyy.zzz.56:uuu10 09/10/23 21:16:00 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76840998 in pipeline xxx.yyy.zzz.56:uuu10, xxx.yyy.zzz.44:uuu10: bad datanode xxx.yyy.zzz.56:uuu10 09/10/23 21:16:00 WARN hdfs.DFSClient: DataStreamer Exception: java.io.IOException: Connection reset by peer 09/10/23 21:16:00 WARN hdfs.DFSClient: Error Recovery for block blk_6703874482275767879_76840999 bad datanode [0] xxx.yyy.zzz.44:uuu10 Exception in thread "main" java.io.IOException: All datanodes xxx.yyy.zzz.44:uuu10 are bad. Aborting.. First attempt to close the file was unsuccessful, but second attempt was successful (but with truncated size).
          Christian Kunz created issue -

            People

            • Assignee:
              Tsz Wo Nicholas Sze
              Reporter:
              Christian Kunz
            • Votes:
              0 Vote for this issue
              Watchers:
              16 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development