Re: New Producer Questions/Feedback

2015-03-24 Thread Grant Henke
Here is the jira: https://issues.apache.org/jira/browse/KAFKA-2043 Thanks, Grant On Mon, Mar 23, 2015 at 11:53 PM, Jun Rao j...@confluent.io wrote: RecordAccumulator is actually not part of the public api since it's internal. The public apis are only those in

Re: New Producer Questions/Feedback

2015-03-23 Thread Jun Rao
RecordAccumulator is actually not part of the public api since it's internal. The public apis are only those in http://kafka.apache.org/082/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html Thanks, Jun On Mon, Mar 23, 2015 at 9:23 PM, Grant Henke ghe...@cloudera.com wrote:

Re: New Producer Questions/Feedback

2015-03-23 Thread Jun Rao
Hi, Grant, The append api seems indeed a bit weird. The compression type is a producer level config. Instead of passing it in for each append, we probably should just pass it in once during the creation RecordAccumulator. Could you file a jira to track this? Thanks, Jun On Mon, Mar 23, 2015 at

New Producer Questions/Feedback

2015-03-23 Thread Grant Henke
I am reading over the new producer code in an effort to understand the implementation more thoroughly and had some questions/feedback. Currently org.apache.kafka.clients.producer.internals.RecordAccumulator append method accepts the compressionType on a per record basis. It looks like the code

Re: New Producer Questions/Feedback

2015-03-23 Thread Grant Henke
Thanks for validating that. I was thinking of solving it in the same fashion. Though I was unsure if there was/would be a use case to have multiple CompressionTypes in the same RecordAccumulator since the API was originally created this way. I would be happy to file a jira and can take on making