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

2015-04-30 Thread Jaikiran Pai
I think this will help a lot in contributions. Some of my local changes that I want to contribute back have been pending because I sometimes switch machines and I then have to go through setting up the Ruby/python and other stuff for the current review process. Using just github is going to hel

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

2015-04-30 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522757#comment-14522757 ] Dong Lin commented on KAFKA-2084: - Here is my 2 cents. I vote for your 2nd approach: Clien

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

2015-04-30 Thread Neha Narkhede
Thanks a bunch for taking this up, Ismael! +1, I think it will be much more convenient to sunset RB and move to github. Especially looking forward to the CIs on PRs and also the merge script. Alas, my wonderful patch-review script will be retired :-) On Thu, Apr 30, 2015 at 6:12 AM, Ismael Juma

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Don Bosco Durai
>>Gwen << regarding additional authorizers >I think having these in the system tests duals as both good confidence in >language independency of the changes. It also makes sure that when we >release that we don't go breaking Sentry or Ranger or anyone else that >wants to integate. As much I would

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Suresh Srinivas
Gwen, Thanks for the clarification. My objection is, we should not do it just because of the reason that databases have always done it this way. May be there is a history there that might have forced a choice like that. That has led to other DBs to comply with it. Kafka is a different system. Let

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Gwen Shapira
I think Kafka's behavior should be driven by what users want. My only indication to what they may want is what we were forced to fix in similar cases. This is why I am advocating this behavior. I agree that this is a minor point that should not be blocking the vote. I already gave my non-binding +

[jira] [Commented] (KAFKA-2145) An option to add topic owners.

2015-04-30 Thread Neelesh Srinivas Salian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522578#comment-14522578 ] Neelesh Srinivas Salian commented on KAFKA-2145: [~parth.brahmbhatt] I ha

[jira] [Commented] (KAFKA-2132) Move Log4J appender to clients module

2015-04-30 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522575#comment-14522575 ] Gwen Shapira commented on KAFKA-2132: - +1 (non-binding). Looks good to me. [~jkreps]

Re: Review Request 33614: Patch for KAFKA-2132

2015-04-30 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33614/#review82231 --- Ship it! Ship It! - Gwen Shapira On April 30, 2015, 10:53 p.m.,

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Don Bosco Durai
Joe, these are good use cases, however in the firt phase the granularity is at the Topic (your e.g. bucket) level and not what you are accessing within the Topic. So in your use case, if you don’t have access to “Bucket A”, then you won’t know who is in it, so you won’t know “Alice” or anyone who a

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

2015-04-30 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522567#comment-14522567 ] Gwen Shapira commented on KAFKA-1928: - Updated reviewboard https://reviews.apache.org/

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

2015-04-30 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1928: Attachment: KAFKA-1928_2015-04-30_17:48:33.patch > Move kafka.network over to using the network clas

Re: Review Request 33065: Patch for KAFKA-1928

2015-04-30 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/ --- (Updated May 1, 2015, 12:48 a.m.) Review request for kafka. Bugs: KAFKA-1928

[jira] [Created] (KAFKA-2162) Kafka Auditing functionality

2015-04-30 Thread Sriharsha Chintalapani (JIRA)
Sriharsha Chintalapani created KAFKA-2162: - Summary: Kafka Auditing functionality Key: KAFKA-2162 URL: https://issues.apache.org/jira/browse/KAFKA-2162 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-04-30 Thread Joel Koshy
Just went through this thread. I'm on-board with this as well. @Gwen - yes at LinkedIn we do need to support both authenticated/unauthenticated users on the same Kafka cluster because we cannot switch all clients simultaneously. I would be surprised if this is unique to LinkedIn. Also, I think th

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Gwen Shapira
On Thu, Apr 30, 2015 at 4:39 PM, Parth Brahmbhatt < pbrahmbh...@hortonworks.com> wrote: > Hi Joe, > > Let me clarify on authZException. The caller gets a 403 regardless of > existence of the topic, even if the topic does not exist you always get > 403. This will fall under the case wherewe do not

Re: [DISCUSS] KIP-21 Configuration Management

2015-04-30 Thread Joel Koshy
>1. I have deep concerns about managing configuration in ZooKeeper. >First, Producers and Consumers shouldn't depend on ZK at all, this seems >to add back a dependency we are trying to get away from. The KIP probably needs to be clarified here - I don't think Aditya was referring to cl

Re: Review Request 33548: KAFKA-2068 Step I: merge in KAFKA-1841

2015-04-30 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33548/#review82224 --- Ship it! Thanks for the patch. +1. Just a few minor comments below.

Re: Review Request 33731: KAFKA-2160

2015-04-30 Thread Onur Karaman
> On April 30, 2015, 10:45 p.m., Onur Karaman wrote: > > core/src/main/scala/kafka/server/DelayedOperation.scala, line 224 > > > > > > We can put the key inside Watchers and just use the watchersForKey > > that's alre

Re: [DISCUSSION] java.io.Closeable in KAFKA-2121

2015-04-30 Thread Jay Kreps
Hey Jun, I think the Closable interface is what we've used elsewhere and what the rest of the java world uses. I don't think it is too hard for us to add the override in our interface--implementors of the interface don't need to do it. -Jay On Thu, Apr 30, 2015 at 4:02 PM, Jun Rao wrote: > Tha

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Suresh Srinivas
Joe, thanks for the clarification. Regarding audits, sorry I might be misunderstanding your email. Currently, if Kafka does not support audits, I think audits should be considered as a separate effort. Here are the reasons: - Audit, whether authorization is available or not, should record operat

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Joe Stein
I kind of thought of the authorization module as something that happens in handle(request: RequestChannel.Reuqest) in the request.requestId match If the request doesn't do what it is allowed too it should stop right there. That "what it is allowed to-do" is a true/false callback to the class loade

Re: [DISCUSSION] java.io.Closeable in KAFKA-2121

2015-04-30 Thread Jun Rao
That makes sense. Then, would it be better to have a KafkaClosable interface that doesn't throw exception? This way, we don't need to override close in every implementing class. Thanks, Jun On Wed, Apr 29, 2015 at 10:36 AM, Steven Wu wrote: > Jun, > > we still get the benefit of extending Clos

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Gwen Shapira
Ah, I'm not talking about security by obscurity. At least in the database world, if you don't have SELECT on a table, you won't even see it when saying "show tables" because the very fact that the table exists is privileged. In that case, a denied SELECT attempt will return "table does not exist",

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Joe Stein
If you have bucket A and Bucket B and in Bucket A there are patients with "Disease X" and Bucket B patients without "Disease X". Now you try to access "Alice" from bucket A and you get a 403 and then from Bucket "B" you get a 404. What does that tell you now about Alice? Yup, she has "Disease X"

[jira] [Commented] (KAFKA-2132) Move Log4J appender to clients module

2015-04-30 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522427#comment-14522427 ] Ashish K Singh commented on KAFKA-2132: --- Updated reviewboard https://reviews.apache.

[jira] [Updated] (KAFKA-2132) Move Log4J appender to clients module

2015-04-30 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-2132: -- Attachment: KAFKA-2132_2015-04-30_15:53:17.patch > Move Log4J appender to clients module > -

Re: Review Request 33614: Patch for KAFKA-2132

2015-04-30 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33614/ --- (Updated April 30, 2015, 10:53 p.m.) Review request for kafka. Bugs: KAFKA-21

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Suresh Srinivas
Comment on AuthorizationException. I think the intent of exception should be to capture why a request is rejected. It is important from API perspective to be specific to aid debugging. Having a generic or obfuscated exception is not very useful. Does someone on getting an exception reach out to

Re: Review Request 33731: KAFKA-2160

2015-04-30 Thread Onur Karaman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33731/#review82217 --- core/src/main/scala/kafka/server/DelayedOperation.scala

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Joe Stein
Ok, I read through it all again a few times. I get the provider broker piece now. The configurations are still confusing if there are 2 or 3 and they should be called out more specifically than as a change to a class. Configs are a public interface we should be a bit more explicit. Was there any

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

2015-04-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522384#comment-14522384 ] Guozhang Wang commented on KAFKA-2160: -- Updated reviewboard https://reviews.apache.or

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

2015-04-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2160: - Attachment: KAFKA-2160_2015-04-30_15:20:14.patch > DelayedOperationPurgatory should remove the pai

Re: Review Request 33731: KAFKA-2160

2015-04-30 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33731/ --- (Updated April 30, 2015, 10:20 p.m.) Review request for kafka. Bugs: KAFKA-21

Re: Review Request 33731: KAFKA-2160

2015-04-30 Thread Onur Karaman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33731/#review82204 --- core/src/main/scala/kafka/server/DelayedOperation.scala

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

2015-04-30 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522285#comment-14522285 ] Joel Koshy commented on KAFKA-2161: --- We did this before and removed in KAFKA-1158. [~joe

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

2015-04-30 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522273#comment-14522273 ] Aditya Auradkar commented on KAFKA-2161: +1 on using Rat. It doesn't seem to have

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

2015-04-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2160: - Status: Patch Available (was: Open) > DelayedOperationPurgatory should remove the pair in watcher

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

2015-04-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2160?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2160: - Attachment: KAFKA-2160.patch > DelayedOperationPurgatory should remove the pair in watchersForKey

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

2015-04-30 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522266#comment-14522266 ] Guozhang Wang commented on KAFKA-2160: -- Created reviewboard https://reviews.apache.or

Review Request 33731: KAFKA-2160

2015-04-30 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33731/ --- Review request for kafka. Bugs: KAFKA-2160 https://issues.apache.org/jira/b

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Parth Brahmbhatt
During the discussion Jun pointed out that mirror maker, which right now does not copy any zookeeper config overrides, will now replicate topics but will not replicate any acls. Given the authorizer interface exposes the acl management apis, list/get/add/remove, weproposed that mirror maker can jus

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Parth Brahmbhatt
I am sorry but I am having a hard time to understand the core concern with json storage in zookeeper. * If you are concerned that in order to support only 150 users our of a million we will have to add a huge json with all 1 million users that is a misunderstanding. By default anyone who does no h

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Joe Stein
Parth, Can you explain how "Mirror maker will have to start using new acl management tool") and it not affect any other client. If you aren't changing the wire protocol then how do clients use it? ~ Joe stein On Thu, Apr 30, 2015 at 3:15 PM, Parth Brahmbhatt < pbrahmbh...@hortonworks.com> wrote

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

2015-04-30 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522112#comment-14522112 ] Gwen Shapira commented on KAFKA-2161: - Good catch :) I think other projects are using

Re: Review Request 33614: Patch for KAFKA-2132

2015-04-30 Thread Ashish Singh
> On April 29, 2015, 11:46 p.m., Gwen Shapira wrote: > > Overall, looks good. > > I had a bunch of nits :) Thanks for the review Gwen! > On April 29, 2015, 11:46 p.m., Gwen Shapira wrote: > > build.gradle, line 402 > >

[jira] [Commented] (KAFKA-2132) Move Log4J appender to clients module

2015-04-30 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522094#comment-14522094 ] Ashish K Singh commented on KAFKA-2132: --- Updated reviewboard https://reviews.apache.

Re: Review Request 33614: Patch for KAFKA-2132

2015-04-30 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33614/ --- (Updated April 30, 2015, 7:22 p.m.) Review request for kafka. Bugs: KAFKA-213

[jira] [Updated] (KAFKA-2132) Move Log4J appender to clients module

2015-04-30 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-2132: -- Attachment: KAFKA-2132_2015-04-30_12:22:02.patch > Move Log4J appender to clients module > -

[GitHub] kafka pull request: Adding ability to provide a prefix for the des...

2015-04-30 Thread pedersen
GitHub user pedersen opened a pull request: https://github.com/apache/kafka/pull/59 Adding ability to provide a prefix for the destination topic name. This ... ...can be used to allow two clusters to mirror to each other without causing a loop. You can merge this pull request into

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Sriharsha Chintalapani
Jun << Could you elaborate on why we should not store JSON in ZK? So far,  all existing ZK data are in JSON.  If I have 1,000,000 users in LDAP and 150 get access to Kafka topics  through this mechanism then I have to go and parse and push all of my  changes into zookeeper for it to take affect? 

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Parth Brahmbhatt
Hi Joe, Regarding open question: I changed the title to “Questions resolved after community discussions” let me know if you have a better name. I have a question and a bullet point under each question describing the final decision. Not sure how can I make it any cleaner so appreciate any suggestio

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2161: - Status: Patch Available (was: Open) > Fix a few copyrights >

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2161: - Attachment: KAFKA-2161.patch > Fix a few copyrights > > >

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2161?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522033#comment-14522033 ] Ewen Cheslack-Postava commented on KAFKA-2161: -- Created reviewboard https://r

Review Request 33729: Patch for KAFKA-2161

2015-04-30 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33729/ --- Review request for kafka. Bugs: KAFKA-2161 https://issues.apache.org/jira/b

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-2161: Summary: Fix a few copyrights Key: KAFKA-2161 URL: https://issues.apache.org/jira/browse/KAFKA-2161 Project: Kafka Issue Type: Bug

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14522016#comment-14522016 ] Ewen Cheslack-Postava commented on KAFKA-2123: -- This version is now ready for

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Joe Stein
Gwen << regarding additional authorizers I think having these in the system tests duals as both good confidence in language independency of the changes. It also makes sure that when we release that we don't go breaking Sentry or Ranger or anyone else that wants to integrate. Gwen << Regarding "Au

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2123: - Status: Patch Available (was: In Progress) > Make new consumer offset commit API

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2123: - Attachment: KAFKA-2123_2015-04-30_11:23:05.patch > Make new consumer offset commit

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

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521992#comment-14521992 ] Ewen Cheslack-Postava commented on KAFKA-2123: -- Updated reviewboard https://r

Re: Review Request 33196: Patch for KAFKA-2123

2015-04-30 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33196/ --- (Updated April 30, 2015, 6:23 p.m.) Review request for kafka. Bugs: KAFKA-212

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

2015-04-30 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2160: Summary: DelayedOperationPurgatory should remove the pair in watchersForKey with empty watchers list Key: KAFKA-2160 URL: https://issues.apache.org/jira/browse/KAFKA-2160

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Gwen Shapira
* Regarding additional authorizers: Prasad, who is a PMC on Apache Sentry reviewed the design and confirmed Sentry can integrate with the current APIs. Dapeng Sun, a committer on Sentry had some concerns about the IP privileges and how we prioritize privileges - but nothing that prevents Sentry fro

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Parth Brahmbhatt
I also wanted to send ping to all he committers. This voting thread has been open for > 1 week and has 2 non-bindng +1s. I would appreciate if the committers raised their concerns or casted their votes. Thanks Parth On 4/30/15, 9:52 AM, "Parth Brahmbhatt" wrote: >Hi Joe, Thanks for taking the t

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Parth Brahmbhatt
Hi Joe, Thanks for taking the time to review. * All the open issues already have a resolution , I can open a jira for each one and add the resolution to it and resolve them immediately if you want this for tracking purposes. * We will update system tests to verify that the code works. We have tho

[jira] [Commented] (KAFKA-824) java.lang.NullPointerException in commitOffsets

2015-04-30 Thread John Humphreys (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-824?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521766#comment-14521766 ] John Humphreys commented on KAFKA-824: -- Awesome, thank you for the quick follow-up :).

[jira] [Comment Edited] (KAFKA-2147) Unbalanced replication can cause extreme purgatory growth

2015-04-30 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521670#comment-14521670 ] Evan Huus edited comment on KAFKA-2147 at 4/30/15 4:13 PM: --- When

[jira] [Comment Edited] (KAFKA-2147) Unbalanced replication can cause extreme purgatory growth

2015-04-30 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521670#comment-14521670 ] Evan Huus edited comment on KAFKA-2147 at 4/30/15 4:12 PM: --- When

[jira] [Commented] (KAFKA-824) java.lang.NullPointerException in commitOffsets

2015-04-30 Thread Johannes Zillmann (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-824?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521688#comment-14521688 ] Johannes Zillmann commented on KAFKA-824: - Hi guys, just had a look at this. Think

[jira] [Commented] (KAFKA-2147) Unbalanced replication can cause extreme purgatory growth

2015-04-30 Thread Evan Huus (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521670#comment-14521670 ] Evan Huus commented on KAFKA-2147: -- When I enable TRACE request logging on a single node

[jira] [Commented] (KAFKA-2152) Console producer fails to start when server running with broker.id != 0

2015-04-30 Thread Lior Gonnen (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521655#comment-14521655 ] Lior Gonnen commented on KAFKA-2152: Thanks Gwen. This was very helpful. > Console pr

[jira] [Commented] (KAFKA-824) java.lang.NullPointerException in commitOffsets

2015-04-30 Thread John Humphreys (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-824?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521596#comment-14521596 ] John Humphreys commented on KAFKA-824: -- Has any progress been made on this, or do any

[jira] [Commented] (KAFKA-2159) offsets.topic.segment.bytes and offsets.topic.retention.minutes are ignored

2015-04-30 Thread JIRA
[ https://issues.apache.org/jira/browse/KAFKA-2159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521518#comment-14521518 ] Rafał Boniecki commented on KAFKA-2159: --- Possible reasons: offsets.topic.retention.m

[jira] [Created] (KAFKA-2159) offsets.topic.segment.bytes and offsets.topic.retention.minutes are ignored

2015-04-30 Thread - (JIRA)
- created KAFKA-2159: Summary: offsets.topic.segment.bytes and offsets.topic.retention.minutes are ignored Key: KAFKA-2159 URL: https://issues.apache.org/jira/browse/KAFKA-2159 Project: Kafka Issue Type

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

2015-04-30 Thread Jun Rao
The following is a description of some of my concerns on allowing the same topic multiple times in AlterTopicRequest. ATP has an array of entries, each corresponding to a topic. We allow multiple changes to a topic in a single entry. Those changes may fail to apply independently (e.g., the config

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

2015-04-30 Thread Ismael Juma
Hi all, Kafka currently uses a combination of Review Board and JIRA for contributions and code review. In my opinion, this makes contribution and code review a bit harder than it has to be. I think the approach used by Spark would improve the current situation: "Generally, Spark uses JIRA to tra

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Jun Rao
Joe, Could you elaborate on why we should not store JSON in ZK? So far, all existing ZK data are in JSON. Thanks, Jun On Thu, Apr 30, 2015 at 2:06 AM, Joe Stein wrote: > Hi, sorry I am coming in late to chime back in on this thread and haven't > been able to make the KIP hangouts the last few

[jira] [Updated] (KAFKA-2158) Close all fetchers in AbstractFetcherManager without blocking

2015-04-30 Thread Jiasheng Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiasheng Wang updated KAFKA-2158: - Affects Version/s: 0.8.2.0 Status: Patch Available (was: Open) > Close all fetcher

[jira] [Updated] (KAFKA-2158) Close all fetchers in AbstractFetcherManager without blocking

2015-04-30 Thread Jiasheng Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiasheng Wang updated KAFKA-2158: - Attachment: KAFKA-2158.patch > Close all fetchers in AbstractFetcherManager without blocking > ---

[jira] [Updated] (KAFKA-2158) Close all fetchers in AbstractFetcherManager without blocking

2015-04-30 Thread Jiasheng Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiasheng Wang updated KAFKA-2158: - Status: Open (was: Patch Available) > Close all fetchers in AbstractFetcherManager without blocki

[jira] [Updated] (KAFKA-2158) Close all fetchers in AbstractFetcherManager without blocking

2015-04-30 Thread Jiasheng Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2158?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiasheng Wang updated KAFKA-2158: - Status: Patch Available (was: Open) > Close all fetchers in AbstractFetcherManager without blocki

Review Request 33719: Patch for KAFKA-2158

2015-04-30 Thread Jiasheng Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33719/ --- Review request for kafka. Bugs: KAFKA-2158 https://issues.apache.org/jira/b

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

2015-04-30 Thread Andras Sereny (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2156?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14521229#comment-14521229 ] Andras Sereny commented on KAFKA-2156: -- What is KM? > Possibility to plug in custom

[jira] [Created] (KAFKA-2158) Close all fetchers in AbstractFetcherManager without blocking

2015-04-30 Thread Jiasheng Wang (JIRA)
Jiasheng Wang created KAFKA-2158: Summary: Close all fetchers in AbstractFetcherManager without blocking Key: KAFKA-2158 URL: https://issues.apache.org/jira/browse/KAFKA-2158 Project: Kafka

Re: [VOTE] KIP-11- Authorization design for kafka security

2015-04-30 Thread Joe Stein
Hi, sorry I am coming in late to chime back in on this thread and haven't been able to make the KIP hangouts the last few weeks. Sorry if any of this was brought up already or I missed it. I read through the KIP and the thread(s) and a couple of things jumped out. - Can we break out the open