[ https://issues.apache.org/jira/browse/HDFS-1529?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12974475#action_12974475 ]
Todd Lipcon commented on HDFS-1529: ----------------------------------- Potentially, but now you're talking about a significant change to the invariants of DFSOutputStream, which seems a little more involved, and also would also require a lot of changes to the way hflush works. For example, if we left an hflush "partial chunk" packet in {{currentPacket}} then we'd have to do some tricky maneuvering to restore the correct state of the various offsets, right? Since interrupt is almost always used to get something to shut down, it seems unlikely we'd do it so many times quickly in a row that we'd overrun memory by allocating these 64KB buffers. Usually there would be one interrupt and then a close. > Incorrect handling of interrupts in waitForAckedSeqno can cause deadlock > ------------------------------------------------------------------------ > > Key: HDFS-1529 > URL: https://issues.apache.org/jira/browse/HDFS-1529 > Project: Hadoop HDFS > Issue Type: Bug > Components: hdfs client > Affects Versions: 0.22.0 > Reporter: Todd Lipcon > Assignee: Todd Lipcon > Priority: Blocker > Attachments: hdfs-1529.txt, hdfs-1529.txt, hdfs-1529.txt, Test.java > > > In HDFS-895 the handling of interrupts during hflush/close was changed to > preserve interrupt status. This ends up creating an infinite loop in > waitForAckedSeqno if the waiting thread gets interrupted, since Object.wait() > has a strange semantic that it doesn't give up the lock even momentarily if > the thread is already in interrupted state at the beginning of the call. > We should decide what the correct behavior is here - if a thread is > interrupted while it's calling hflush() or close() should we (a) throw an > exception, perhaps InterruptedIOException (b) ignore, or (c) wait for the > flush to finish but preserve interrupt status on exit? -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.