Re: Metrics in new producer

2014-02-11 Thread Sriram Subramanian
I think answering the questions below would help to make a better decision. I am all for writing better code and having superior functionalities but it is worth thinking about stuff outside just code in this case - 1. Does metric form a core piece of kafka? Does it help kafka greatly in providing

[jira] [Commented] (KAFKA-1258) Delete temporary data directory after unit test finishes

2014-02-11 Thread Sriram Subramanian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1258?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898861#comment-13898861 ] Sriram Subramanian commented on KAFKA-1258: --- Look into http://junit.org/javadoc/

Re: Config for new clients (and server)

2014-02-11 Thread Steve Morin
I would agree Jun > On Feb 11, 2014, at 21:30, Jun Rao wrote: > > I actually think this is useful for non-LinkedIn users as well. The > following is the tradeoff that I see. > > Most users probably won't care about seeing an extra 10-20 lines of INFO > level logging when starting up a client. H

Re: Config for new clients (and server)

2014-02-11 Thread Jun Rao
I actually think this is useful for non-LinkedIn users as well. The following is the tradeoff that I see. Most users probably won't care about seeing an extra 10-20 lines of INFO level logging when starting up a client. However, it's very easy for users to (1) mis-spell a config name (there was an

[jira] [Updated] (KAFKA-847) kafka appender layout does not work for kafka 0.7.1

2014-02-11 Thread Peter Pham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Pham updated KAFKA-847: - Attachment: KAFKA-847-v1.patch Set requiresLayout to true so that the log4j Kafka appender accepts layout

[jira] [Updated] (KAFKA-847) kafka appender layout does not work for kafka 0.7.1

2014-02-11 Thread Peter Pham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Pham updated KAFKA-847: - Labels: easyfix newbie (was: ) Affects Version/s: 0.8.0 Status: Patch Availa

[jira] [Issue Comment Deleted] (KAFKA-847) kafka appender layout does not work for kafka 0.7.1

2014-02-11 Thread Peter Pham (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Pham updated KAFKA-847: - Comment: was deleted (was: Set requiresLayout to true so that the log4j Kafka appender accepts layout par

[jira] [Commented] (KAFKA-1255) Offset in RecordMetadata is Incorrect with New Producer Ack = -1

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898775#comment-13898775 ] Jay Kreps commented on KAFKA-1255: -- I looked into this and the offset is indeed being set

[jira] [Resolved] (KAFKA-1259) Producer close function no longer block until all messages be sent

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps resolved KAFKA-1259. -- Resolution: Fixed The logic we had was to block until all requests were sent but not until all respons

Re: Review Request 17987: Followup patch incorporting Jay's suggestions

2014-02-11 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17987/#review34260 --- clients/src/test/java/org/apache/kafka/clients/producer/Partitioner

Re: Review Request 17880: Patch for KAFKA-1237

2014-02-11 Thread Neha Narkhede
> On Feb. 10, 2014, 11:46 p.m., Guozhang Wang wrote: > > build.gradle, line 139 > > > > > > If clients is included in compile do we still need to specify it in > > testCompile? I don't think so but probably Jun can c

[jira] [Updated] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1233: - Attachment: KAFKA-1233.patch > Integration test for the new producer > --

[jira] [Commented] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898728#comment-13898728 ] Guozhang Wang commented on KAFKA-1233: -- Created reviewboard https://reviews.apache.or

Review Request 17987: Followup patch incorporting Jay's suggestions

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17987/ --- Review request for kafka. Bugs: KAFKA-1233 https://issues.apache.org/jira/b

Re: Metrics in new producer

2014-02-11 Thread Jun Rao
Thanks for the detailed write-up. It's well thought through. A few comments: 1. I have a couple of concerns on the percentiles. The first issue is that It requires the user to know the value range. Since the range for things like message size (in millions) is quite different from those like reques

Re: Review Request 17973: Producer should block until all requests are fully acknowleged

2014-02-11 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17973/#review34253 --- Ship it! Ship It! - Neha Narkhede On Feb. 12, 2014, 12:38 a.m.,

Re: Review Request 17977: Patch for KAFKA-1257

2014-02-11 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17977/#review34252 --- Ship it! Ship It! - Neha Narkhede On Feb. 12, 2014, 1:14 a.m., J

[jira] [Commented] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898667#comment-13898667 ] Neha Narkhede commented on KAFKA-1233: -- Committed the existing file to trunk to not l

Re: Review Request 16360: Patch for KAFKA-1188

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16360/ --- (Updated Feb. 12, 2014, 1:58 a.m.) Review request for kafka. Bugs: KAFKA-1188

Re: Review Request 16360: Patch for KAFKA-1188

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16360/ --- (Updated Feb. 12, 2014, 1:58 a.m.) Review request for kafka. Bugs: KAFKA-1188

[jira] [Reopened] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede reopened KAFKA-1233: -- Ah, my bad. Will include it in and we leave this JIRA open for more improvements. > Integration t

[jira] [Commented] (KAFKA-1188) Stale LeaderAndIsr request could be handled by the broker on Controller failover

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1188?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898648#comment-13898648 ] Guozhang Wang commented on KAFKA-1188: -- Updated reviewboard against branch origin/tr

[jira] [Updated] (KAFKA-1188) Stale LeaderAndIsr request could be handled by the broker on Controller failover

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1188?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1188: - Attachment: KAFKA-1188_2014-02-11_17:50:06.patch > Stale LeaderAndIsr request could be handled by

[jira] [Updated] (KAFKA-1188) Stale LeaderAndIsr request could be handled by the broker on Controller failover

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1188?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1188: - Attachment: KAFKA-1188_2014-02-11_17:48:06.patch > Stale LeaderAndIsr request could be handled by

[jira] [Commented] (KAFKA-1188) Stale LeaderAndIsr request could be handled by the broker on Controller failover

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1188?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898646#comment-13898646 ] Guozhang Wang commented on KAFKA-1188: -- Updated reviewboard against branch origin/tr

Re: Review Request 16360: Patch for KAFKA-1188

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16360/ --- (Updated Feb. 12, 2014, 1:49 a.m.) Review request for kafka. Bugs: KAFKA-1188

Re: Review Request 16360: Patch for KAFKA-1188

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/16360/ --- (Updated Feb. 12, 2014, 1:48 a.m.) Review request for kafka. Bugs: KAFKA-1188

Re: Review Request 17977: Patch for KAFKA-1257

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17977/#review34248 --- Ship it! Ship It! - Guozhang Wang On Feb. 12, 2014, 1:14 a.m., J

Re: Review Request 17973: Producer should block until all requests are fully acknowleged

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17973/#review34247 --- Ship it! Ship It! - Guozhang Wang On Feb. 12, 2014, 12:38 a.m.,

[jira] [Assigned] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps reassigned KAFKA-1257: Assignee: Jay Kreps > New producer runs into out of order producer responses >

[jira] [Commented] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898611#comment-13898611 ] Jay Kreps commented on KAFKA-1257: -- Hey Neha, I'm not 100% sure the cause of this issue.

[jira] [Commented] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898604#comment-13898604 ] Jay Kreps commented on KAFKA-1257: -- Updated reviewboard https://reviews.apache.org/r/1797

Re: Review Request 17977: Patch for KAFKA-1257

2014-02-11 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17977/ --- (Updated Feb. 12, 2014, 1:14 a.m.) Review request for kafka. Bugs: KAFKA-1257

[jira] [Updated] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1257: - Attachment: KAFKA-1257_2014-02-11_17:14:48.patch > New producer runs into out of order producer responses

[jira] [Updated] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1257: - Attachment: KAFKA-1257_2014-02-11_17:11:54.patch > New producer runs into out of order producer responses

[jira] [Commented] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898597#comment-13898597 ] Jay Kreps commented on KAFKA-1257: -- Updated reviewboard https://reviews.apache.org/r/1797

Re: Review Request 17977: Patch for KAFKA-1257

2014-02-11 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17977/ --- (Updated Feb. 12, 2014, 1:12 a.m.) Review request for kafka. Summary (updated

[jira] [Updated] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1257: - Attachment: KAFKA-1257.patch > New producer runs into out of order producer responses > -

[jira] [Commented] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898589#comment-13898589 ] Jay Kreps commented on KAFKA-1257: -- Created reviewboard https://reviews.apache.org/r/1797

Review Request 17977: Metadata request and producer request can be sent at the same time.

2014-02-11 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17977/ --- Review request for kafka. Bugs: KAFKA-1257 https://issues.apache.org/jira/b

Re: Review Request 17879: Patch for KAFKA-1245

2014-02-11 Thread Joe Stein
> On Feb. 10, 2014, 10:19 p.m., Jun Rao wrote: > > gradle.properties, lines 21-24 > > > > > > Do you really intend to remove those lines? > > Joe Stein wrote: > Yes, that was intentional. > > Those value

[jira] [Commented] (KAFKA-1259) Producer close function no longer block until all messages be sent

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898562#comment-13898562 ] Jay Kreps commented on KAFKA-1259: -- Created reviewboard https://reviews.apache.org/r/1797

[jira] [Updated] (KAFKA-1259) Producer close function no longer block until all messages be sent

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1259: - Attachment: KAFKA-1259.patch > Producer close function no longer block until all messages be sent > -

Review Request 17973: Producer should block until all requests are fully acknowleged

2014-02-11 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17973/ --- Review request for kafka. Bugs: KAFKA-1259 https://issues.apache.org/jira/b

[jira] [Created] (KAFKA-1260) Integration Test for New Producer Part II: Broker Failure Handling

2014-02-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-1260: Summary: Integration Test for New Producer Part II: Broker Failure Handling Key: KAFKA-1260 URL: https://issues.apache.org/jira/browse/KAFKA-1260 Project: Kafka

[jira] [Created] (KAFKA-1259) Producer close function no longer block until all messages be sent

2014-02-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-1259: Summary: Producer close function no longer block until all messages be sent Key: KAFKA-1259 URL: https://issues.apache.org/jira/browse/KAFKA-1259 Project: Kafka

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Guozhang Wang
> On Feb. 11, 2014, 11:27 p.m., Jay Kreps wrote: > > core/src/test/scala/integration/kafka/api/ProducerSendTest.scala, line 283 > > > > > > It would be good to add a test where we send a message, stop one of the > > s

Re: New Consumer API discussion

2014-02-11 Thread Guozhang Wang
Hi Pradeep: 1. I think TopicPartition is designed as an internal class and the plan was not to expose it to users just for simplicity. We probably will change the commit APIs not exposing them. 2. We have thought about that before, and finally decide to make it as subscribe(topic, partition) pos

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17918/#review34241 --- core/src/test/scala/integration/kafka/api/ProducerSendTest.scala

[jira] [Commented] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898485#comment-13898485 ] Jay Kreps commented on KAFKA-1233: -- Also, that checkin didn't include the test. > Integr

[jira] [Commented] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898484#comment-13898484 ] Jay Kreps commented on KAFKA-1233: -- Can we leave this ticket open I have a few improvemen

[jira] [Resolved] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede resolved KAFKA-1233. -- Resolution: Fixed Thanks for the patch, committed to trunk > Integration test for the new prod

Re: Review Request 17923: Patch for KAFKA-1243

2014-02-11 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17923/#review34234 --- Ship it! Ship It! - Neha Narkhede On Feb. 11, 2014, 9:37 p.m., J

Re: New Consumer API discussion

2014-02-11 Thread Imran Rashid
Hi, thanks for sharing this and getting feedback. Sorry I am probably missing something basic, but I'm not sure how a multi-threaded consumer would work. I can imagine its either: a) I just have one thread poll kafka. If I want to process msgs in multiple threads, than I deal w/ that after pol

[jira] [Commented] (KAFKA-1243) Gradle issues for release

2014-02-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898364#comment-13898364 ] Jun Rao commented on KAFKA-1243: KAFKA-1246: It seems that the sbt way is correct. The rea

[jira] [Updated] (KAFKA-1243) Gradle issues for release

2014-02-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1243?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1243: --- Attachment: KAFKA-1243_2014-02-11_13:37:25.patch > Gradle issues for release > - > >

[jira] [Commented] (KAFKA-1243) Gradle issues for release

2014-02-11 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898348#comment-13898348 ] Jun Rao commented on KAFKA-1243: Updated reviewboard https://reviews.apache.org/r/17923/

Re: Review Request 17923: Patch for KAFKA-1243

2014-02-11 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17923/ --- (Updated Feb. 11, 2014, 9:37 p.m.) Review request for kafka. Summary (updated

[jira] [Commented] (KAFKA-1258) Delete temporary data directory after unit test finishes

2014-02-11 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1258?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898307#comment-13898307 ] Jay Kreps commented on KAFKA-1258: -- Yeah this annoys me too. The core problem is that jav

Re: New Consumer API discussion

2014-02-11 Thread Pradeep Gollakota
Updated thoughts. 1. subscribe(String topic, int... paritions) and unsubscribe(String topic, int... partitions) should be subscribe(TopicPartition... topicPartitions)and unsubscribe(TopicPartition... topicPartitons) 2. Does it make sense to provide a convenience method to subs

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17918/#review34230 --- Ship it! Ship It! - Neha Narkhede On Feb. 11, 2014, 8:28 p.m., G

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Neha Narkhede
> On Feb. 11, 2014, 7:18 p.m., Neha Narkhede wrote: > > core/src/test/scala/integration/kafka/api/ProducerSendTest.scala, line 109 > > > > > > These 3 statements are repeated in many tests. Can we create a helper > >

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17918/ --- (Updated Feb. 11, 2014, 8:28 p.m.) Review request for kafka. Bugs: KAFKA-1233

[jira] [Updated] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1233: - Attachment: KAFKA-1233_2014-02-11_12:27:15.patch > Integration test for the new producer > --

[jira] [Commented] (KAFKA-1233) Integration test for the new producer

2014-02-11 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13898267#comment-13898267 ] Guozhang Wang commented on KAFKA-1233: -- Updated reviewboard https://reviews.apache.or

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17918/ --- (Updated Feb. 11, 2014, 8:27 p.m.) Review request for kafka. Bugs: KAFKA-1233

[jira] [Created] (KAFKA-1258) Delete temporary data directory after unit test finishes

2014-02-11 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-1258: Summary: Delete temporary data directory after unit test finishes Key: KAFKA-1258 URL: https://issues.apache.org/jira/browse/KAFKA-1258 Project: Kafka Issue

Re: New Consumer API discussion

2014-02-11 Thread Pradeep Gollakota
Hi Jay, I apologize for derailing the conversation about the consumer API. We should start a new discussion about hierarchical topics, if we want to keep talking about it. My final thought on the matter is that, hierarchical topics is still an important feature to have in Kafka, because it gives u

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Guozhang Wang
> On Feb. 11, 2014, 7:18 p.m., Neha Narkhede wrote: > > core/src/test/resources/log4j.properties, line 15 > > > > > > this is probably included in the patch by accident right? Yeah, will fix. > On Feb. 11, 2014, 7:18

Re: Review Request 17918: Patch for KAFKA-1233

2014-02-11 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17918/ --- (Updated Feb. 11, 2014, 7:24 p.m.) Review request for kafka. Summary (updated

Re: Review Request 17918: Patch for KAFKA-1232

2014-02-11 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/17918/#review34158 --- core/src/test/resources/log4j.properties

[jira] [Created] (KAFKA-1257) New producer runs into out of order producer responses

2014-02-11 Thread Neha Narkhede (JIRA)
Neha Narkhede created KAFKA-1257: Summary: New producer runs into out of order producer responses Key: KAFKA-1257 URL: https://issues.apache.org/jira/browse/KAFKA-1257 Project: Kafka Issue Ty

[jira] [Created] (KAFKA-1256) Improve error reporting in the Sender for request types

2014-02-11 Thread Neha Narkhede (JIRA)
Neha Narkhede created KAFKA-1256: Summary: Improve error reporting in the Sender for request types Key: KAFKA-1256 URL: https://issues.apache.org/jira/browse/KAFKA-1256 Project: Kafka Issue T

Re: New Consumer API discussion

2014-02-11 Thread Jay Kreps
Hey Pradeep, That wiki is fairly old and it predated more flexible subscription mechanisms. In the high-level consumer you currently have wildcard subscription and in the new proposed interface you can actually subscribe based on any logic you want to create a "union" of streams. Personally I thin