> On April 7, 2014, 6:13 p.m., Timothy Chen wrote:
> > clients/src/main/java/kafka/clients/consumer/Consumer.java, line 61
> > <https://reviews.apache.org/r/19731/diff/1/?file=538460#file538460line61>
> >
> >     I wonder if it's worth adding more comments about the poll behavior 
> > here, about what conditions will it return from poll (ie: when timeout hit, 
> > first topic data available, etc).

Makes sense. Improved that.


> On April 7, 2014, 6:13 p.m., Timothy Chen wrote:
> > clients/src/main/java/kafka/clients/consumer/ConsumerConfig.java, line 55
> > <https://reviews.apache.org/r/19731/diff/1/?file=538461#file538461line55>
> >
> >     What happens when you explicitly call commit when auto commit is 
> > enabled?

That is valid. It commits synchronously or asynchronously depending on whether 
you use commit() or commitAsync().


> On April 7, 2014, 6:13 p.m., Timothy Chen wrote:
> > clients/src/main/java/kafka/clients/consumer/ConsumerConfig.java, line 109
> > <https://reviews.apache.org/r/19731/diff/1/?file=538461#file538461line109>
> >
> >     Don't we throw an exception instead of being stuck? Being stuck doesn't 
> > sounds like a acceptable behavior

That's true. This is not ideal. Currently we do not have the ability to stream 
a single large message and if a message is larger than the largest fetch buffer 
the consumer can allocate (total.memory.bytes), then the poll() would continue 
timing out and returning no messages for that partition. So I see that 
fetch.memory.bytes is not required since we should automatically size the fetch 
buffer to accommodate large messages. However, if the message is larger than 
that, there are 2 choices -
1. Throw an exception from poll() indicating a single large message that is 
larger than (total.memory.bytes)
2. Keep returning no messages from that partition, effectively building up a 
lag on that partition that the user can only find out about through monitoring.


> On April 7, 2014, 6:13 p.m., Timothy Chen wrote:
> > clients/src/main/java/kafka/clients/consumer/KafkaConsumer.java, line 113
> > <https://reviews.apache.org/r/19731/diff/1/?file=538464#file538464line113>
> >
> >     How does one actually get the list of partitions that returns failure?

Here is one way - process() is implemented by the user, so one can keep track 
of the partitions for which the processing failed and let failedPartitions() 
return that.


> On April 7, 2014, 6:13 p.m., Timothy Chen wrote:
> > clients/src/main/java/kafka/clients/consumer/MockConsumer.java, line 51
> > <https://reviews.apache.org/r/19731/diff/1/?file=538465#file538465line51>
> >
> >     Looks like the mock consumer doesn't follow the mutually exclusive rule.

Good point, this needs to be fixed.


> On April 7, 2014, 6:13 p.m., Timothy Chen wrote:
> > clients/src/main/java/kafka/clients/consumer/KafkaConsumer.java, line 121
> > <https://reviews.apache.org/r/19731/diff/1/?file=538464#file538464line121>
> >
> >     Potentially in this sample code you could throw an exception in your 
> > catch block and never call close. Is that ok too?

Not ok, but I also don't want to make it excessively complex. I added a try 
catch and a break to quit the loop


- Neha


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


On March 27, 2014, 4:16 p.m., Neha Narkhede wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/19731/
> -----------------------------------------------------------
> 
> (Updated March 27, 2014, 4:16 p.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-1328
>     https://issues.apache.org/jira/browse/KAFKA-1328
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> Added license headers
> 
> 
> Cleaned javadoc for ConsumerConfig
> 
> 
> Fixed minor indentation in ConsumerConfig
> 
> 
> Improve docs on ConsumerConfig
> 
> 
> 1. Added ClientUtils 2. Added basic constructor implementation for 
> KafkaConsumer
> 
> 
> Improved MockConsumer
> 
> 
> Chris's feedback and also consumer rewind example code
> 
> 
> Added commit() and commitAsync() APIs to the consumer and updated docs and 
> examples to reflect that
> 
> 
> 1. Added consumer usage examples to javadoc 2. Changed signature of APIs that 
> accept or return offsets from list of offsets to map of offsets
> 
> 
> Improved example for using ConsumerRebalanceCallback
> 
> 
> Improved example for using ConsumerRebalanceCallback
> 
> 
> Included Jun's review comments and renamed positions to seek. Also included 
> position()
> 
> 
> Changes to javadoc for positions()
> 
> 
> Changed the javadoc for ConsumerRebalanceCallback
> 
> 
> Changing unsubscribe to also take in var args for topic list
> 
> 
> Incorporated first round of feedback from Jay, Pradeep and Mattijs on the 
> mailing list
> 
> 
> Updated configs
> 
> 
> Javadoc for consumer complete
> 
> 
> Completed docs for Consumer and ConsumerRebalanceCallback. Added MockConsumer
> 
> 
> Added the initial interfaces and related documentation for the consumer. More 
> docs required to complete the public API
> 
> 
> Diffs
> -----
> 
>   clients/src/main/java/kafka/clients/consumer/Consumer.java PRE-CREATION 
>   clients/src/main/java/kafka/clients/consumer/ConsumerConfig.java 
> PRE-CREATION 
>   clients/src/main/java/kafka/clients/consumer/ConsumerRebalanceCallback.java 
> PRE-CREATION 
>   clients/src/main/java/kafka/clients/consumer/ConsumerRecord.java 
> PRE-CREATION 
>   clients/src/main/java/kafka/clients/consumer/KafkaConsumer.java 
> PRE-CREATION 
>   clients/src/main/java/kafka/clients/consumer/MockConsumer.java PRE-CREATION 
>   clients/src/main/java/kafka/common/TopicPartitionOffset.java PRE-CREATION 
>   clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java 
> 1ff9174870a8c9cd97eb6655416edd4124377b0e 
>   clients/src/main/java/org/apache/kafka/common/utils/ClientUtils.java 
> PRE-CREATION 
>   
> clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java
>  PRE-CREATION 
> 
> Diff: https://reviews.apache.org/r/19731/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Neha Narkhede
> 
>

Reply via email to