You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by "Gulia, Vikram" <Vi...@dish.com> on 2016/06/17 03:19:31 UTC

Message loss with kafka 0.8.2.2

Hi Users, I am facing message loss while using kafka v 0.8.2.2. Please see details below and help me if you can.

Issue: 2 messages produced to same partition one by one – Kafka producer returns same offset back which means message produced earlier is lost.<http://stackoverflow.com/questions/37732088/2-messages-produced-to-same-partition-one-by-one-message-1-overridden-by-next>

Details:
I have a unique problem which is happening like 50-100 times a day with message volume of more than 2 millions per day on the topic.I am using Kafka producer API 0.8.2.2 and I have 12 brokers (v 0.8.2.2) running in prod with replication of 4. I have a topic with 60 partitions and I am calculating partition for all my messages and providing the value in the ProducerRecord itself. Now, the issue -

Application creates 'ProducerRecord' using -

new ProducerRecord<String, String>(topic, 30, null, message1);
providing topic, value message1 and partition 30. Then application call the send method and future is returned -

// null is for callback
Future<RecordMetadata> future = producer.send(producerRecord. null);
Now, app prints the offset and partition value by calling get on Future and then getting values from RecordMetadata - this is what i get -

Kafka Response : partition 30, offset 3416092
Now, the app produce the next message - message2 to same partition -

new ProducerRecord<String, String>(topic, 30, null, message2);
and kafka response -

Kafka Response : partition 30, offset 3416092
I received the same offset again, and if I pull message from the offset of partition 30 using simple consumer, it ends up being the message2 which essentially mean i lost the message1.

Currently, the messages are produced using 10 threads each having its own instance of kafka producer (Earlier threads shared 1 Kafka producer but it was performing slow and we still had message loss).
I am using all default properties for producer except a few mentioned below, the message (String payload) size can be a few kbs to a 500 kbs. I am using acks value of 1.

value.serializer: org.apache.kafka.common.serialization.StringSerializer
key.serializer: org.apache.kafka.common.serialization.StringSerializer
bootstrap.servers: {SERVER VIP ENDPOINT}
acks: 1
batch.size: 204800
linger.ms: 10
send.buffer.bytes: 1048576
max.request.size: 10000000

What am i doing wrong here? Is there something I can look into or any producer property or server property I can tweak to make sure i don't lose any messages. I need some help here soon as I am losing some critical messages in production which is not good at all because as there is no exception given by Kafka Producer its even hard to find out the message lost unless downstream process reports it.

Thank you,
Vikram Gulia

Re: Message loss with kafka 0.8.2.2

Posted by Tom Crayford <tc...@heroku.com>.
Did you check if the controller is active in the cluster? If the controller
isn't active (there are known 0.8 bugs that can lead to this), then this
could cause this kind of data loss issue. I recommend upgrading to 0.9 ASAP.

Thanks

Tom Crayford
Heroku Kafka

On Friday, 17 June 2016, Gulia, Vikram <Vi...@dish.com> wrote:

> Hi Gerard, thanks for the reply. Few follow ups -
>
> 1. I can try setting acks = all but wouldn't it lead to performance hit (I
> am using sync produce thus response time will be more).
> 2. I will try unclean.leader.election.enable = false and update you if it
> helps.
> 3. Regarding your last point, I am confused. What I understood about kafka
> is that the producer client always retrieve the topic metadata and already
> knows who the leader for the topic is. And the producer client always
> sends the message to the leader only (the replicas replicate the message
> and send acknowledgements to the leader). Are you saying the producer
> client can send message to any broker who is not leader or to two or more
> brokers one of which may or may not be leader?
>
> Thank you,
> Vikram Gulia
>
>
>
>
> On 6/17/16, 12:29 AM, "Gerard Klijs" <gerard.klijs@dizzit.com
> <javascript:;>> wrote:
>
> >You could try set the acks to -1, so you wait for the produce to be
> >succesfull, until most other brokers also received the message. Another
> >thing you could try is set the unclean.leader.election.enable to false
> >(this is a setting on the broker).
> >I think what's happening now is that the message in your example is send
> >to
> >two different brokers, because one of them is not sending the record to
> >the
> >actual leader. Since you have set your acks to one, you wont see any error
> >in the producer, cause it succeeded in sending it to the broker. You most
> >likely will see some error on the broker, because it is not the leader.
> >
> >On Fri, Jun 17, 2016 at 5:19 AM Gulia, Vikram <Vikram.Gulia@dish.com
> <javascript:;>>
> >wrote:
> >
> >> Hi Users, I am facing message loss while using kafka v 0.8.2.2. Please
> >>see
> >> details below and help me if you can.
> >>
> >> Issue: 2 messages produced to same partition one by one ­ Kafka producer
> >> returns same offset back which means message produced earlier is lost.<
> >>
> >>
> http://stackoverflow.com/questions/37732088/2-messages-produced-to-same-p
> >>artition-one-by-one-message-1-overridden-by-next
> >> >
> >>
> >> Details:
> >> I have a unique problem which is happening like 50-100 times a day with
> >> message volume of more than 2 millions per day on the topic.I am using
> >> Kafka producer API 0.8.2.2 and I have 12 brokers (v 0.8.2.2) running in
> >> prod with replication of 4. I have a topic with 60 partitions and I am
> >> calculating partition for all my messages and providing the value in the
> >> ProducerRecord itself. Now, the issue -
> >>
> >> Application creates 'ProducerRecord' using -
> >>
> >> new ProducerRecord<String, String>(topic, 30, null, message1);
> >> providing topic, value message1 and partition 30. Then application call
> >> the send method and future is returned -
> >>
> >> // null is for callback
> >> Future<RecordMetadata> future = producer.send(producerRecord. null);
> >> Now, app prints the offset and partition value by calling get on Future
> >> and then getting values from RecordMetadata - this is what i get -
> >>
> >> Kafka Response : partition 30, offset 3416092
> >> Now, the app produce the next message - message2 to same partition -
> >>
> >> new ProducerRecord<String, String>(topic, 30, null, message2);
> >> and kafka response -
> >>
> >> Kafka Response : partition 30, offset 3416092
> >> I received the same offset again, and if I pull message from the offset
> >>of
> >> partition 30 using simple consumer, it ends up being the message2 which
> >> essentially mean i lost the message1.
> >>
> >> Currently, the messages are produced using 10 threads each having its
> >>own
> >> instance of kafka producer (Earlier threads shared 1 Kafka producer but
> >>it
> >> was performing slow and we still had message loss).
> >> I am using all default properties for producer except a few mentioned
> >> below, the message (String payload) size can be a few kbs to a 500 kbs.
> >>I
> >> am using acks value of 1.
> >>
> >> value.serializer: org.apache.kafka.common.serialization.StringSerializer
> >> key.serializer: org.apache.kafka.common.serialization.StringSerializer
> >> bootstrap.servers: {SERVER VIP ENDPOINT}
> >> acks: 1
> >> batch.size: 204800
> >> linger.ms: 10
> >> send.buffer.bytes: 1048576
> >> max.request.size: 10000000
> >>
> >> What am i doing wrong here? Is there something I can look into or any
> >> producer property or server property I can tweak to make sure i don't
> >>lose
> >> any messages. I need some help here soon as I am losing some critical
> >> messages in production which is not good at all because as there is no
> >> exception given by Kafka Producer its even hard to find out the message
> >> lost unless downstream process reports it.
> >>
> >> Thank you,
> >> Vikram Gulia
> >>
>
>

Re: Message loss with kafka 0.8.2.2

Posted by "Gulia, Vikram" <Vi...@dish.com>.
Hi Gerard, thanks for the reply. Few follow ups -

1. I can try setting acks = all but wouldn't it lead to performance hit (I
am using sync produce thus response time will be more).
2. I will try unclean.leader.election.enable = false and update you if it
helps.
3. Regarding your last point, I am confused. What I understood about kafka
is that the producer client always retrieve the topic metadata and already
knows who the leader for the topic is. And the producer client always
sends the message to the leader only (the replicas replicate the message
and send acknowledgements to the leader). Are you saying the producer
client can send message to any broker who is not leader or to two or more
brokers one of which may or may not be leader?

Thank you,
Vikram Gulia




On 6/17/16, 12:29 AM, "Gerard Klijs" <ge...@dizzit.com> wrote:

>You could try set the acks to -1, so you wait for the produce to be
>succesfull, until most other brokers also received the message. Another
>thing you could try is set the unclean.leader.election.enable to false
>(this is a setting on the broker).
>I think what's happening now is that the message in your example is send
>to
>two different brokers, because one of them is not sending the record to
>the
>actual leader. Since you have set your acks to one, you wont see any error
>in the producer, cause it succeeded in sending it to the broker. You most
>likely will see some error on the broker, because it is not the leader.
>
>On Fri, Jun 17, 2016 at 5:19 AM Gulia, Vikram <Vi...@dish.com>
>wrote:
>
>> Hi Users, I am facing message loss while using kafka v 0.8.2.2. Please
>>see
>> details below and help me if you can.
>>
>> Issue: 2 messages produced to same partition one by one ­ Kafka producer
>> returns same offset back which means message produced earlier is lost.<
>> 
>>http://stackoverflow.com/questions/37732088/2-messages-produced-to-same-p
>>artition-one-by-one-message-1-overridden-by-next
>> >
>>
>> Details:
>> I have a unique problem which is happening like 50-100 times a day with
>> message volume of more than 2 millions per day on the topic.I am using
>> Kafka producer API 0.8.2.2 and I have 12 brokers (v 0.8.2.2) running in
>> prod with replication of 4. I have a topic with 60 partitions and I am
>> calculating partition for all my messages and providing the value in the
>> ProducerRecord itself. Now, the issue -
>>
>> Application creates 'ProducerRecord' using -
>>
>> new ProducerRecord<String, String>(topic, 30, null, message1);
>> providing topic, value message1 and partition 30. Then application call
>> the send method and future is returned -
>>
>> // null is for callback
>> Future<RecordMetadata> future = producer.send(producerRecord. null);
>> Now, app prints the offset and partition value by calling get on Future
>> and then getting values from RecordMetadata - this is what i get -
>>
>> Kafka Response : partition 30, offset 3416092
>> Now, the app produce the next message - message2 to same partition -
>>
>> new ProducerRecord<String, String>(topic, 30, null, message2);
>> and kafka response -
>>
>> Kafka Response : partition 30, offset 3416092
>> I received the same offset again, and if I pull message from the offset
>>of
>> partition 30 using simple consumer, it ends up being the message2 which
>> essentially mean i lost the message1.
>>
>> Currently, the messages are produced using 10 threads each having its
>>own
>> instance of kafka producer (Earlier threads shared 1 Kafka producer but
>>it
>> was performing slow and we still had message loss).
>> I am using all default properties for producer except a few mentioned
>> below, the message (String payload) size can be a few kbs to a 500 kbs.
>>I
>> am using acks value of 1.
>>
>> value.serializer: org.apache.kafka.common.serialization.StringSerializer
>> key.serializer: org.apache.kafka.common.serialization.StringSerializer
>> bootstrap.servers: {SERVER VIP ENDPOINT}
>> acks: 1
>> batch.size: 204800
>> linger.ms: 10
>> send.buffer.bytes: 1048576
>> max.request.size: 10000000
>>
>> What am i doing wrong here? Is there something I can look into or any
>> producer property or server property I can tweak to make sure i don't
>>lose
>> any messages. I need some help here soon as I am losing some critical
>> messages in production which is not good at all because as there is no
>> exception given by Kafka Producer its even hard to find out the message
>> lost unless downstream process reports it.
>>
>> Thank you,
>> Vikram Gulia
>>


Re: Message loss with kafka 0.8.2.2

Posted by Gerard Klijs <ge...@dizzit.com>.
You could try set the acks to -1, so you wait for the produce to be
succesfull, until most other brokers also received the message. Another
thing you could try is set the unclean.leader.election.enable to false
(this is a setting on the broker).
I think what's happening now is that the message in your example is send to
two different brokers, because one of them is not sending the record to the
actual leader. Since you have set your acks to one, you wont see any error
in the producer, cause it succeeded in sending it to the broker. You most
likely will see some error on the broker, because it is not the leader.

On Fri, Jun 17, 2016 at 5:19 AM Gulia, Vikram <Vi...@dish.com> wrote:

> Hi Users, I am facing message loss while using kafka v 0.8.2.2. Please see
> details below and help me if you can.
>
> Issue: 2 messages produced to same partition one by one – Kafka producer
> returns same offset back which means message produced earlier is lost.<
> http://stackoverflow.com/questions/37732088/2-messages-produced-to-same-partition-one-by-one-message-1-overridden-by-next
> >
>
> Details:
> I have a unique problem which is happening like 50-100 times a day with
> message volume of more than 2 millions per day on the topic.I am using
> Kafka producer API 0.8.2.2 and I have 12 brokers (v 0.8.2.2) running in
> prod with replication of 4. I have a topic with 60 partitions and I am
> calculating partition for all my messages and providing the value in the
> ProducerRecord itself. Now, the issue -
>
> Application creates 'ProducerRecord' using -
>
> new ProducerRecord<String, String>(topic, 30, null, message1);
> providing topic, value message1 and partition 30. Then application call
> the send method and future is returned -
>
> // null is for callback
> Future<RecordMetadata> future = producer.send(producerRecord. null);
> Now, app prints the offset and partition value by calling get on Future
> and then getting values from RecordMetadata - this is what i get -
>
> Kafka Response : partition 30, offset 3416092
> Now, the app produce the next message - message2 to same partition -
>
> new ProducerRecord<String, String>(topic, 30, null, message2);
> and kafka response -
>
> Kafka Response : partition 30, offset 3416092
> I received the same offset again, and if I pull message from the offset of
> partition 30 using simple consumer, it ends up being the message2 which
> essentially mean i lost the message1.
>
> Currently, the messages are produced using 10 threads each having its own
> instance of kafka producer (Earlier threads shared 1 Kafka producer but it
> was performing slow and we still had message loss).
> I am using all default properties for producer except a few mentioned
> below, the message (String payload) size can be a few kbs to a 500 kbs. I
> am using acks value of 1.
>
> value.serializer: org.apache.kafka.common.serialization.StringSerializer
> key.serializer: org.apache.kafka.common.serialization.StringSerializer
> bootstrap.servers: {SERVER VIP ENDPOINT}
> acks: 1
> batch.size: 204800
> linger.ms: 10
> send.buffer.bytes: 1048576
> max.request.size: 10000000
>
> What am i doing wrong here? Is there something I can look into or any
> producer property or server property I can tweak to make sure i don't lose
> any messages. I need some help here soon as I am losing some critical
> messages in production which is not good at all because as there is no
> exception given by Kafka Producer its even hard to find out the message
> lost unless downstream process reports it.
>
> Thank you,
> Vikram Gulia
>