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

Aaron Dossett commented on STORM-960:
-------------------------------------

[~sriharsha] PR updated to reflect only my commits.  Thank you for the comment 
about flushAndCloseAllWriters, I have made another commit to account for this 
and make the unit tests better.

Here's the test scenario where I see tuple loss:

- Set hive batch size = 10
- process nine tuples, hive table has zero rows as expected
- take down Hive metastore
- process one more tuple.  That tuple is failed because hive metastore is down 
as expected.  Note at this point all previous nine tuples have been acked.
- restart hive metastore
- the 10th tuple is eventually processed and the hive writer flushes (because 
currentBatchSize is now 10)
- one row is in the hive table
- wait several minutes to ensure that first nine tuples are not retried
- confirm only one row exists in the hive table

With my change and the same test scenario:
- after restarting the metastore and waiting several minutes for a retry, 
confirmed that 10 rows were in the Hive table.

If the metastore is completely down then flushAllWriters and 
flushAndCloseWriters will fail to write to Hive.  In that case no tuples should 
end up being acknowledged.

Note: a key change for this was to have flushAndCloseWriters re-throw any 
exception it catches from flushAllWriters

> Hive-Bolt can lose tuples when flushing data
> --------------------------------------------
>
>                 Key: STORM-960
>                 URL: https://issues.apache.org/jira/browse/STORM-960
>             Project: Apache Storm
>          Issue Type: Improvement
>          Components: external
>            Reporter: Aaron Dossett
>            Assignee: Aaron Dossett
>            Priority: Minor
>
> In HiveBolt's execute method tuples are ack'd as they are received.  When a 
> batchsize of tuples has been received, the writers are flushed.  However, if 
> the flush fails only the most recent tuple will be marked as failed.  All 
> prior tuples will already have been ack'd.  This creates a window for data 
> loss.



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

Reply via email to