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

Jiangjie Qin commented on KAFKA-3236:
-------------------------------------

[~tgraves] Got it. It is true that to achieve a purely non-blocking behavior is 
more involved due to the first metadata fetch. It sounds reasonable to keep 
block.on.buffer.full. That said, I'm not sure if keeping block.on.buffer.full 
completely solve the problem in general. 

If max.block.ms > 0 and block.on.buffer.full=false, User might still be blocked 
on send(). To have a guaranteed non-blocking behavior, there are more 
requirements such as users have to send to a static list of topics and those 
topics cannot be deleted from the server.
If max.block.ms = 0, regardless of block.on.buffer.full setting, users will be 
guaranteed for non-blocking send(), but the metadata prefetch needs additional 
work.

So either way, there are some additional works or requirements. I am not sure 
which way is better. Maybe having a timeout argument for partitionsFor() will 
solve the problem, but it seems weird to have max.block.ms and the timeout 
argument at the same time.

> Honor Producer Configuration "block.on.buffer.full"
> ---------------------------------------------------
>
>                 Key: KAFKA-3236
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3236
>             Project: Kafka
>          Issue Type: Improvement
>          Components: producer 
>    Affects Versions: 0.9.0.0
>            Reporter: Thomas Graves
>            Assignee: Thomas Graves
>
> In Kafka-0.9, "max.block.ms" is used to control how long the following 
> methods will block.
> KafkaProducer.send() when
>    * Buffer is full
>    * Metadata is unavailable
> KafkaProducer.partitionsFor() when
>    * Metadata is unavailable
> However when "block.on.buffer.full" is set to false, "max.block.ms" is in 
> effect whenever a buffer is requested/allocated from the Producer BufferPool. 
> Instead it should throw a BufferExhaustedException without waiting for 
> "max.block.ms"
> This is particulary useful if a producer application does not wish to block 
> at all on KafkaProducer.send() . We avoid waiting on KafkaProducer.send() 
> when metadata is unavailable by invoking send() only if the producer instance 
> has fetched the metadata for the topic in a different thread using the same 
> producer instance. However "max.block.ms" is still required to specify a 
> timeout for bootstrapping the metadata fetch.
> We should resolve this limitation by decoupling "max.block.ms" and 
> "block.on.buffer.full".
>    * "max.block.ms" will be used exclusively for fetching metadata when    
> "block.on.buffer.full" = false (in pure non-blocking mode )
>    * "max.block.ms" will be applicable to both fetching metadata as well as 
> buffer allocation when "block.on.buffer.full = true



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

Reply via email to