[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627485#comment-14627485 ]
Jiangjie Qin commented on KAFKA-1835: ------------------------------------- My example for use case (3) is probably a bad one... However, partitionsFor() might also be called independently but not only for sending data, right? So if user wants a completely non-blocking API, letting partionsFor() to block would not work. I think the main benefit to have partitionsFor() to return a future is it can cover both use case (2) and (3) dynamically, which have opposite blocking requirements on partitonsFor(). Not sure if it is just my imagined use case, but let's say in use case (3), user also wants to pre-initialize the metadata but after that they don't want to block on any call. Then they can do something like this: {code} ... partitionsFor("foo").get(); try { send(); ... // some other logic Future<..> partitionInfo = partitionsFor(); // if (partitionInfo.isDone()) { ... // do something with partition info. } ... } catch (TimeoutException te){ // handle exception } {code} So I guess returning a future gives user the max freedom. That said, throwing exception for non-blocking calls and letting user to handle it also sounds reasonable to me. Functionality wise, I don't think there is much difference between the above proposal and having an additional configuration of topic.metadata.preinit.list. The benefit of the configuration approach may be it is a smaller and backward compatible change. Do you see any issue on adding that configuration? > Kafka new producer needs options to make blocking behavior explicit > ------------------------------------------------------------------- > > Key: KAFKA-1835 > URL: https://issues.apache.org/jira/browse/KAFKA-1835 > Project: Kafka > Issue Type: Improvement > Components: clients > Affects Versions: 0.8.2.0, 0.8.3, 0.9.0 > Reporter: Paul Pearcy > Fix For: 0.8.3 > > Attachments: KAFKA-1835-New-producer--blocking_v0.patch, > KAFKA-1835.patch > > Original Estimate: 504h > Remaining Estimate: 504h > > The new (0.8.2 standalone) producer will block the first time it attempts to > retrieve metadata for a topic. This is not the desired behavior in some use > cases where async non-blocking guarantees are required and message loss is > acceptable in known cases. Also, most developers will assume an API that > returns a future is safe to call in a critical request path. > Discussing on the mailing list, the most viable option is to have the > following settings: > pre.initialize.topics=x,y,z > pre.initialize.timeout=x > > This moves potential blocking to the init of the producer and outside of some > random request. The potential will still exist for blocking in a corner case > where connectivity with Kafka is lost and a topic not included in pre-init > has a message sent for the first time. > There is the question of what to do when initialization fails. There are a > couple of options that I'd like available: > - Fail creation of the client > - Fail all sends until the meta is available > Open to input on how the above option should be expressed. > It is also worth noting more nuanced solutions exist that could work without > the extra settings, they just end up having extra complications and at the > end of the day not adding much value. For instance, the producer could accept > and queue messages(note: more complicated than I am making it sound due to > storing all accepted messages in pre-partitioned compact binary form), but > you're still going to be forced to choose to either start blocking or > dropping messages at some point. > I have some test cases I am going to port over to the Kafka producer > integration ones and start from there. My current impl is in scala, but > porting to Java shouldn't be a big deal (was using a promise to track init > status, but will likely need to make that an atomic bool). -- This message was sent by Atlassian JIRA (v6.3.4#6332)