You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by Greg Lloyd <gl...@gmail.com> on 2015/06/18 06:07:47 UTC

Re: New Producer API - batched sync mode support

@Shapira You are correct from my perspective. We are using kafka for a
system where panels can send multiple events in a single message. The
current contract is such that all events fail or succeed as a whole. If
there is a failure the panel resends all the events. The existing producer
api supports this fine, am I getting left behind here for the sake of
brevity?

I can get behind not adding every feature people ask for but taking away
something is a different story all together.

On Wed, Apr 29, 2015 at 9:08 PM, Gwen Shapira <gs...@cloudera.com> wrote:

> I'm starting to think that the old adage "If two people say you are drunk,
> lie down" applies here :)
>
> Current API seems perfectly clear, useful and logical to everyone who wrote
> it... but we are getting multiple users asking for the old batch behavior
> back.
> One reason to get it back is to make upgrades easier - people won't need to
> rethink their existing logic if they get an API with the same behavior in
> the new producer. The other reason is what Ewen mentioned earlier - if
> everyone re-implements Joel's logic, we can provide something for that.
>
> How about getting the old batch send behavior back by adding a new API
> with:
> public void batchSend(List<ProducerRecord<K,V>>)
>
> With this implementation (mixes the old behavior with Joel's snippet):
> * send records one by one
> * flush
> * iterate on futures and "get" them
> * log a detailed message on each error
> * throw an exception if any send failed.
>
> It reproduces the old behavior - which apparently everyone really liked,
> and I don't think it is overly weird. It is very limited, but anyone who
> needs more control over his sends already have plenty of options.
>
> Thoughts?
>
> Gwen
>
>
>
>
> On Tue, Apr 28, 2015 at 5:29 PM, Jay Kreps <ja...@gmail.com> wrote:
>
> > Hey guys,
> >
> > The locking argument is correct for very small records (< 50 bytes),
> > batching will help here because for small records locking becomes the big
> > bottleneck. I think these use cases are rare but not unreasonable.
> >
> > Overall I'd emphasize that the new producer is way faster at virtually
> all
> > use cases. If there is a use case where that isn't true, let's look at it
> > in a data driven way by comparing the old producer to the new producer
> and
> > looking for any areas where things got worse.
> >
> > I suspect the "reducing allocations" argument to be not a big thing. We
> do
> > a number of small per-message allocations and it didn't seem to have much
> > impact. I do think there are a couple of big producer memory
> optimizations
> > we could do by reusing the arrays in the accumulator in the serialization
> > of the request but I don't think this is one of them.
> >
> > I'd be skeptical of any api that was too weird--i.e. introduces a new way
> > of partitioning, gives back errors on a per-partition rather than per
> > message basis (given that partitioning is transparent this is really hard
> > to think about), etc. Bad apis end up causing a ton of churn and just
> don't
> > end up being a good long term commitment as we change how the underlying
> > code works over time (i.e. we hyper optimize for something then have to
> > maintain some super weird api as it becomes hyper unoptimized for the
> > client over time).
> >
> > Roshan--Flush works as you would hope, it blocks on the completion of all
> > outstanding requests. Calling get on the future for the request gives you
> > the associated error code back. Flush doesn't throw any exceptions
> because
> > waiting for requests to complete doesn't error, the individual requests
> > fail or succeed which is always reported with each request.
> >
> > Ivan--The batches you send in the scala producer today actually aren't
> > truely atomic, they just get sent in a single request.
> >
> > One tricky problem to solve when user's do batching is size limits on
> > requests. This can be very hard to manage since predicting the serialized
> > size of a bunch of java objects is not always obvious. This was
> repeatedly
> > a problem before.
> >
> > -Jay
> >
> > On Tue, Apr 28, 2015 at 4:51 PM, Ivan Balashov <ib...@gmail.com>
> > wrote:
> >
> > > I must agree with @Roshan – it's hard to imagine anything more
> intuitive
> > > and easy to use for atomic batching as old sync batch api. Also, it's
> > fast.
> > > Coupled with a separate instance of producer per
> > > broker:port:topic:partition it works very well. I would be glad if it
> > finds
> > > its way into new producer api.
> > >
> > > On a side-side-side note, could anyone confirm/deny if SimpleConsumer's
> > > fetchSize must be set at least as batch bytes (before or after
> > > compression), otherwise client risks not getting any messages?
> > >
> >
>