-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/30158/#review69138
-----------------------------------------------------------



clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
<https://reviews.apache.org/r/30158/#comment113759>

    Here are some comments on this file from Ewan:
    * No need to use the object forms of primitive types, change Boolean -> 
boolean, Long -> long, etc.
    * initialized should be an AtomicBoolean or volatile boolean since it's 
read/written from different threads
    * Error handling when waiting for the Future to finish seems wrong -- if 
there is an exception, we probably want to pass it along/throw another one to 
indicate the problem to the caller. Currently it just falls through and then 
only throws an exception when send() is called, so the error ends up 
disconnected from the source of the problem. It seems like it would be better 
to just handle the error immediately.
    * Similarly, I don't think send() should check initialized if 
preinitialization is handled in the constructor -- if failure to preinitialize 
also threw an exception, then it would be impossible to call send() unless 
preinitialization was complete.
    * If you follow the above approach, you can avoid making initialized a 
field in the class. It would only need to be a local variable since it would 
only be used in the constructor.
    * Do we even need the ExecutorService? Since the thread creating the 
producer is going to block by calling Future.get(), what does having the 
executor accomplish?
    * initializeProducer() doesn't need a return value since only ever returns 
true.



clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
<https://reviews.apache.org/r/30158/#comment113760>

    Comments from Ewan:
    
    * Config has a getList() method and ConfigDef has a LIST type. Use those 
for pre.initialize.topics instead of parsing the list yourself.
    * I think the docstrings could be better, e.g.:
    pre.initialize.topics: "List of topics to preload metadata for when 
creating the producer so subsequent calls to send are guaranteed not to block. 
If metadata for these topics cannot be loaded within 
<code>pre.initialize.timeout.ms</code> milliseconds, the producer constructor 
will throw an exception."
    pre.initialize.timeout.ms:  "The producer blocks when sending the first 
message to a topic if metadata is not yet available for that topic. When this 
configuration is greater than 0, metadata for the topics specified by 
<code>pre.initialize.topics</code> are prefetched during construction, throwing 
an exception after <code>pre.initialize.timeout.ms</code> milliseconds if the 
metadata has not been populated."


- Paul Pearcy


On Jan. 22, 2015, 7:04 a.m., Paul Pearcy wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/30158/
> -----------------------------------------------------------
> 
> (Updated Jan. 22, 2015, 7:04 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1835
>     https://issues.apache.org/jira/browse/KAFKA-1835
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-1835 - New producer updates to make blocking behavior explicit
> 
> 
> Diffs
> -----
> 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> fc71710dd5997576d3841a1c3b0f7e19a8c9698e 
>   clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java 
> 8b3e565edd1ae04d8d34bd9f1a41e9fa8c880a75 
>   core/src/test/scala/integration/kafka/api/ProducerBlockingTest.scala 
> PRE-CREATION 
>   core/src/test/scala/unit/kafka/utils/TestUtils.scala 
> ac15d34425795d5be20c51b01fa1108bdcd66583 
> 
> Diff: https://reviews.apache.org/r/30158/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Paul Pearcy
> 
>

Reply via email to