[ 
https://issues.apache.org/jira/browse/KAFKA-6699?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16607642#comment-16607642
 ] 

Matthias J. Sax commented on KAFKA-6699:
----------------------------------------

I think I need to explain the configs again. Replication factor should be 
clear. If you create a topic with replication factor X and don't have enough 
brokers, you cannot create the topic. However, if you have X brokers and create 
a topic with replication factor X and later a broker fails, the topic is of 
course still there and can potentially write to it.

For each topic, you can configure an "desired" in-sync.replica factor – this 
defines how many in-sync-replicas you want in order to allow writing into the 
topic. If you have less isr than configured, the broker rejects writes.

What you see via _describe_ is the current list of ISR – note, that a check 
when the client start does not make sense, because even if the check is 
successful, the ISR can change after the check but before the write. Thus, the 
check are performed during writing. Also note, that the replication factor only 
indirectly determines if you can write to a topic or not – you need to consider 
both, replication factor and ISR setting to reason about it.

If you have 2 broker with replication factor 2 and configures isr=2 and one 
broker goes down, you have only one isr left and thus write are rejected. You 
can still consume the topic though.

Does this make sense?

Thus, if  you want to be able to write to a topic even if one broker is down, 
your isr config must be smaller than your replication factor config. The 
difference between both determines how many broker failures you can tolerate 
and still be able to write to the topic.

> When one of two Kafka nodes are dead, streaming API cannot handle messaging
> ---------------------------------------------------------------------------
>
>                 Key: KAFKA-6699
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6699
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.11.0.2
>            Reporter: Seweryn Habdank-Wojewodzki
>            Priority: Major
>
> Dears,
> I am observing quite often, when Kafka Broker is partly dead(*), then 
> application, which uses streaming API are doing nothing.
> (*) Partly dead in my case it means that one of two Kafka nodes are out of 
> order. 
> Especially when disk is full on one machine, then Broker is going in some 
> strange state, where streaming API goes vacations. It seems like regular 
> producer/consumer API has no problem in such a case.
> Can you have a look on that matter?



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to