Home | About | Sematext search-lucene.com search-hadoop.com
 Search Hadoop and all its subprojects:

Switch to Threaded View
Kafka, mail # user - Re: Transactional writing


Copy link to this message
-
Re: Transactional writing
Jonathan Hodges 2013-03-28, 16:31
Awesome!  Thanks for confirmation and continued great work on Kafka!
On Thu, Mar 28, 2013 at 9:22 AM, Jun Rao <[EMAIL PROTECTED]> wrote:

> Jonathan,
>
> With a single writer, the producer can achieve exact once write. If a send
> request fails, the producer first checks the end of the log to see if the
> previous write succeeded or not. The producer will only resend if the
> previous write fails.
>
> To do this, the producer needs the offset of appended messages. In 0.8,
> such offsets are not returned in our high level producer API yet. We plan
> to extend our producer API post 0.8 to expose this information.
>
> Thanks,
>
> Jun
>
> On Wed, Mar 27, 2013 at 2:41 PM, Jonathan Hodges <[EMAIL PROTECTED]>
> wrote:
>
> > I know this is a really old thread, but it looked like the only pertinent
> > one that came up when searching for ‘exactly once’ in the archives.  I
> just
> > want to confirm my understanding of the 0.8 version in that it still
> > doesn’t completely support exactly once semantics.  With the producer
> > configured in sync mode and quorum commits there are still some edge case
> > failure modes where the producer won’t receive the ack and resends the
> > message(s).  I think I read that the consumers don’t see uncommitted
> > messages in the log, but I don’t think that addresses this producer case.
> > Please correct me if I am missing something here.
> >
> >
> > Don’t get me wrong we are very thankful for the 0.8 features.  It offers
> by
> > far the best message delivery guarantees out of the products we evaluated
> > like Rabbit and ActiveMQ.  We attempt to make are downstream consumer
> > processes idempotent to mitigate this edge case, but it isn’t always
> > feasible.  Also the suggestion by Milind in this thread of using Storm
> for
> > exactly once guarantees makes a lot of sense.  Trident State seems to
> > address this very issue (
> > https://github.com/nathanmarz/storm/wiki/Trident-state) so we could just
> > have it mediate our topics that required exactly once.
> >
> >
> > -Jonathan
> >
> >
> >
> > On Sat, Nov 3, 2012 at 1:53 PM, Milind Parikh <[EMAIL PROTECTED]
> > >wrote:
> >
> > > Why wouldn't the storm approach provide semantics of exactly once
> > > delivery? https://github.com/nathanmarz/storm
> > >
> > > Nathan actually credits the Kafka_devs for the basic idea of
> transaction
> > > persisting in one of his talks.
> > >
> > > Regards
> > > Milind
> > >
> > > On Nov 3, 2012 11:51 AM, "Rohit Prasad" <[EMAIL PROTECTED]>
> > wrote:
> > >
> > > > I agree that this approach only prevents duplicate messages to
> > partition
> > > > from the Producer side. There needs to be a similar approach on the
> > > > consumer side too. Using Zk can be one solution, or other non-ZK
> > > > approaches.
> > > >
> > > > Even if Consumer reads none or all messages of a transaction. But
> that
> > > does
> > > > not solve the transaction problem yet. Because the
> business/application
> > > > logic inside the Consumer thread may execute partially and fail. So
> it
> > > > becomes tricky to decide the point when you want to say that you have
> > > > "consumed" the message and increase consumption offset. If your
> > consumer
> > > > thread is saving some value  into DB/HDFS/etc, ideally you want this
> > save
> > > > operation and consumption offset to be incremented atomically. Thats
> > why
> > > it
> > > > boils down to Application logic implementing transactions and dealing
> > > with
> > > > duplicates.
> > > > Maybe a journalling or redo log approach on Consumer side can help
> > build
> > > > such a system.
> > > >
> > > > It will be nice if eventually kafka can be a transport which provides
> > > > "exactly once" semantics for message delivery. Then consumer threads
> > can
> > > be
> > > > sure that they receive messages once, and can build appln logic on
> top
> > of
> > > > that.
> > > >
> > > > I have a use case similar to what Jay mentioned in a previous mail. I
> > > want
> > > > to do aggregation but want the aggregated data to be correct,