[
https://issues.apache.org/jira/browse/STORM-969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14698819#comment-14698819
]
ASF GitHub Bot commented on STORM-969:
--------------------------------------
Github user arunmahadevan commented on the pull request:
https://github.com/apache/storm/pull/664#issuecomment-131602837
@harshach the changes I made were in the trident implementation (HDFSState)
which is independent of this. Anyways I reviewed the changes.
Overall it appears that the exceptions are now handled individually at
write, sync and rotate phases. But I see a few issues with the change.
- the tuples are acked only on sync - if the tuple rates are low, they will
never be acked and keep timing out and same tuples will be replayed again and
again.
- there is an attempt to sync even when the write fails with an
IOException. Since write already threw an IOException, chances are high that
the sync would also fail with IOException.
I think it might be simpler to keep the existing logic and just rotate the
file whenever we see an IOException (or maybe after a few times we repeatedly
hit the IOException) and completely fail by propagating the exception up if the
situation does not improve after a few rotations.
Also I see that the existing implementation acks the tuples before actually
syncing them to disk, which might result in data loss. I think we should change
this to ack only after an hsync and have a sync policy that considers both
count and time based thresholds.
> HDFS Bolt can end up in an unrecoverable state
> ----------------------------------------------
>
> Key: STORM-969
> URL: https://issues.apache.org/jira/browse/STORM-969
> Project: Apache Storm
> Issue Type: Improvement
> Components: storm-hdfs
> Reporter: Aaron Dossett
> Assignee: Aaron Dossett
>
> The body of the HDFSBolt.execute() method is essentially one try-catch block.
> The catch block reports the error and fails the current tuple. In some
> cases the bolt's FSDataOutputStream object (named 'out') is in an
> unrecoverable state and no subsequent calls to execute() can succeed.
> To produce this scenario:
> - process some tuples through HDFS bolt
> - put the underlying HDFS system into safemode
> - process some more tuples and receive a correct ClosedChannelException
> - take the underlying HDFS system out of safemode
> - subsequent tuples continue to fail with the same exception
> The three fundamental operations that execute takes (writing, sync'ing,
> rotating) need to be isolated so that errors from each are specifically
> handled.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)