hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Colin Patrick McCabe (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HDFS-7199) DFSOutputStream can silently drop data if DataStreamer crashes with a non-I/O exception
Date Wed, 29 Oct 2014 19:38:34 GMT

    [ https://issues.apache.org/jira/browse/HDFS-7199?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14188861#comment-14188861
] 

Colin Patrick McCabe commented on HDFS-7199:
--------------------------------------------

I'm having trouble understanding this patch.  Won't the exception you are setting with {{setLastException(new
IOException("DataStreamer Exception: ",e))}} overwrite the exception set on these previous
lines:

{code}
          if (e instanceof IOException) {
            setLastException((IOException)e);
          }
 {code}

Wouldn't it make more sense to simply add an else statement here where we wrap the non-IOE
in an IOE?

bq. working in progress patch. I will work on creating the test case. It is a littble bit
hard.

It looks like this will end up being a 1 or 2 line patch.  So we could potentially commit
this JIRA and file a follow-up JIRA for the test case.  I think it should be possible to write
a good test case using Mockito or perhaps one of the fault injectors.

> DFSOutputStream can silently drop data if DataStreamer crashes with a non-I/O exception
> ---------------------------------------------------------------------------------------
>
>                 Key: HDFS-7199
>                 URL: https://issues.apache.org/jira/browse/HDFS-7199
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: hdfs-client
>    Affects Versions: 2.5.0
>            Reporter: Jason Lowe
>            Assignee: Rushabh S Shah
>            Priority: Critical
>         Attachments: HDFS-7199-WIP.patch
>
>
> If the DataStreamer thread encounters a non-I/O exception then it closes the output stream
but does not set lastException.  When the client later calls close on the output stream then
it will see the stream is already closed with lastException == null, mistakently think this
is a redundant close call, and fail to report any error to the client.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message