hadoop-hdfs-issues mailing list archives

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

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

Jason Lowe commented on HDFS-7199:
----------------------------------

I believe the problem lies in the way DataStreamer is handling the error:
{code}
        } catch (Throwable e) {
          // Log warning if there was a real error.
          if (restartingNodeIndex == -1) {
            DFSClient.LOG.warn("DataStreamer Exception", e);
          }
          if (e instanceof IOException) {
            setLastException((IOException)e);
          }
          hasError = true;
          if (errorIndex == -1 && restartingNodeIndex == -1) {
            // Not a datanode issue
            streamerClosed = true;
          }
        }
{code}

We should either always call setLastException, wrapping the exception in an I/O exception
if necessary, or at least set it to something if we're going to set streamerClosed=true and
exit the datastreamer thread.  That way there will always be some kind of exception to be
picked up either in checkClosed() or close() in the output stream.

> 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
>            Priority: Critical
>
> 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