Hi Todd, thanks very much for the explanations here and in the previous mail - helps a lot!
(I assumed the callback would provide an exception when a message could not be sent - I observe the callback being called for every message to the up broker, but yes, maybe the developers can clarify how to detect a broker being down) Jan > On 22 Nov 2015, at 21:42, Todd Palino <tpal...@gmail.com> wrote: > > Hopefully one of the developers can jump in here. I believe there is a > future you can use to get the errors back from the producer. In addition, > you should check the following configs on the producer: > > request.required.acks - this controls whether or not your producer is going > to wait for an acknowledgement from the broker, and how many brokers it > waits for > request.timeout.ms - how long the producer waits to satisfy the acks > setting before marking the request failed > retry.backoff.ms - how long the producer waits between retries > message.send.max.retries - the maximum number of retries the producer will > attempt a failed request > > -Todd > > > On Sun, Nov 22, 2015 at 12:31 PM, Jan Algermissen < > algermissen1...@icloud.com> wrote: > >> Hi Todd, >> >> yes, correct - thanks. >> >> However, what I am not getting is that the KafkaProducer (see my other >> mail from today) silently accepts the messages and fills them up in the >> buffer until it is exhausted instead of saying that the broker is not >> reachable. >> >> IOW, it seems from an application perspective I am unable to detect that >> messages are not being sent out. Is this normal behavior and I am simply >> doing something wrong or could it be a producer bug? >> >> Jan >> >> Config and code again: >> >> ProducerConfig.BOOTSTRAP_SERVERS_CONFIG -> brokers, >> ProducerConfig.RETRIES_CONFIG -> "0", >> ProducerConfig.ACKS_CONFIG -> "1", >> ProducerConfig.COMPRESSION_TYPE_CONFIG -> "none", >> ProducerConfig.TIMEOUT_CONFIG -> new Integer(30000), >> // ProducerConfig.BATCH_SIZE_CONFIG -> new Integer(16384), >> ProducerConfig.BATCH_SIZE_CONFIG -> new Integer(10), >> ProducerConfig.BUFFER_MEMORY_CONFIG -> new Integer(66554432), >> ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG -> new java.lang.Boolean(false), >> ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> >> "org.apache.kafka.common.serialization.StringSerializer", >> ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> >> "org.apache.kafka.common.serialization.StringSerializer" >> >> >> .... >> >> kafkaProducer.send(new ProducerRecord[String,String](topic, key, data),new >> Callback { >> def onCompletion(recordMetadata: RecordMetadata, e: Exception):Unit = { >> if(e != null) { >> logger.error(s"Could not send $data",e) >> } >> logger.info("The offset of the record we just sent is: " + >> recordMetadata.offset()) >> () >> } >> >> }) >> >> >>> On 22 Nov 2015, at 20:49, Todd Palino <tpal...@gmail.com> wrote: >>> >>> Replicas and Isr are both a comma separated list of broker IDs. So in >> this >>> output, I am seeing that you have two Kafka brokers with IDs 1 and 2. You >>> have a topic, capture, with 16 partitions at replication factor 1 (1 >>> replica per partition). The broker with ID 2 is not online, which is why >> it >>> shows in the Replica list for some partitions (meaning that it is >> assigned >>> to be a replica), but not in the Isr list (which would indicate that it >> is >>> currently in-sync). >>> >>> The Leader field is the broker ID which is currently the leader for that >>> partition. For the partitions that are assigned to broker 1, you see that >>> broker 1 is the leader. For the partitions that are assigned to broker 2, >>> the leader is listed as -1, which indicates that there is no available >>> leader. These partitions are considered offline and cannot be produced to >>> or consumed from. When broker 2 comes back online, the controller will >>> perform an unclean leader election and select broker 2 (the only replica >>> available) as the leader for those partitions. >>> >>> -Todd >>> >>> >>> On Sun, Nov 22, 2015 at 11:39 AM, Jan Algermissen < >>> algermissen1...@icloud.com> wrote: >>> >>>> Hi, >>>> >>>> I have a topic with 16 partitions that shows the following output for >>>> >>>> kafka-topics.sh --zookeeper xxxxx:2181 --topic capture --describe >>>> >>>> Can anyone explain to me what the difference in replicas means and what >>>> Leader of -1 means? >>>> >>>> In the logs of my producer I see that no messages seem to be sent to the >>>> partitions with '-1' and th eproducer buffer becomes exhausted afetr a >>>> while (maybe that is related?) >>>> >>>> Jan >>>> >>>> Topic:capture PartitionCount:16 ReplicationFactor:1 >>>> Configs: >>>> >>>> Topic: capture Partition: 0 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 1 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 2 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 3 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 4 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 5 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 6 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 7 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 8 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 9 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 10 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 11 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 12 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 13 Leader: 1 Replicas: 1 >>>> Isr: 1 >>>> Topic: capture Partition: 14 Leader: -1 Replicas: 2 >>>> Isr: >>>> Topic: capture Partition: 15 Leader: 1 Replicas: 1 >>>> Isr: 1 >> >>