[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15261700#comment-15261700 ] ASF GitHub Bot commented on KAFKA-3236: --- Github user knusbaum closed the pull request at: https://github.com/apache/kafka/pull/934 > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15178334#comment-15178334 ] Jiangjie Qin commented on KAFKA-3236: - [~tgraves] I think we are on the same page that adding more configurations will provide finer control. As you might already noticed, the producer used to have many different configurations for timeout. And we are trying to move away from that and see if less configurations can provide good enough support for all the use cases. That does mean some applications have to do a little more work like catch exceptions. I think you might want to take a look at KAFKA-1835. This is the ticket I mentioned before that has a long discussion on this issue. > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15178094#comment-15178094 ] Thomas Graves commented on KAFKA-3236: -- So you are thinking you would rather not have this change unless it makes the send() guaranteed to be non-blocking? I agree with what you are saying that it might still block on the send() but this at least adds the option to not block on the full call and adds some flexibility. I would be fine with adding parameter to partitionsFor too, and I could see other possible use cases but that is also an API which is a bit more public then just the config. > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15176728#comment-15176728 ] Thomas Graves commented on KAFKA-3236: -- Yeah it would be possible to do it outside in our code its just not as nice. >From my understanding the metadata gather thread is ok to block. it needs the >metadata to do anything with that topic. Note this is one of our users I'm >representing here so I'm just going by what they are saying. I think its ok >to block here because the send() thread can still go ahead and send to the >topics it already has metadata on. Note that also send() only blocks waiting on the metadata if its not already cached, in waitOnMetadata: while(metadata.fetch().partitionsForTopic(topic) == null) { so by our metadata thread making sure it is there first I don't think send() will block waiting on metadata at all. Then I think the metadata gets updated when needed by internal kafka threads and doesn't block the sender. We do also expect the buffer full to be much more frequent give the metadata should be cached anyway. > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15176626#comment-15176626 ] Jiangjie Qin commented on KAFKA-3236: - [~tgraves] I remember there were some discussions about the requirement as you described. i.e. you want to block to first get the metadata but don't want to block afterwards. Unfortunately I forgot what exact the conclusion was from that discussion. For your use case, would the following solution work? 1. set {{max.block.ms = 0}} 2. Let the metadata discovery thread call partitionsFor() and catch the timeout exception in a while loop until it gets the metadata. 3. let the actual producing thread start produce. Given your metadata discovery thread has to call partitionsFor() on different producers, I assume you probably don't want it to be blocked on one of the producer either, right? I'm also wondering that if blocking on metadata refreshing during sending is rare and you are OK with that occasional short blocking behavior, do you expect buffer full to be frequent and you are not OK with that? > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15174110#comment-15174110 ] Jiangjie Qin commented on KAFKA-3236: - Hi Thomas, >From the description of the ticket, I think your confusion is from the >vagueness of the behavior when {{block.on.buffer.full}} is set to false. Is it >still a problem for you after we remove {{block.on.buffer.full}}? I agree that having separate setting of blocking gives finer tuning granularity, but we found it might not necessary for user. In your use case, if {{block.on.buffer.full = false}} and {{max.block.ms > 0}} is not a pure non-blocking mode because producer.send() can still block up to max.bock.ms, right? That is the exact rationale of having a single {{max.block.ms}} instead of decoupling time blocking on buffer full and metadata, because they provides the same guarantee from user's perspective: at most block for {{max.blocking.ms}}. > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15173834#comment-15173834 ] Thomas Graves commented on KAFKA-3236: -- Thanks for the response. I saw that the config were deprecated in the code but we were hoping to bring them back for this use case. Currently (Kafka 0.9) if block.on.buffer.full = false it blocks up to max.block.ms if either the metadata is unavailable of the buffer is full. The behavior we want (and is implemented in pr) is for it to throw BufferExhaustedException immediately instead of waiting the max.block.ms when the buffer is full. We still use max.block.ms for the metadata unavailable. Do you think this is a reasonable request or is there a reason not to decouple these? > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15172838#comment-15172838 ] Jiangjie Qin commented on KAFKA-3236: - [~tgraves] We had extensive discussion on the configurations in KIP-19. https://cwiki.apache.org/confluence/display/KAFKA/KIP-19+-+Add+a+request+timeout+to+NetworkClient We will actually deprecate the {{block.on.buffer.full}} configuration in the next release. The rationale behind this is that any synchronized call for producer is guaranteed to be returned in {{max.block.ms}}. That includes buffer full, metadata refresh and so on. In your use case, {{block.on.buffer.full = false}} and {{max.block.ms > 0}} is not a pure non-blocking mode because {{producer.send()}} can still block up to {{max.bock.ms}}, right? > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15172729#comment-15172729 ] ASF GitHub Bot commented on KAFKA-3236: --- Github user knusbaum closed the pull request at: https://github.com/apache/kafka/pull/934 > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15172730#comment-15172730 ] ASF GitHub Bot commented on KAFKA-3236: --- GitHub user knusbaum reopened a pull request: https://github.com/apache/kafka/pull/934 KAFKA-3236: Honor Producer Configuration "block.on.buffer.full" You can merge this pull request into a Git repository by running: $ git pull https://github.com/knusbaum/kafka KAFKA-3236-master Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/934.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #934 commit e8b63f045ff5d854d3d3d4bc0751cbba0d37d69f Author: Sanjiv RajDate: 2016-02-11T22:20:26Z [ADDHR-1240] Honor block.on.buffer.full producer configuration commit 50e4f01bc5f2b004533ac60bad5d8c396508e762 Author: Sanjiv Raj Date: 2016-02-18T18:43:20Z Fix failing producer integration test commit 836afe6159ee3b902a4c809cefd0345f61e6b026 Author: Kyle Nusbaum Date: 2016-02-17T17:05:38Z Updating config documentation. commit 6009eccb3a65c0a8cc8f441c89d902708475271e Author: Kyle Nusbaum Date: 2016-02-18T21:45:39Z Fixing TestUtils commit 5cf40a2065674d72298bdcce2a64ada1c6ca0163 Author: Kyle Nusbaum Date: 2016-02-19T16:50:46Z Merge branch 'trunk' of github.com:apache/kafka into KAFKA-3236-master commit 6e2d64ee8eedc90efff54fdd952c8d5f98a8b0d5 Author: Kyle Nusbaum Date: 2016-02-24T21:31:03Z Fixing config descriptions. > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15153204#comment-15153204 ] ASF GitHub Bot commented on KAFKA-3236: --- GitHub user knusbaum opened a pull request: https://github.com/apache/kafka/pull/934 KAFKA-3236: Honor Producer Configuration "block.on.buffer.full" You can merge this pull request into a Git repository by running: $ git pull https://github.com/knusbaum/kafka KAFKA-3236-master Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/934.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #934 commit e8b63f045ff5d854d3d3d4bc0751cbba0d37d69f Author: Sanjiv RajDate: 2016-02-11T22:20:26Z [ADDHR-1240] Honor block.on.buffer.full producer configuration commit 50e4f01bc5f2b004533ac60bad5d8c396508e762 Author: Sanjiv Raj Date: 2016-02-18T18:43:20Z Fix failing producer integration test commit 836afe6159ee3b902a4c809cefd0345f61e6b026 Author: Kyle Nusbaum Date: 2016-02-17T17:05:38Z Updating config documentation. commit 6009eccb3a65c0a8cc8f441c89d902708475271e Author: Kyle Nusbaum Date: 2016-02-18T21:45:39Z Fixing TestUtils > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15152555#comment-15152555 ] ASF GitHub Bot commented on KAFKA-3236: --- Github user knusbaum closed the pull request at: https://github.com/apache/kafka/pull/929 > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15150802#comment-15150802 ] Thomas Graves commented on KAFKA-3236: -- The reason I filed as a defect is because I think the documentation is confusing. Looking at the code I think its a bug with documentation and as you say this is actual improvement so I'll change it. > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Bug > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15150778#comment-15150778 ] ASF GitHub Bot commented on KAFKA-3236: --- GitHub user knusbaum opened a pull request: https://github.com/apache/kafka/pull/929 KAFKA-3236: Honor Producer Configuration "block.on.buffer.full" 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` You can merge this pull request into a Git repository by running: $ git pull https://github.com/knusbaum/kafka KAFKA-3236 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/kafka/pull/929.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #929 commit 8cc454d2bc9d2be1b0ee916a5dd76042b79954fb Author: Sanjiv RajDate: 2016-02-11T22:20:26Z [ADDHR-1240] Honor block.on.buffer.full producer configuration > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Bug > 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)
[jira] [Commented] (KAFKA-3236) Honor Producer Configuration "block.on.buffer.full"
[ https://issues.apache.org/jira/browse/KAFKA-3236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15144778#comment-15144778 ] Ismael Juma commented on KAFKA-3236: I'd mark this as an "Improvement" instead of "Bug" as the current behaviour is by design as far as I know. > Honor Producer Configuration "block.on.buffer.full" > --- > > Key: KAFKA-3236 > URL: https://issues.apache.org/jira/browse/KAFKA-3236 > Project: Kafka > Issue Type: Bug > 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)