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

ASF GitHub Bot commented on FLINK-2055:
---------------------------------------

Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/2332
  
    Don't you loose any guarantees regarding order of mutations the moment you 
use asynchronous updates anyway?
    
    The WriteAheadSink should only be used if you want to deal with 
non-deterministic programs or want to send data in atomic mini-batches and must 
rely on the order of elements. Otherwise there are much simpler solutions.
    
    If you do idempotent updates the only thing you have to do is write the 
data into HBase, and make sure that every update sent for a given checkpoint is 
acknowledged before it is regarded as complete. If you don't acknowledge them 
you lose at-least-once guarantees. This scheme does not provide exactly-once 
*delivery* guarantees, however at any given point in time the table would be in 
a state as if the updates were only sent once. This is the same guarantee that 
we provide for Cassandra.
    
    For non-idempotent updates the thing gets a lot more difficult.
    
    If you can fire an entire checkpoint as a single atomic batch you just won 
the lottery, as you can use the above scheme and a small auxiliary table to 
track completed checkpoints per sink subtask.
    
    if you can't do that you will have to use system-specific 
features/guarantees to engineer a solution that provides exactly-once 
guarantees. Versioning, rollbacks, unique ID's; something that either allows 
you to revert the table to a clean state or track precisely which updates were 
applied and sent the remaining updates.


> Implement Streaming HBaseSink
> -----------------------------
>
>                 Key: FLINK-2055
>                 URL: https://issues.apache.org/jira/browse/FLINK-2055
>             Project: Flink
>          Issue Type: New Feature
>          Components: Streaming, Streaming Connectors
>    Affects Versions: 0.9
>            Reporter: Robert Metzger
>            Assignee: Erli Ding
>
> As per : 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Write-Stream-to-HBase-td1300.html



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

Reply via email to