flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Piotr Nowojski <pi...@data-artisans.com>
Subject Re: Sink buffering
Date Wed, 04 Oct 2017 08:06:14 GMT
Hi,

Do you mean buffer on state and you want to achieve exactly-once HBase sink? If so keep in
mind that you will need some kind of transactions support in HBase to make it 100% reliable.

Without transactions, buffering messages on state only reduces chance of duplicated records.
How much “reduced” depends on checkpointing interval and how long does it take to rewrite
messages from state buffer to HBase - if checkpoint interval is 10s, rewriting takes 2s, your
duplicate chances are 2/10 of what you would have without buffering, because you will actively
write to HBase only 20% of time you would normally write without the buffering.

Having said that you can take a look at following classes on which you could base your sink:
GenericWriteAheadSink - probably this is better for you
TwoPhaseCommitSinkFunction - this one is in master branch, but hasn’t yet been released

Piotrek

> On Sep 29, 2017, at 6:21 PM, nragon <nuno.goncalves@wedotechnologies.com> wrote:
> 
> Hi,
> 
> Just like mentioned at Berlin FF17, Pravega talk, can we simulate, somehow,
> sink buffering(pravega transactions) and coordinate them with checkpoints?
> My intension is to buffer records before sending them to hbase.
> Any opinions or tips?
> 
> Thanks
> 
> 
> 
> --
> Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/


Mime
View raw message