----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/19731/#review39700 -----------------------------------------------------------
clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72291> subscribe(topic, partitions) => subscribe(partitions) clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72287> Just wanted to confirm again that we agree to expose TopicPartition to users? Though it is already in common package, it currently only used internally. clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72288> Does this mean subscribe(TopicPartition) followed by an unsubscribe(String) will also throw an error? clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72293> Ditto as above. clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72294> "If no offsets are specified", how does this mean as map values? Or should we say sth like "If the specified offset is negative.." clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72295> Ditto above. clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72296> Not sure if we have discussed details about how to implement async commit in the new consumer? clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72297> "starting fetch offsets", "fetch position": inconsistency terms. clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72298> "...that the consumer currently consumes from" => from the API it seems we can get any partitions' offsets? clients/src/main/java/kafka/clients/consumer/Consumer.java <https://reviews.apache.org/r/19731/#comment72299> We need to be a bit clearer about the timestamp: it is the timestamp of the message when it reaches the broker, not the consumer. clients/src/main/java/kafka/clients/consumer/ConsumerConfig.java <https://reviews.apache.org/r/19731/#comment72300> ENABLE_AUTO_COMMIT_CONFIG to be consistent with ENABLE_JMX_CONFIG clients/src/main/java/kafka/clients/consumer/ConsumerConfig.java <https://reviews.apache.org/r/19731/#comment72301> fetch.buffer.bytes clients/src/main/java/kafka/clients/consumer/ConsumerConfig.java <https://reviews.apache.org/r/19731/#comment72302> AUTO_OFFSET_RESET_CONFIG is missing. clients/src/main/java/kafka/clients/consumer/ConsumerRebalanceCallback.java <https://reviews.apache.org/r/19731/#comment72304> Have we considered the following case: in a poll() function the consumer realized a rebalance is triggered, and hence call onPartitionRevoked and onPartitionAssigned, and then poll() times out, the user then call commit(partitions) on the old partitions. clients/src/main/java/kafka/clients/consumer/ConsumerRecord.java <https://reviews.apache.org/r/19731/#comment72306> Suggest add the following functions: compressionType() clients/src/main/java/kafka/clients/consumer/KafkaConsumer.java <https://reviews.apache.org/r/19731/#comment72309> Two constructors KafkaConsumer(ConsumerConfig config) and KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback) missing. clients/src/main/java/kafka/common/TopicPartitionOffset.java <https://reviews.apache.org/r/19731/#comment72311> This class is currently only used in user customized callback function. So do we really need to provide this class? - Guozhang Wang 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 > >