Hi, a) If there are no brokers available, producer send() will block up to max_block_ms (default: 60000) after which it will throw a TimeoutException. No retries in this case. by-default console producer uses Callback and just logs the error after max_block_ms.
b) Current behavior is, If metadata is not available and producer batches will Timedout after request.timeout.ms. retries are applicable to failed produce requests with transient/retriable exceptions. Thanks, On Tue, Mar 27, 2018 at 3:56 PM, Sandor Murakozi <smurak...@gmail.com> wrote: > Hi All, > > I have a question about intended producer behavior if the broker is lost - > do I see a bug or the code works as specified? > > What I do and see using trunk: > > *a) No message send timeout at all if there is no available broker* > - no broker is started > - consoleproducer is started using --broker-list localhost:9092 --topic > test --request-timeout-ms 2000 --property retries=3 > - I try to send some messages. The request doesn't time out, I just see an > endless amount of log entries like this: > [2018-03-27 10:04:42,084] WARN [Producer clientId=console-producer] > Connection to node -1 could not be established. Broker may not be > available. (org.apache.kafka.clients.NetworkClient) > > I would expect that the producer eventually times out, preferably retrying > sending 3 times. > > *b) No retries if the last broker disappears after successful > communication* > - broker is started > - consoleproducer is started with the same config as in a) > - 1 message is sent successfully > - broker is stopped - producer starts to add log entries like > [2018-03-27 10:09:11,597] WARN [Producer clientId=console-producer] > Connection to node 0 could not be established. Broker may not be available. > (org.apache.kafka.clients.NetworkClient) > - I try to send another message > - it times out after ~2 seconds > [2018-03-27 10:09:17,442] ERROR Error when sending message to topic test > with key: null, value: 1 bytes with error: > (org.apache.kafka.clients.producer.internals.ErrorLoggingCallback) > org.apache.kafka.common.errors.TimeoutException: Expiring 1 record(s) for > test-0: 2018 ms has passed since batch creation plus linger time > - the producer *does not retry* sending > > Some debugging have shown that after the connection was broken the producer > will attempt to fetch metadata. As there are no more available brokers this > will not be possible. > In this case it does not retry fetching of the metadata but it considers > the batch expired. > > My expectation would be that the producer times out but it retries sending. > > I've spent quite some time trying to derive what's the correct behavior > according to KIP-19 > <https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 19+-+Add+a+request+timeout+to+NetworkClient> > and > jiras/pull requests related to the topic (e.g. KAFKA-2805 > <https://issues.apache.org/jira/browse/KAFKA-2805> and KAFKA-3388 > <https://issues.apache.org/jira/browse/KAFKA-3388>) but it's still not > 100% > clear to me if the code works as intended. > > Could someone help me decide this question? > > Many thanks in advance, > Sandor >