@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  producer.send(list<>)


@Joel
  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.
  Imagine a simple use case of client simply reading a directory of log
files and splitting them into log messages/events and pushing them through
kafka. Becomes more complex when all this tracking data needs to be
persisted to accommodate for client restarts/crashes. Tracking a simple
current line number and file name is easy to programming with and persist
to accommodateÅ  as opposed start/end position of each log message in the
file. 


-roshan





On 4/27/15 2:07 PM, "Joel Koshy" <jjkosh...@gmail.com> wrote:

>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 +0000, Roshan Naik wrote:
>> 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 all threads (correct me if I am wrong).
>> 
>> -roshan
>> 
>> 
>> 
>> On 4/27/15 1:36 PM, "Joel Koshy" <jjkosh...@gmail.com> wrote:
>> 
>> >This sounds like flush:
>> 
>>>https://cwiki.apache.org/confluence/display/KAFKA/KIP-8+-+Add+a+flush+me
>>>th
>> >od+to+the+producer+API
>> >
>> >which was recently implemented in trunk.
>> >
>> >Joel
>> >
>> >On Mon, Apr 27, 2015 at 08:19:40PM +0000, Roshan Naik wrote:
>> >> 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 do a future.get() after every
>> >>Producer.send(). I changed the new
>> >>o.a.k.clients.tools.ProducerPerformance tool to asses the perf of this
>> >>mode of operation. May not be surprising that it much slower than the
>> >>async mode... hard t push it beyond 4MB/s.
>> >> 
>> >> The 0.8.1 Scala based producer API supported a batched sync mode via
>> >>Producer.send( List<KeyedMessage> ) . My measurements show that it was
>> >>able to approach (and sometimes exceed) the old async speeds...
>>266MB/s
>> >> 
>> >> 
>> >> Supporting this batched sync mode is very critical for streaming
>> >>clients (such as flume for example) that need delivery guarantees.
>> >>Although it can be done with Async mode, it requires additional book
>> >>keeping as to which events are delivered and which ones are not. The
>> >>programming model becomes much simpler with the batched sync mode.
>> >>Client having to deal with one single future.get() helps performance
>> >>greatly too as I noted.
>> >> 
>> >> Wanted to propose adding this as an enhancement to the new Producer
>>API.
>> >
>> 
>

Reply via email to