Re: [DISCUSS] Using GitHub Pull Requests for contributions and code review

2015-05-05 Thread Ismael Juma
Hi Jun, On Sat, May 2, 2015 at 2:50 PM, Jun Rao j...@confluent.io wrote: We will also need to figure out if we need CONTRIBUTING.md like the following to take care of the Apache licensing stuff. https://github.com/apache/spark/blob/master/CONTRIBUTING.md Yes indeed. That is in step 3 of

Re: Kafka KIP hangout May 5

2015-05-05 Thread Tom Graves
Is there information on how to get into the hangout?  Or is it by invitation?   I'm very interested in the authorization changes. Thanks,Tom On Monday, May 4, 2015 9:20 AM, Jun Rao j...@confluent.io wrote: Hi, Everyone, We will have a KIP hangout at 11 PST on May 5. The following is

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-05 Thread Sriharsha Chintalapani
Thanks for the comments everyone. Hi Jay,      I do have a question regarding configurable interface on how to pass a MapString, ? properties. I couldn’t find any other classes using it. JMX reporter overrides it but doesn’t implement it.  So with configurable partitioner how can a user pass in

Re: [DISCUSS] Using GitHub Pull Requests for contributions and code review

2015-05-05 Thread Ismael Juma
Hi Ewen, Comments inline. On Fri, May 1, 2015 at 8:38 AM, Ewen Cheslack-Postava e...@confluent.io wrote: Also +1. There are some drawbacks to using Github for reviews, e.g. lots of emails for each review because they don't let you publish your entire review in one go like RB does, but it

Re: [DISCUSS] KIP-19 Add a request timeout to NetworkClient

2015-05-05 Thread Mayuresh Gharat
Just a quick question, can we handle REQUEST TIMEOUT as disconnections and do a fresh MetaDataRequest and retry instead of failing the request? Thanks, Mayuresh On Mon, May 4, 2015 at 10:32 AM, Jiangjie Qin j...@linkedin.com.invalid wrote: I incorporated Ewen and Guozhang’s comments in the

Re: [DISCUSSION] Reuse o.a.k.clients.NetworkClient in controller.

2015-05-05 Thread Jun Rao
Hi, Jiangjie, Thanks for taking on this. I was thinking that one way to decouple the dependency on Metadata in NetworkClient is the following. 1. Make Metadata an interface. 2. Rename current Metadata class to sth like KafkaMetadata that implements the Metadata interface. 3. Have a new

Re: Review Request 33049: Patch for KAFKA-2084

2015-05-05 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated May 6, 2015, 12:52 a.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Commented] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-05-05 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529672#comment-14529672 ] Aditya A Auradkar commented on KAFKA-2084: -- Updated reviewboard

[jira] [Updated] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-05-05 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2084: - Attachment: KAFKA-2084_2015-05-05_17:52:02.patch byte rate metrics per client ID

[jira] [Commented] (KAFKA-1646) Improve consumer read performance for Windows

2015-05-05 Thread Honghai Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529694#comment-14529694 ] Honghai Chen commented on KAFKA-1646: - When trying add test case for Log, got test

[jira] [Comment Edited] (KAFKA-1646) Improve consumer read performance for Windows

2015-05-05 Thread Honghai Chen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529694#comment-14529694 ] Honghai Chen edited comment on KAFKA-1646 at 5/6/15 1:13 AM: -

Re: Kafka KIP hangout May 5

2015-05-05 Thread Ashish Singh
Below are my notes from today’s KIP hangout. Please feel free to add/ correct the content. Kafka KIP Discussion (May 5) KIP-4 (admin commands): any remaining issues - Only issue left is how to handle multiple instructions for one topic. Suggestion here was to silently ignore duplicate

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-05 Thread Gwen Shapira
Sorry I missed the call today :) I think an additional requirement would be: Make sure that traditional deployment tools (Puppet, Chef, etc) are still capable of managing Kafka configuration. For this reason, I'd like the configuration refresh to be pretty close to what most Linux services are

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-05-05 Thread Andrii Biletskyi
Guys, I've updated the wiki to reflect all previously discussed items (regarding the schema only - this is included to phase 1). https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations I think we can have the final discussion today (for

[jira] [Commented] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-05-05 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529422#comment-14529422 ] Aditya A Auradkar commented on KAFKA-2084: -- Updated reviewboard

Re: Review Request 33049: Patch for KAFKA-2084

2015-05-05 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated May 5, 2015, 10:27 p.m.) Review request for kafka, Joel Koshy and Jun

Re: Review Request 33049: Patch for KAFKA-2084

2015-05-05 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33049/ --- (Updated May 5, 2015, 10:29 p.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Updated] (KAFKA-2084) byte rate metrics per client ID (producer and consumer)

2015-05-05 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2084: - Attachment: KAFKA-2084_2015-05-05_15:27:35.patch byte rate metrics per client ID

[jira] [Created] (KAFKA-2172) Round-robin partition assignment strategy too restrictive

2015-05-05 Thread Jason Rosenberg (JIRA)
Jason Rosenberg created KAFKA-2172: -- Summary: Round-robin partition assignment strategy too restrictive Key: KAFKA-2172 URL: https://issues.apache.org/jira/browse/KAFKA-2172 Project: Kafka

[jira] [Commented] (KAFKA-2172) Round-robin partition assignment strategy too restrictive

2015-05-05 Thread Jason Rosenberg (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2172?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529436#comment-14529436 ] Jason Rosenberg commented on KAFKA-2172: as a side note, the documentation here:

[VOTE] KIP-4 Admin Commands / Phase-1

2015-05-05 Thread Andrii Biletskyi
Hi all, This is a voting thread for KIP-4 Phase-1. It will include Wire protocol changes and server side handling code. https://cwiki.apache.org/confluence/display/KAFKA/KIP-4+-+Command+line+and+centralized+administrative+operations Thanks, Andrii Biletskyi

Re: Review Request 33088: add heartbeat to coordinator

2015-05-05 Thread Onur Karaman
On April 28, 2015, 12:13 a.m., Guozhang Wang wrote: core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala, lines 281-355 https://reviews.apache.org/r/33088/diff/9/?file=941787#file941787line281 If we add the CoordinatorMetadata class, these functions can all be moved

[jira] [Updated] (KAFKA-2160) DelayedOperationPurgatory should remove the pair in watchersForKey with empty watchers list

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2160: --- Reviewer: Jun Rao DelayedOperationPurgatory should remove the pair in watchersForKey with empty watchers

[jira] [Updated] (KAFKA-1387) Kafka getting stuck creating ephemeral node it has already created when two zookeeper sessions are established in a very short period of time

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1387?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1387: --- Reviewer: Jun Rao Kafka getting stuck creating ephemeral node it has already created when two zookeeper

[jira] [Updated] (KAFKA-2165) ReplicaFetcherThread: data loss on unknown exception

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2165?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2165: --- Reviewer: Jun Rao ReplicaFetcherThread: data loss on unknown exception

[jira] [Updated] (KAFKA-1690) new java producer needs ssl support as a client

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1690: --- Reviewer: Jun Rao new java producer needs ssl support as a client

[jira] [Updated] (KAFKA-1928) Move kafka.network over to using the network classes in org.apache.kafka.common.network

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-1928: --- Reviewer: Jay Kreps Move kafka.network over to using the network classes in

[jira] [Updated] (KAFKA-2161) Fix a few copyrights

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2161: --- Reviewer: Joe Stein Fix a few copyrights Key: KAFKA-2161

Re: Review Request 33760: Patch for KAFKA-2121

2015-05-05 Thread Jun Rao
On May 4, 2015, 5:31 p.m., Neha Narkhede wrote: clients/src/main/java/org/apache/kafka/common/serialization/Serializer.java, line 25 https://reviews.apache.org/r/33760/diff/2/?file=947511#file947511line25 This and also the Deserializer should extend Configurable too right? We

[jira] [Updated] (KAFKA-2129) Consumer could make multiple concurrent metadata requests

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2129?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2129: --- Reviewer: Guozhang Wang Consumer could make multiple concurrent metadata requests

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-05 Thread Joel Koshy
Good discussion. Since we will be talking about this at 11am, I wanted to organize these comments into requirements to see if we are all on the same page. REQUIREMENT 1: Needs to accept dynamic config changes. This needs to be general enough to work for all configs that we envision may need to

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2123: --- Reviewer: Jay Kreps Make new consumer offset commit API use callback + future

[jira] [Updated] (KAFKA-2121) prevent potential resource leak in KafkaProducer and KafkaConsumer

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2121?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2121: --- Reviewer: Guozhang Wang prevent potential resource leak in KafkaProducer and KafkaConsumer

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Ashish Singh
Adrian, Trying to follow up the discussion here. Is my understanding correct that if we have topic hierarchies then we can do without namespaces. To me namespace is an abstraction, it can be implemented with topic hierarchies as well, would you agree? If so I guess topic hierarchies is the way to

Re: Review Request 33088: add heartbeat to coordinator

2015-05-05 Thread Onur Karaman
On April 28, 2015, 12:13 a.m., Guozhang Wang wrote: clients/src/main/java/org/apache/kafka/common/protocol/Errors.java, lines 73-76 https://reviews.apache.org/r/33088/diff/9/?file=941786#file941786line73 Maybe we can merge them into one error code,

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Ashish Singh
Jay, I agree with you. This will kind of depend on how KIP-11 and KIP-12 shapes up, but we can definitely start putting together the ideas. Would you suggest starting a kip or just a generic shared document that can be later translated into a KIP? On Tuesday, May 5, 2015, Jay Kreps

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Gari Singh
I agree. If we can really do pluggable authorization (and even pluggable authentication), it would not actually be hard to effectively implement a multi-tenant solution. I am hoping to attempt to implement something like this once there is code / patches for KIP-11 and KIP-12. So I wonder if we

RE: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-05-05 Thread Honghai Chen
Use config.segmentSize should be ok. Previously add that one for make sure the file not exceed config.segmentSize, actually the function maybeRoll already make sure that. When try add test case for recover, blocked by the rename related issue, just open one jira at

[jira] [Created] (KAFKA-2170) 10 LogTest cases failed for file.renameTo failed under windows

2015-05-05 Thread Honghai Chen (JIRA)
Honghai Chen created KAFKA-2170: --- Summary: 10 LogTest cases failed for file.renameTo failed under windows Key: KAFKA-2170 URL: https://issues.apache.org/jira/browse/KAFKA-2170 Project: Kafka

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Adrian Preston
I'm don't agree. At least not completely ;) Pluggable authorization (or hierarchies with the ability to define ACLs at each level) would allow you to isolate different groups of users. However each user will still be aware of one another's presence in the topic namespace - and could also be

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Adrian Preston
I'm don't agree. At least not completely ;) Pluggable authorization (or hierarchies with the ability to define ACLs at each level) would allow you to isolate different groups of users. However each user will still be aware of one another's presence in the topic namespace - and could also be

[DISCUSSION] Reuse o.a.k.clients.NetworkClient in controller.

2015-05-05 Thread Jiangjie Qin
I am trying to see if we can reuse the NetworkClient class to be used in controller to broker communication. (Also, we can probably use KafkaConsumer which is already using NetworkClient in replica fetchers). Currently NetworkClient does the following things in addition to sending requests.

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-05 Thread Neha Narkhede
Joel, thanks for summarizing the requirements. It makes sense for the KIP to focus on Req #1, unless any future configs as dynamic ones warrants a completely different design. My main concern is going with a design by keeping only quotas in mind and then continue shoehorning other dynamic configs

Re: Adding multi-tenancy capabilities to Kafka

2015-05-05 Thread Jay Kreps
Hey guys, It would be nice to get a design around this. Though there are currently so many big things in flight I do wonder if we should start another parallel thing...? But working out a design can't hurt. Personally I think since one of the goals of Kafka is data integration we really want to

[jira] [Created] (KAFKA-2171) System Test for Quotas

2015-05-05 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-2171: --- Summary: System Test for Quotas Key: KAFKA-2171 URL: https://issues.apache.org/jira/browse/KAFKA-2171 Project: Kafka Issue Type: Sub-task Reporter: Dong

Re: Review Request 33088: add heartbeat to coordinator

2015-05-05 Thread Onur Karaman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33088/ --- (Updated May 5, 2015, 5:50 p.m.) Review request for kafka. Bugs: KAFKA-1334

[jira] [Commented] (KAFKA-1334) Add failure detection capability to the coordinator / consumer

2015-05-05 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14528896#comment-14528896 ] Onur Karaman commented on KAFKA-1334: - Updated reviewboard

[jira] [Commented] (KAFKA-2103) kafka.producer.AsyncProducerTest failure.

2015-05-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529033#comment-14529033 ] Ewen Cheslack-Postava commented on KAFKA-2103: -- Agreed with [~becket_qin]

Review Request 33857: Patch for KAFKA-2103

2015-05-05 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33857/ --- Review request for kafka. Bugs: KAFKA-2103

[jira] [Updated] (KAFKA-2103) kafka.producer.AsyncProducerTest failure.

2015-05-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2103: - Attachment: KAFKA-2103.patch kafka.producer.AsyncProducerTest failure.

[jira] [Updated] (KAFKA-2103) kafka.producer.AsyncProducerTest failure.

2015-05-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2103?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2103: - Assignee: Ewen Cheslack-Postava (was: Dong Lin) Status: Patch Available

[jira] [Commented] (KAFKA-2103) kafka.producer.AsyncProducerTest failure.

2015-05-05 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2103?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529032#comment-14529032 ] Ewen Cheslack-Postava commented on KAFKA-2103: -- Created reviewboard

[jira] [Commented] (KAFKA-1690) new java producer needs ssl support as a client

2015-05-05 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529043#comment-14529043 ] Sriharsha Chintalapani commented on KAFKA-1690: --- [~junrao] [~jjkoshy] I got

[jira] [Commented] (KAFKA-2156) Possibility to plug in custom MetricRegistry

2015-05-05 Thread Andras Sereny (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14528203#comment-14528203 ] Andras Sereny commented on KAFKA-2156: -- Thanks Neha. This might already work with

[jira] [Commented] (KAFKA-2170) 10 LogTest cases failed for file.renameTo failed under windows

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2170?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14529337#comment-14529337 ] Jun Rao commented on KAFKA-2170: We do have the logic in OffsetIndex.resize() to unmap

Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-05-05 Thread Jun Rao
Thanks. Could you updated the wiki? Also, commented on the jira. Jun On Tue, May 5, 2015 at 12:48 AM, Honghai Chen honghai.c...@microsoft.com wrote: Use config.segmentSize should be ok. Previously add that one for make sure the file not exceed config.segmentSize, actually the function

[jira] [Comment Edited] (KAFKA-2156) Possibility to plug in custom MetricRegistry

2015-05-05 Thread Andras Sereny (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14528203#comment-14528203 ] Andras Sereny edited comment on KAFKA-2156 at 5/5/15 10:58 AM: