You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Luca Bruno (JIRA)" <ji...@apache.org> on 2016/05/10 16:40:12 UTC

[jira] [Created] (KAFKA-3686) Kafka producer is not fault tolerant

Luca Bruno created KAFKA-3686:
---------------------------------

             Summary: Kafka producer is not fault tolerant
                 Key: KAFKA-3686
                 URL: https://issues.apache.org/jira/browse/KAFKA-3686
             Project: Kafka
          Issue Type: Bug
    Affects Versions: 0.9.0.1
            Reporter: Luca Bruno


*Setup*

I have a cluster of 3 kafka server, a topic with 12 partitions with replica 2, and a zookeeper cluster of 3 nodes.

Producer config:

{code}
 props.put("bootstrap.servers", "k1:9092,k2:9092,k3:9092");
 props.put("acks", "1");
 props.put("batch.size", 16384);
 props.put("retries", 3);
 props.put("buffer.memory", 33554432);
 props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
 props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer");
{code}

Producer code:

{code}
 Producer<String, String> producer = new KafkaProducer<>(props);
 for(int i = 0; i < 10; i++) {
     Future<RecordMetadata> f = producer.send(new ProducerRecord<String, String>("topic", null, Integer.toString(i)));
     f.get();
 }
{code}

*Problem*

Cut the network between the producer (p1) and one of the kafka servers (say k1).

The cluster is healthy, hence the kafka bootstrap tells the producer that there are 3 kafka servers (as I understood it), and the leaders of the partitions of the topic.

So the producer will send messages to all of the 3 leaders for each partition. If the leader happens to be k1 for a message, the producer raises the following exception:

{code}
Exception in thread "main" java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.TimeoutException: Batch Expired
	at org.apache.kafka.clients.producer.internals.FutureRecordMetadata.valueOrError(FutureRecordMetadata.java:56)
	at org.apache.kafka.clients.producer.internals.FutureRecordMetadata.get(FutureRecordMetadata.java:43)
	at org.apache.kafka.clients.producer.internals.FutureRecordMetadata.get(FutureRecordMetadata.java:25)
	at Test.main(Test.java:25)
Caused by: org.apache.kafka.common.errors.TimeoutException: Batch Expired
{code}

In theory, the application should handle the failure. In practice, messages are getting lost, even though there are other 2 leaders available for writing.

*What I expected*

Given the client is automatically deciding the hashing / round robin schema for the partition, I would say it's not very important which partition is the message being sent to.
I expect the client to handle the failure, and send the message to a partition of a different leader.

Neither kafka-clients nor rdkafka handle this failure. Given those are the main client libraries being used for kafka as far as I know, I find it a serious problem in terms of fault tolerance.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)