> On May 19, 2014, 5:30 p.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java, > > lines 70-71 > > <https://reviews.apache.org/r/19731/diff/11/?file=583622#file583622line70> > > > > Instead of introducing ConsumerRecords, I'd prefer just returning > > Map<String, List<ConsumerRecord>>. > > Guozhang Wang wrote: > The question I think we are trying to answer is how to expose the > per-partition error code back to the user. So far it seems we do not have an > ideal solution yet.
Agree with Guozhang. For that, it's worth keeping it the way it is. > On May 19, 2014, 5:30 p.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java, > > lines 99-119 > > <https://reviews.apache.org/r/19731/diff/11/?file=583622#file583622line99> > > > > Since these apis have Map in the return value, they are really intended > > as a batch api. So, would it better to have the input as a set of > > TopicPartitions? This will also make sure that the passed in partitions are > > unique. We discussed this offline, changing it to Collection. > On May 19, 2014, 5:30 p.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/internals/FutureOffsetMetadata.java, > > lines 1-36 > > <https://reviews.apache.org/r/19731/diff/11/?file=583630#file583630line1> > > > > Do we still need this class? Removed. > On May 19, 2014, 5:30 p.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java, > > lines 36-47 > > <https://reviews.apache.org/r/19731/diff/11/?file=583624#file583624line36> > > > > Would it be better to use Set[TopicPartition] instead of ellipsis? This > > will make it clear that they are unique. Changing to Collection. > On May 19, 2014, 5:30 p.m., Jun Rao wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java, > > lines 84-87 > > <https://reviews.apache.org/r/19731/diff/11/?file=583622#file583622line84> > > > > It's going to be a bit confusing to the caller to expect both an error > > code in the return value and an exception. It seems that we can just > > translate exceptions into error codes. In async mode, the return value will > > be null. So it's impossible for the caller to get the error code. However, > > by choosing the async mode, the caller doesn't assume the commit to succeed > > immediately. It's probably ok just to try to commit again when it's called. Makes sense. Changed the javadoc to reflect that it does not throw any exception - Neha ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/19731/#review43365 ----------------------------------------------------------- On May 16, 2014, 6:46 p.m., Neha Narkhede wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/19731/ > ----------------------------------------------------------- > > (Updated May 16, 2014, 6:46 p.m.) > > > Review request for kafka. > > > Bugs: KAFKA-1328 > https://issues.apache.org/jira/browse/KAFKA-1328 > > > Repository: kafka > > > Description > ------- > > 1. Improved documentation on the position() API 2. Changed signature of > commit API to remove Future and include a sync flag > > > Included Jun's review suggestions part 2, except change to the commit() API > since it needs more thought > > > Review comments from Jun and Guozhang > > > Checked in ConsumerRecordMetadata > > > Fixed the javadoc usage examples in KafkaConsumer to match the API changes > > > Changed the signature of poll to return Map<String,ConsumerRecordMetadata> to > organize the ConsumerRecords around topic and then optionally around > partition. This will serve the group management as well as custom partition > subscription use cases > > > 1. Changed the signature of poll() to return Map<String, > List<ConsumerRecord>> 2. Changed ConsumerRecord to throw an exception if an > error is detected for the partition. For example, if a single large message > is larger than the total memory just for that partition, we don't want poll() > to throw an exception since that will affect the processing of the remaining > partitions as well > > > Fixed MockConsumer to make subscribe(topics) and subscribe(partitions) > mutually exclusive > > > Changed the package to org.apache.kafka.clients.consumer from > kafka.clients.consumer > > > Changed the package to org.apache.kafka.clients.consumer from > kafka.clients.consumer > > > 1. Removed the commitAsync() APIs 2. Changed the commit() APIs to return a > Future > > > Fixed configs to match the producer side configs for metrics > > > Renamed AUTO_COMMIT_ENABLE_CONFIG to ENABLE_AUTO_COMMIT_CONFIG > > > Addressing review comments from Tim and Guozhang > > > Rebasing after producer side config cleanup > > > 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/org/apache/kafka/clients/consumer/Consumer.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java > PRE-CREATION > > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java > PRE-CREATION > > clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java > PRE-CREATION > > clients/src/main/java/org/apache/kafka/clients/consumer/internals/FutureOffsetMetadata.java > PRE-CREATION > clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java > 90cacbd8941b7c8f15d1417c821945c1ac1b4d00 > 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 > >