@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
Gwen, I don't care what anyone says I think we are totally stlone cold
slobar. :-)
I think the only caution I would have is that in general people ask for
many things and yet the systems we all admire tend to keep their surface
area really really simple. My observation is that never in the
2015-04-30 8:50 GMT+03:00 Ewen Cheslack-Postava e...@confluent.io:
They aren't going to get this anyway (as Jay pointed out) given the current
broker implementation
Is it also incorrect to assume atomicity even if all messages in the batch
go to the same partition?
Why do we think atomicity is expected, if the old API we are emulating here
lacks atomicity?
I don't remember emails to the mailing list saying: I expected this batch
to be atomic, but instead I got duplicates when retrying after a failed
batch send.
Maybe atomicity isn't as strong requirement as
Roshan,
If I understand correctly, you just want to make sure a number of messages
has been sent successfully. Using callback might be easier to do so.
Public class MyCallback implements Callback {
public SetRecordMetadata failedSend;
@Override
Public void
@Gwen, @Ewen,
While atomicity of a batch is nice to have, it is not essential. I don't
think users always expect such atomicity. Atomicity is not even guaranteed
in many un-batched systems let alone batched systems.
As long as the client gets informed about the ones that failed in the
batch..
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
@Ewen
No I did not use compression in my measurements.
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
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
@Joel,
If flush() works for this use case it may be an acceptable starting point
(although not as clean as a native batched sync). I am not as yet clear
about some aspects of flush's batch semantics and its suitability for this
mode of operation. Allow me explore it with you folks..
1) flush()
This sounds like flush:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+method+to+the+producer+API
which was recently implemented in trunk.
Joel
On Mon, Apr 27, 2015 at 08:19:40PM +, Roshan Naik wrote:
Been evaluating the perf of old and new Produce APIs for reliable
@Roshan - if the data was already written to Kafka, your approach will
generate LOTS of duplicates. I'm not convinced its ideal.
What's wrong with callbacks?
On Mon, Apr 27, 2015 at 2:53 PM, Roshan Naik ros...@hortonworks.com wrote:
@Gwen
- A failure in delivery of one or more events in the
I should have been clearer - I used Roshan's terminology in my reply.
Basically, the old producer batch Send() just took a sequence of
messages. I assumed Roshan is looking for something similar - which allows
for mixing messages for multiple partitions and therefore can fail for some
messages
Been evaluating the perf of old and new Produce APIs for reliable high volume
streaming data movement. I do see one area of improvement that the new API
could use for synchronous clients.
AFAIKT, the new API does not support batched synchronous transfers. To do
synchronous send, one needs to
Hi Gwen,
can you clarify: by batch do you mean the protocol MessageSet, or some java
client internal construct?
If the former I was under the impression that a produced MessageSet either
succeeds delivery or errors in its entirety on the broker.
Thanks,
Magnus
2015-04-27 23:05 GMT+02:00 Gwen
The important guarantee that is needed for a client producer thread is
that it requires an indication of success/failure of the batch of events
it pushed. Essentially it needs to retry producer.send() on that same
batch in case of failure. My understanding is that flush will simply flush
data from
As long as you retain the returned futures somewhere, you can always
iterate over the futures after the flush completes and check for
success/failure. Would that work for you?
On Mon, Apr 27, 2015 at 08:53:36PM +, Roshan Naik wrote:
The important guarantee that is needed for a client
Batch failure is a bit meaningless, since in the same batch, some records
can succeed and others may fail.
To implement an error handling logic (usually different than retry, since
the producer has a configuration controlling retries), we recommend using
the callback option of Send().
Gwen
P.S
@Gwen
- A failure in delivery of one or more events in the batch (typical Flume
case) is considered a failure of the entire batch and the client
redelivers the entire batch.
- If clients want more fine grained control, alternative option is to
indicate which events failed in the return value of
On 4/27/15 2:59 PM, Gwen Shapira gshap...@cloudera.com wrote:
@Roshan - if the data was already written to Kafka, your approach will
generate LOTS of duplicates. I'm not convinced its ideal.
Only if the delivery failure rate is very high (i.e. short lived but very
frequent). This batch
Fine grained tracking of status of individual events is quite painful in
contrast to simply blocking on every batch. Old style Batched-sync mode
has great advantages in terms of simplicity and performance.
I may be missing something, but I'm not so convinced that it is that
painful/very
A couple of thoughts:
1. @Joel I agree it's not hard to use the new API but it definitely is more
verbose. If that snippet of code is being written across hundreds of
projects, that probably means we're missing an important API. Right now
I've only seen the one complaint, but it's worth finding
23 matches
Mail list logo