I have tested using "async" producer with "required.ack=-1" and got really
good performance.

We have not used async producer much previously, any potential dataloss
when a broker goes down? For example, when a broker goes down, does
producer resend all the messages in a batch?


On Wed, Dec 17, 2014 at 1:16 PM, Xiaoyu Wang <xw...@rocketfuel.com> wrote:
>
> Thanks Jun.
>
> We have tested our producer with the different required.ack config. Even
> with the required.ack=1, the producer is > 10 times slower than with
> required.ack=0. Does this confirm with your  testing?
>
> I saw the presentation of LinkedIn Kafka SRE. Wondering what configuration
> you guys have at LinkedIn to guarantee zero data loss.
>
> Thanks again and really appreciate your help!
>
> On Tue, Dec 16, 2014 at 9:50 PM, Jun Rao <j...@confluent.io> wrote:
>>
>> replica.lag.max.messages only controls when a replica should be dropped
>> out
>> of the in-sync replica set (ISR). For a message to be considered
>> committed,
>> it has to be added to every replica in ISR. When the producer uses ack=-1,
>> the broker waits until the produced message is committed before
>> acknowledging the client. So in the case of a clean leader election (i.e.,
>> there is at least one remaining replica in ISR), no committed messages are
>> lost. In the case of an unclean leader election, the number of messages
>> that can be lost depends on the state of the replicas and it's possible to
>> lose more than replica.lag.max.messages messages.
>>
>> We do have the lag jmx metric per replica (see
>> http://kafka.apache.org/documentation.html#monitoring).
>>
>> Thanks,
>>
>> Jun
>>
>> On Sun, Dec 14, 2014 at 7:20 AM, Xiaoyu Wang <xw...@rocketfuel.com>
>> wrote:
>> >
>> > Hello,
>> >
>> > If I understand it correctly, when the number of messages a replica is
>> > behind from the leader is < replica.lag.max.messages, the replica is
>> > considered in sync with the master and are eligible for leader election.
>> >
>> > This means we can lose at most replica.lag.max.messages messages during
>> > leader election, is it? We can set the replica.lag.max.messages to be
>> very
>> > low, but then we may result in unclean leader election, so still we can
>> > lose data.
>> >
>> > Can you recommend some way to prevent data loss? We have tried setting
>> > require ack from all replicas, but that slows down producer
>> significantly.
>> >
>> > In addition, do we have metrics about how far each replica is behind? If
>> > not, can we add them.
>> >
>> >
>> > Thanks,
>> >
>>
>

Reply via email to