Re: Inquiry regarding unreviewed patch (KAFKA-1614)

2015-07-14 Thread Jiangjie Qin
Hi Kim, I took a quick look at the patch. It seems all the information you need are available in Kafka now. There are several mBeans you can take a look. They are in kafka.log. Including log start/end offsets, size and number of log segments. Thanks, Jiangjie (Becket) Qin On 7/13/15, 5:01 PM,

[jira] [Commented] (KAFKA-2260) Allow specifying expected offset on produce

2015-07-14 Thread Ben Kirwin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2260?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627484#comment-14627484 ] Ben Kirwin commented on KAFKA-2260: --- Will do; thanks! > Allow specifying expected offse

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627485#comment-14627485 ] Jiangjie Qin commented on KAFKA-1835: - My example for use case (3) is probably a bad o

[jira] [Commented] (KAFKA-2260) Allow specifying expected offset on produce

2015-07-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2260?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627482#comment-14627482 ] Ewen Cheslack-Postava commented on KAFKA-2260: -- [~bkirwi] Unfortunately I don

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627474#comment-14627474 ] Joel Koshy commented on KAFKA-1835: --- My earlier comment was more to clarify the reasonin

[jira] [Commented] (KAFKA-2260) Allow specifying expected offset on produce

2015-07-14 Thread Ben Kirwin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2260?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627460#comment-14627460 ] Ben Kirwin commented on KAFKA-2260: --- Hi [~ewencp] -- thanks for the interest! I'd be gla

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627400#comment-14627400 ] Guozhang Wang commented on KAFKA-1835: -- Joel, I think you are referring to some patte

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627395#comment-14627395 ] Jay Kreps commented on KAFKA-1835: -- Changing the return type of a method is a hard compat

Re: Request to be added to the contributor list

2015-07-14 Thread Jun Rao
Added. Thanks, Jun On Tue, Jul 14, 2015 at 11:58 AM, Aravind Selvan wrote: > Hi, > > Please add me to the contributor list. Thanks. > > ~Aravind Selvan >

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

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627383#comment-14627383 ] Jason Gustafson commented on KAFKA-2123: Updated reviewboard https://reviews.apach

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

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123_2015-07-14_18:21:38.patch > Make new consumer offset commit API use cal

Re: Review Request 36333: Patch for KAFKA-2123

2015-07-14 Thread Jason Gustafson
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36333/ --- (Updated July 15, 2015, 1:21 a.m.) Review request for kafka. Bugs: KAFKA-2123

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Jay Kreps
Yeah we really can't add any dependencies for that client jar. Maybe we could move the perf test to the tools jar though? -Jay On Tue, Jul 14, 2015 at 6:01 PM, Geoffrey Anderson wrote: > Hi all, my pull request here: > > https://github.com/apache/kafka/pull/70/files#diff-59f3fe36571d1eee9f923df

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627375#comment-14627375 ] Joel Koshy commented on KAFKA-1835: --- bq. I think case (3) is only for users who are will

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Geoffrey Anderson
Hi all, my pull request here: https://github.com/apache/kafka/pull/70/files#diff-59f3fe36571d1eee9f923df927a643eb would introduce a client-side dependency on the json-simple package. It is only used in a tool (VerifiableProducer.java), but it sounds like I should probably use Jackson instead? Fee

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

2015-07-14 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2145?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2145 started by Parth Brahmbhatt. --- > An option to add topic owners. > --- > >

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

2015-07-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2145?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627359#comment-14627359 ] ASF GitHub Bot commented on KAFKA-2145: --- GitHub user Parth-Brahmbhatt opened a pull

[GitHub] kafka pull request: KAFKA-2145: Add a log config so users can defi...

2015-07-14 Thread Parth-Brahmbhatt
GitHub user Parth-Brahmbhatt opened a pull request: https://github.com/apache/kafka/pull/77 KAFKA-2145: Add a log config so users can define topic owners. You can merge this pull request into a Git repository by running: $ git pull https://github.com/Parth-Brahmbhatt/kafka KAF

Re: [VOTE] KIP-26 Add Copycat connector framework for data import/export

2015-07-14 Thread Sriram Subramanian
+1. This was long due! On 7/14/15, 4:42 PM, "Guozhang Wang" wrote: >+1. Thanks Ewen!! > >On Tue, Jul 14, 2015 at 3:01 PM, Neha Narkhede wrote: > >> +1 (binding) >> >> Thanks Ewen for taking on something that the Kafka project has long >>waited >> for! >> >> On Tue, Jul 14, 2015 at 2:58 PM, Jay

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

2015-07-14 Thread Ismael Juma
On Tue, Jul 14, 2015 at 6:15 PM, Jun Rao wrote: > I made a couple of changes to the new Jenkins job. Could you try again? > It's still not working, unfortunately. It may or may not be related to: https://blogs.apache.org/infra/entry/mirroring_to_github_issues For b, if we can't easily change t

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Jay Kreps
Ah, makes sense. Yes that addresses my concerns. -Jay On Tue, Jul 14, 2015 at 5:19 PM, Ismael Juma wrote: > Hi Jay, > > Comments inline. > > On Tue, Jul 14, 2015 at 11:04 PM, Jay Kreps wrote: > > > Is this going to become a dependency for core and then transitively for > the > > old clients? >

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Ismael Juma
Hi Jay, Comments inline. On Tue, Jul 14, 2015 at 11:04 PM, Jay Kreps wrote: > Is this going to become a dependency for core and then transitively for the > old clients? That's right. > The current json library is definitely not great, but it does > parse json and it's not used in any contex

[jira] [Commented] (KAFKA-2092) New partitioning for better load balancing

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2092?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627325#comment-14627325 ] Jason Gustafson commented on KAFKA-2092: [~azaroth], thanks for the update. I thin

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Ismael Juma
On Tue, Jul 14, 2015 at 10:46 PM, Joe Stein wrote: > Fasterxml/Jackson +1 to that. The scala databinds to case classes are gr8. > To be clear, case classes support would require the Scala module for Jackson and the Scala versions headache that goes with it (2.9 support is likely to be an issue,

[jira] [Commented] (KAFKA-2210) KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation.

2015-07-14 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627292#comment-14627292 ] Edward Ribeiro commented on KAFKA-2210: --- Hi [~parth.brahmbhatt], left some minor com

Re: [CFP] Developer showcase at Strata NYC

2015-07-14 Thread Gwen Shapira
You (or someone working on them) should submit the projects :) On Tue, Jul 14, 2015 at 4:27 PM, Neha Narkhede wrote: > All of the ones you've mentioned there sound great. I'd want to see copycat > there. > > On Tue, Jul 14, 2015 at 4:22 PM, Gwen Shapira wrote: > >> Hi Kafka fans, >> >> Strata NY

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Ismael Juma
Ewen, On Tue, Jul 14, 2015 at 10:41 PM, Ewen Cheslack-Postava wrote: > Currently the clients/server mismatch wouldn't be an issue since there are > no client-side uses of JSON, right? That said, if Copycat ends up included > in Kafka we'll need to provide at least one serializer which would be >

Re: [VOTE] KIP-26 Add Copycat connector framework for data import/export

2015-07-14 Thread Guozhang Wang
+1. Thanks Ewen!! On Tue, Jul 14, 2015 at 3:01 PM, Neha Narkhede wrote: > +1 (binding) > > Thanks Ewen for taking on something that the Kafka project has long waited > for! > > On Tue, Jul 14, 2015 at 2:58 PM, Jay Kreps wrote: > > > +1 > > > > Super excited! > > > > -Jay > > > > On Tue, Jul 14,

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91682 --- core/src/main/scala/kafka/security/auth/ResourceType.scala (line 45

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627279#comment-14627279 ] Guozhang Wang commented on KAFKA-1835: -- Thanks [~becket_qin] for the summary. Some fo

[GitHub] kafka pull request: kafka-2320; Test commit

2015-07-14 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/76 kafka-2320; Test commit Trying to test the CI build created via kafka-2320. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-2320 A

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91680 --- core/src/test/scala/unit/kafka/security/auth/AclTest.scala (line 24

[GitHub] kafka pull request: kafka-2320; Test PR builder

2015-07-14 Thread ijuma
Github user ijuma closed the pull request at: https://github.com/apache/kafka/pull/75 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled

[GitHub] kafka pull request: KAFKA-1595; Remove deprecated and slower scala...

2015-07-14 Thread ijuma
Github user ijuma closed the pull request at: https://github.com/apache/kafka/pull/74 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled

[jira] [Commented] (KAFKA-1595) Remove deprecated and slower scala JSON parser from kafka.consumer.TopicCount

2015-07-14 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1595?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627269#comment-14627269 ] ASF GitHub Bot commented on KAFKA-1595: --- Github user ijuma closed the pull request a

[GitHub] kafka pull request: kafka-2320; Test PR builder

2015-07-14 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/75 kafka-2320; Test PR builder Just trying to trigger a build in Jenkins. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-2320 Altern

Re: [CFP] Developer showcase at Strata NYC

2015-07-14 Thread Neha Narkhede
All of the ones you've mentioned there sound great. I'd want to see copycat there. On Tue, Jul 14, 2015 at 4:22 PM, Gwen Shapira wrote: > Hi Kafka fans, > > Strata NYC is looking for emerging, open source, big data projects to > highlight in the developer showcase: > http://strataconf.com/big-da

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91679 --- core/src/main/scala/kafka/security/auth/Acl.scala (line 110)

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627263#comment-14627263 ] Mayuresh Gharat commented on KAFKA-1835: Either way user will have to block or han

[CFP] Developer showcase at Strata NYC

2015-07-14 Thread Gwen Shapira
Hi Kafka fans, Strata NYC is looking for emerging, open source, big data projects to highlight in the developer showcase: http://strataconf.com/big-data-conference-ny-2015/public/cfp/409 Depending on how you look at it, it is either science faire for grownups or free conference booth for open sou

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91676 --- core/src/main/scala/kafka/security/auth/Resource.scala (line 25)

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91675 --- core/src/main/scala/kafka/security/auth/PermissionType.scala (line

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91672 --- core/src/main/scala/kafka/security/auth/Acl.scala (line 24)

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91671 --- core/src/main/scala/kafka/security/auth/Operation.scala (line 35) <

[jira] [Updated] (KAFKA-1901) Move Kafka version to be generated in code by build (instead of in manifest)

2015-07-14 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1901?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-1901: -- Reviewer: Joel Koshy > Move Kafka version to be generated in code by build (instead of in manifest) > --

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627190#comment-14627190 ] Joel Koshy commented on KAFKA-1835: --- bq. Similar proposal applies to KAFKA-2275 https:/

[jira] [Commented] (KAFKA-2275) Add a ListTopics() API to the new consumer

2015-07-14 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627186#comment-14627186 ] Onur Karaman commented on KAFKA-2275: - Currently, the consumer has the following API:

[jira] [Commented] (KAFKA-1835) Kafka new producer needs options to make blocking behavior explicit

2015-07-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1835?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627184#comment-14627184 ] Jiangjie Qin commented on KAFKA-1835: - We discussed this in today's KIP hangout. Just

Re: Review Request 36333: Patch for KAFKA-2123

2015-07-14 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36333/#review91651 --- LGTM overall, just some minor comments below. clients/src/main/jav

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Joe Stein
Maybe after the existing scala clients are deprecated. ~ Joestein On Jul 14, 2015 6:04 PM, "Jay Kreps" wrote: > Is this going to become a dependency for core and then transitively for the > old clients? The current json library is definitely not great, but it does > parse json and it's not used

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Jay Kreps
Is this going to become a dependency for core and then transitively for the old clients? The current json library is definitely not great, but it does parse json and it's not used in any context where performance is a concern. Because the older clients aren't well modularized, adding core dependen

Re: [VOTE] KIP-26 Add Copycat connector framework for data import/export

2015-07-14 Thread Neha Narkhede
+1 (binding) Thanks Ewen for taking on something that the Kafka project has long waited for! On Tue, Jul 14, 2015 at 2:58 PM, Jay Kreps wrote: > +1 > > Super excited! > > -Jay > > On Tue, Jul 14, 2015 at 2:09 PM, Ewen Cheslack-Postava > wrote: > > > Hi all, > > > > Let's start a vote on KIP-26

Re: [VOTE] KIP-26 Add Copycat connector framework for data import/export

2015-07-14 Thread Jay Kreps
+1 Super excited! -Jay On Tue, Jul 14, 2015 at 2:09 PM, Ewen Cheslack-Postava wrote: > Hi all, > > Let's start a vote on KIP-26: Add Copycat connector framework for data > import/export > > For reference, here's the wiki: > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Joe Stein
Fasterxml/Jackson +1 to that. The scala databinds to case classes are gr8. ~ Joestein On Jul 14, 2015 5:42 PM, "Ewen Cheslack-Postava" wrote: > Currently the clients/server mismatch wouldn't be an issue since there are > no client-side uses of JSON, right? That said, if Copycat ends up included

Re: [DISCUSS] Json libraries for Kafka

2015-07-14 Thread Ewen Cheslack-Postava
Currently the clients/server mismatch wouldn't be an issue since there are no client-side uses of JSON, right? That said, if Copycat ends up included in Kafka we'll need to provide at least one serializer which would be written in Java and I suspect some people would like JSON to be a candidate for

[jira] [Commented] (KAFKA-2210) KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation.

2015-07-14 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627049#comment-14627049 ] Parth Brahmbhatt commented on KAFKA-2210: - Updated reviewboard https://reviews.apa

[jira] [Updated] (KAFKA-2210) KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation.

2015-07-14 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-2210: Attachment: KAFKA-2210_2015-07-14_14:13:19.patch > KafkaAuthorizer: Add all public entities,

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated July 14, 2015, 9:13 p.m.) Review request for kafka. Bugs: KAFKA-2210

Re: [VOTE] KIP-26 Add Copycat connector framework for data import/export

2015-07-14 Thread Joe Stein
+1 (binding) ~ Joe Stein - - - - - - - - - - - - - - - - - - - [image: Logo-Black.jpg] http://www.elodina.net http://www.stealth.ly - - - - - - - - - - - - - - - - - - - On Tue, Jul 14, 2015 at 5:09 PM, Ewen Cheslack-Postava wrote: > Hi all, > > Let's start a vote on KIP-26: Add Copy

[VOTE] KIP-26 Add Copycat connector framework for data import/export

2015-07-14 Thread Ewen Cheslack-Postava
Hi all, Let's start a vote on KIP-26: Add Copycat connector framework for data import/export For reference, here's the wiki: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=58851767 And the mailing list thread (split across two months): http://mail-archives.apache.org/mod_mbox/ka

[jira] [Commented] (KAFKA-2055) ConsumerBounceTest.testSeekAndCommitWithBrokerFailures transient failure

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2055?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627041#comment-14627041 ] Guozhang Wang commented on KAFKA-2055: -- [~lvfangmin] Thanks for the patch, I agree th

[jira] [Commented] (KAFKA-2334) Prevent HW from going back during leader failover

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627040#comment-14627040 ] Guozhang Wang commented on KAFKA-2334: -- One possible solution to this issue is to let

[jira] [Created] (KAFKA-2334) Prevent HW from going back during leader failover

2015-07-14 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2334: Summary: Prevent HW from going back during leader failover Key: KAFKA-2334 URL: https://issues.apache.org/jira/browse/KAFKA-2334 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-2260) Allow specifying expected offset on produce

2015-07-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2260?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14627018#comment-14627018 ] Ewen Cheslack-Postava commented on KAFKA-2260: -- Hey [~bkirwi], this looks int

Re: Review Request 36333: Patch for KAFKA-2123

2015-07-14 Thread Jason Gustafson
> On July 14, 2015, 6 p.m., Ewen Cheslack-Postava wrote: > > clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java, > > line 274 > > > > > > Seems like we're not handling this anymore? sen

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

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626987#comment-14626987 ] Jason Gustafson commented on KAFKA-2123: Updated reviewboard https://reviews.apach

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

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123_2015-07-14_13:20:25.patch > Make new consumer offset commit API use cal

Re: Review Request 36333: Patch for KAFKA-2123

2015-07-14 Thread Jason Gustafson
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36333/ --- (Updated July 14, 2015, 8:21 p.m.) Review request for kafka. Bugs: KAFKA-2123

[jira] [Updated] (KAFKA-2055) ConsumerBounceTest.testSeekAndCommitWithBrokerFailures transient failure

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2055?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2055: - Reviewer: Jason Gustafson > ConsumerBounceTest.testSeekAndCommitWithBrokerFailures transient failu

[jira] [Updated] (KAFKA-2300) Error in controller log when broker tries to rejoin cluster

2015-07-14 Thread Bob Cotton (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2300?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bob Cotton updated KAFKA-2300: -- Attachment: KAFKA-2300-controller-logs.tar.gz Controller logs > Error in controller log when broker tr

[jira] [Commented] (KAFKA-1977) Make logEndOffset available in the Zookeeper consumer

2015-07-14 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626900#comment-14626900 ] James Cheng commented on KAFKA-1977: Will maxEndOffset will be exposed in the new Java

Request to be added to the contributor list

2015-07-14 Thread Aravind Selvan
Hi, Please add me to the contributor list. Thanks. ~Aravind Selvan

[jira] [Updated] (KAFKA-2323) Simplify ScalaTest dependency versions

2015-07-14 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-2323: - Reviewer: Jun Rao > Simplify ScalaTest dependency versions > -

[jira] [Updated] (KAFKA-2321) Introduce CONTRIBUTING.md

2015-07-14 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2321?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-2321: - Reviewer: Neha Narkhede > Introduce CONTRIBUTING.md > - > >

[jira] [Updated] (KAFKA-2324) Update to Scala 2.11.7

2015-07-14 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-2324: - Reviewer: Neha Narkhede > Update to Scala 2.11.7 > -- > > Key:

[jira] [Updated] (KAFKA-2311) Consumer's ensureNotClosed method not thread safe

2015-07-14 Thread Neha Narkhede (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2311?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Neha Narkhede updated KAFKA-2311: - Reviewer: Neha Narkhede > Consumer's ensureNotClosed method not thread safe >

[jira] [Updated] (KAFKA-2329) Consumers balance fails when multiple consumers are started simultaneously.

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2329?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2329: - Reviewer: Guozhang Wang > Consumers balance fails when multiple consumers are started simultaneous

[jira] [Updated] (KAFKA-2032) ConsumerConfig doesn't validate partition.assignment.strategy values

2015-07-14 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2032?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2032: - Reviewer: Guozhang Wang > ConsumerConfig doesn't validate partition.assignment.strategy values > -

[jira] [Assigned] (KAFKA-2260) Allow specifying expected offset on produce

2015-07-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava reassigned KAFKA-2260: Assignee: Ewen Cheslack-Postava > Allow specifying expected offset on produ

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/#review91638 --- core/src/main/scala/kafka/security/auth/Acl.scala (line 24)

[jira] [Commented] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-07-14 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626814#comment-14626814 ] Aditya Auradkar commented on KAFKA-2205: [~junrao] Thanks! I addressed your remain

[jira] [Commented] (KAFKA-972) MetadataRequest returns stale list of brokers

2015-07-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-972?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626807#comment-14626807 ] Ashish K Singh commented on KAFKA-972: -- Thanks Jun! > MetadataRequest returns stale l

[jira] [Comment Edited] (KAFKA-972) MetadataRequest returns stale list of brokers

2015-07-14 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-972?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626807#comment-14626807 ] Ashish K Singh edited comment on KAFKA-972 at 7/14/15 6:22 PM: --

Re: Review Request 36333: Patch for KAFKA-2123

2015-07-14 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36333/#review91646 --- clients/src/main/java/org/apache/kafka/clients/consumer/internals/F

Re: Review Request 34554: Patch for KAFKA-2205

2015-07-14 Thread Aditya Auradkar
> On July 9, 2015, 2:28 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/admin/ConfigCommand.scala, line 123 > > > > > > Could we list the valid configs name for each entity-type as we did in > > TopicCommand? I'

[jira] [Updated] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-07-14 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2205: - Attachment: KAFKA-2205_2015-07-14_10:36:36.patch > Generalize TopicConfigManager to handle

Re: Review Request 34554: Patch for KAFKA-2205

2015-07-14 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34554/ --- (Updated July 14, 2015, 5:37 p.m.) Review request for kafka, Joel Koshy and Jun

Re: Review Request 34554: Patch for KAFKA-2205

2015-07-14 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34554/ --- (Updated July 14, 2015, 5:36 p.m.) Review request for kafka, Joel Koshy and Jun

[jira] [Commented] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-07-14 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626734#comment-14626734 ] Aditya A Auradkar commented on KAFKA-2205: -- Updated reviewboard https://reviews.a

[jira] [Updated] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-07-14 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-2205: - Attachment: KAFKA-2205_2015-07-14_10:33:47.patch > Generalize TopicConfigManager to handle

[jira] [Commented] (KAFKA-2205) Generalize TopicConfigManager to handle multiple entity configs

2015-07-14 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626731#comment-14626731 ] Aditya A Auradkar commented on KAFKA-2205: -- Updated reviewboard https://reviews.a

Re: Review Request 34554: Patch for KAFKA-2205

2015-07-14 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34554/ --- (Updated July 14, 2015, 5:34 p.m.) Review request for kafka, Joel Koshy and Jun

Re: Review Request 34554: Patch for KAFKA-2205

2015-07-14 Thread Aditya Auradkar
> On July 9, 2015, 2:28 a.m., Jun Rao wrote: > > core/src/main/scala/kafka/server/ConfigHandler.scala, line 27 > > > > > > Do we need JavaConversions? If this is needed, it would be better to > > import it in the con

[jira] [Commented] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2015-07-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626707#comment-14626707 ] Jiangjie Qin commented on KAFKA-1788: - [~parth.brahmbhatt], yes, we can resolve this t

[jira] [Resolved] (KAFKA-1788) producer record can stay in RecordAccumulator forever if leader is no available

2015-07-14 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1788?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin resolved KAFKA-1788. - Resolution: Fixed Will be covered by KAFKA-2120 > producer record can stay in RecordAccumulator f

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

2015-07-14 Thread Jun Rao
I made a couple of changes to the new Jenkins job. Could you try again? For b, if we can't easily change the behavior of pull request bot, we can also just flip back and forth btw "In Progress" and "Open". Thanks, Jun On Tue, Jul 14, 2015 at 2:13 AM, Ismael Juma wrote: > Hi Jun, > > On Tue, J

[jira] [Commented] (KAFKA-2210) KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation.

2015-07-14 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14626663#comment-14626663 ] Parth Brahmbhatt commented on KAFKA-2210: - Updated reviewboard https://reviews.apa

[jira] [Updated] (KAFKA-2210) KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation.

2015-07-14 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-2210: Attachment: KAFKA-2210_2015-07-14_10:02:19.patch > KafkaAuthorizer: Add all public entities,

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-14 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated July 14, 2015, 5:02 p.m.) Review request for kafka. Bugs: KAFKA-2210

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

2015-07-14 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: Guozhang Wang (was: Jay Kreps) > Make new consumer offset commit API use callback + future > --

  1   2   >