[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15176573#comment-15176573 ]
Thomas Graves commented on KAFKA-3236: -------------------------------------- [~becket_qin] So if your concern that this change doesn't make it fully non-blocking since it could potentially still block on the metadata query? You are correct in what you say that block.on.buffer.full = false and max.block.ms > 0 its not purely non-blocking and that is actually what we want. We want to be able to control the metadata fetch block time separate from if the buffer is full. I expect most users to leave block.on.buffer.full=true. We have a use case where we have a process sending to multiple different Kafka clusters. It has one thread that is getting the metadata for all the topics from all the clusters and then another thread that is doing the send() on the produce. The second thread sends to multiple kafka clusters so we don't want it to block if at all possible. It using a single instance of a producer per topic. Since its a single instance we can't set the max.blocking.ms to 0 because then the thread getting the metadata wouldn't block which we want it to. Note the thread getting metadata is calling partitionsFor(). With this patch and block.on.buffer.full = false and max.block.ms > 0 it is still possible that the send() blocks but I consider this a rare/special case. We don't want it to block if the buffer is full though. An alternative to this would be to add another interface for partitionsFor() that would take in the maxBlockTimeMs rather then using it from the config. Then the thread doing the send could set max.blocking.ms to 0 and all is well. Thoughts or other ideas? > 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)