[
https://issues.apache.org/jira/browse/STORM-837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14629736#comment-14629736
]
Arun Mahadevan commented on STORM-837:
--------------------------------------
Agree with the approach of keeping data and txd id index in separate files.
Based on my understanding of transactional and opaque transactional spouts,
1. Transactional spout - (if same batch is re-emitted, it would contain the
same tuples as before)
- If we have seen the txn earlier and committed we can ignore this batch
- If we have seen the txn earlier and not committed, we go to the txn offset
and overwrite (to keep it simple).
2. Opaque transactional spout - (same batch could be re-emitted with tuples
having different values, which should override)
- If we have seen the txn earlier, we need to overwrite from the txn start
offset whether we have committed or not since the tuple values may have changed.
Hence if we get a txn where we have seen “preCommit”, we go to the start offset
in the corresponding data file and start writing from that offset (which would
handle both 1 & 2).
I think the third case where we get a txn id and the prev txn id is not
committed ideally should not happen since trident guarantees that state updates
would be ordered. (https://storm.apache.org/documentation/Trident-state).
> HdfsState ignores commits
> -------------------------
>
> Key: STORM-837
> URL: https://issues.apache.org/jira/browse/STORM-837
> Project: Apache Storm
> Issue Type: Bug
> Reporter: Robert Joseph Evans
> Assignee: Arun Mahadevan
> Priority: Critical
>
> HdfsState works with trident which is supposed to provide exactly once
> processing. It does this two ways, first by informing the state about
> commits so it can be sure the data is written out, and second by having a
> commit id, so that double commits can be handled.
> HdfsState ignores the beginCommit and commit calls, and with that ignores the
> ids. This means that if you use HdfsState and your worker crashes you may
> both lose data and get some data twice.
> At a minimum the flush and file rotation should be tied to the commit in some
> way. The commit ID should at a minimum be written out with the data so
> someone reading the data can have a hope of deduping it themselves.
> Also with the rotationActions it is possible for a file that was partially
> written is leaked, and never moved to the final location, because it is not
> rotated. I personally think the actions are too generic for this case and
> need to be deprecated.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)