Re: Review Request 36590: Patch for KAFKA-2275

2015-07-21 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36590/#review92557 --- Hey guys, I realized that most of this code has changed as the desig

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

2015-07-21 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-2275: -- Attachment: KAFKA-2275.patch > Add a ListTopics() API to the new consumer >

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

2015-07-21 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636378#comment-14636378 ] Ashish K Singh commented on KAFKA-2275: --- Created reviewboard https://reviews.apache.

Review Request 36681: Patch for KAFKA-2275

2015-07-21 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36681/ --- Review request for kafka. Bugs: KAFKA-2275 https://issues.apache.org/jira/b

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

2015-07-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2092?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636353#comment-14636353 ] Jason Gustafson commented on KAFKA-2092: [~azaroth], I'm actually not sure that Sa

Re: Review Request 36670: Patch for KAFKA-2355

2015-07-21 Thread Ashish Singh
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36670/#review92551 --- Ship it! Thanks for the patch Edward. - Ashish Singh On July 22,

Re: Review Request 36664: Patch for KAFKA-2353

2015-07-21 Thread Jiangjie Qin
> On July 21, 2015, 11:15 p.m., Gwen Shapira wrote: > > Thanks for looking into that. Exception handling was the most challenging > > part of rewriting SocketServer, so I'm glad to see more eyes on this > > implementation. > > > > I have a concern regarding the right way to handle an unexpecte

[jira] [Comment Edited] (KAFKA-2350) Add KafkaConsumer pause capability

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636319#comment-14636319 ] Jiangjie Qin edited comment on KAFKA-2350 at 7/22/15 5:27 AM: --

[jira] [Commented] (KAFKA-2350) Add KafkaConsumer pause capability

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636319#comment-14636319 ] Jiangjie Qin commented on KAFKA-2350: - [~jkreps][~hachikuji], I actually was not propo

[jira] [Updated] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2353: Attachment: KAFKA-2353_2015-07-21_22:02:24.patch > SocketServer.Processor should catch exception and

[jira] [Commented] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636235#comment-14636235 ] Jiangjie Qin commented on KAFKA-2353: - Updated reviewboard https://reviews.apache.org/

Re: Review Request 36664: Patch for KAFKA-2353

2015-07-21 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36664/ --- (Updated July 22, 2015, 5:02 a.m.) Review request for kafka. Bugs: KAFKA-2353

Re: Review Request 36664: Patch for KAFKA-2353

2015-07-21 Thread Jiangjie Qin
> On July 21, 2015, 11:15 p.m., Gwen Shapira wrote: > > core/src/main/scala/kafka/network/SocketServer.scala, line 465 > > > > > > Turns out that catching Throwable is a really bad idea: > > https://www.sumologic.co

[jira] [Updated] (KAFKA-2355) Add an unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Ribeiro updated KAFKA-2355: -- Issue Type: Test (was: Sub-task) Parent: (was: KAFKA-2345) > Add an unit test to va

[jira] [Updated] (KAFKA-2355) Add an unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Ribeiro updated KAFKA-2355: -- Affects Version/s: 0.8.2.1 Status: Patch Available (was: Open) > Add an unit tes

Re: Review Request 36578: Patch for KAFKA-2338

2015-07-21 Thread Ewen Cheslack-Postava
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36578/#review92526 --- core/src/main/scala/kafka/server/AbstractFetcherThread.scala (line

[jira] [Resolved] (KAFKA-863) System Test - update 0.7 version of kafka-run-class.sh for Migration Tool test cases

2015-07-21 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-863?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-863. - Resolution: Won't Fix Assignee: Ewen Cheslack-Postava Fix Version/s:

[jira] [Commented] (KAFKA-2355) Add an unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636119#comment-14636119 ] Edward Ribeiro commented on KAFKA-2355: --- Hi [~singhashish] and [~gwenshap]. I hope y

[jira] [Commented] (KAFKA-2355) Add an unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636116#comment-14636116 ] Edward Ribeiro commented on KAFKA-2355: --- Created reviewboard https://reviews.apache.

[jira] [Updated] (KAFKA-2355) Add an unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Ribeiro updated KAFKA-2355: -- Attachment: KAFKA-2355.patch > Add an unit test to validate the deletion of a partition marked a

Review Request 36670: Patch for KAFKA-2355

2015-07-21 Thread Edward Ribeiro
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36670/ --- Review request for kafka. Bugs: KAFKA-2355 https://issues.apache.org/jira/b

[jira] [Commented] (KAFKA-2350) Add KafkaConsumer pause capability

2015-07-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636105#comment-14636105 ] Jason Gustafson commented on KAFKA-2350: Hey [~becket_qin], thanks for the suggest

[jira] [Updated] (KAFKA-2355) Add an unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2355?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Ribeiro updated KAFKA-2355: -- Summary: Add an unit test to validate the deletion of a partition marked as deleted (was: Creat

[jira] [Created] (KAFKA-2355) Creating a unit test to validate the deletion of a partition marked as deleted

2015-07-21 Thread Edward Ribeiro (JIRA)
Edward Ribeiro created KAFKA-2355: - Summary: Creating a unit test to validate the deletion of a partition marked as deleted Key: KAFKA-2355 URL: https://issues.apache.org/jira/browse/KAFKA-2355 Projec

[jira] [Commented] (KAFKA-2350) Add KafkaConsumer pause capability

2015-07-21 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636098#comment-14636098 ] Jay Kreps commented on KAFKA-2350: -- +1 on resume instead of unpause though it doesn't mat

Re: Kafka High level consumer rebalancing

2015-07-21 Thread Pranay Agarwal
Thanks Mayuresh, Can I at least control the rebalance of consumers? Currently consumes die after specific partition has no more messages, and there is rebalance of consumes triggered, which causes more consumers to die who get assigned to empty partition(because zookeeper treat empty partition no

[jira] [Comment Edited] (KAFKA-2350) Add KafkaConsumer pause capability

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636074#comment-14636074 ] Jiangjie Qin edited comment on KAFKA-2350 at 7/22/15 12:44 AM: -

[jira] [Commented] (KAFKA-2350) Add KafkaConsumer pause capability

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636074#comment-14636074 ] Jiangjie Qin commented on KAFKA-2350: - I am thinking that currently we keep two collec

Re: [VOTE] Switch to GitHub pull requests for new contributions

2015-07-21 Thread Edward Ribeiro
+1 (non binding) On Tue, Jul 21, 2015 at 7:36 PM, Jay Kreps wrote: > +1 > > -Jay > > On Tue, Jul 21, 2015 at 4:28 AM, Ismael Juma wrote: > > > Hi all, > > > > I would like to start a vote on switching to GitHub pull requests for new > > contributions. To be precise, the vote is on whether we sh

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 14, 2015, 11:09 p.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/PermissionType.scala, line 40 > > > > > > Same comment of Operation.scala also applies here. In addition, the > > retu

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 14, 2015, 11:12 p.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Resource.scala, line 25 > > > > > > In KafkaPrincipal you split like: > > > > val arr: Array[String] = str.spl

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:30 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Acl.scala, line 71 > > > > > > Disclaimer: I am not claiming that you should change the code commented > > here. > >

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 12:49 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Acl.scala, line 86 > > > > > > It's better to return None here, no? Can't return None or nil where a Map[String, Any]

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 14, 2015, 10:59 p.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Operation.scala, line 35 > > > > > > Scala's match is a powerful mechanism but using it to decode as below > > seems b

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 14, 2015, 11:26 p.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Acl.scala, line 110 > > > > > > As the values are fixed you could have written toMap() as below so that > > we can sa

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 14, 2015, 11:01 p.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Acl.scala, line 24 > > > > > > nit: what about switch this lines 23 and 24 and then use WildCardHost > > as replacemen

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:37 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/PermissionType.scala, line 47 > > > > > > The ``return`` here is redundant. In fact the L#46 - L#48 could be > > rewrit

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:40 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Acl.scala, line 136 > > > > > > The return is redundant here. fixed. - Parth -

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:49 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala, line 55 > > > > > > the parenthesis after the ``!`` is not required. fixed. - Parth -

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:47 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Acl.scala, line 86 > > > > > > Or maybe an Map.empty[String, Any] fixed. - Parth --

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:50 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/KafkaPrincipal.scala, line 28 > > > > > > Please, put a space between ``if`` and ``(``. fixed. - Parth

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:59 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/server/KafkaApis.scala, line 540 > > > > > > Please, put a space between ``if`` and ``(``. Fixed. - Parth --

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:43 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/server/KafkaApis.scala, line 624 > > > > > > Lines L#620 and L#621 could be merged (with a &&) into a single > > if-condition. No n

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:55 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/Operation.scala, line 43 > > > > > > The ``return`` here is redundant. Fixed. - Parth -

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 2:02 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/server/KafkaApis.scala, line 139 > > > > > > Put a space between ``if`` and ``(``. Fixed. - Parth --

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 1:57 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/server/KafkaApis.scala, line 151 > > > > > > Please, put a space between ``if`` and ``(`` here. Fixed. - Parth -

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 2:09 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/server/KafkaApis.scala, line 166 > > > > > > Please, put a space between ``if`` and ``(``. Fixed. - Parth --

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 2:15 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/PermissionType.scala, line 21 > > > > > > Just kidding, please remove it. removed. - Parth

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On July 21, 2015, 2:13 a.m., Edward Ribeiro wrote: > > core/src/main/scala/kafka/security/auth/PermissionType.scala, line 21 > > > > > > the semi-colon is trying to scape here, catch it! :) Indeed it was, removed.

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

2015-07-21 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-21_17:08:21.patch > KafkaAuthorizer: Add all public entities,

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
> On June 1, 2015, 1:11 a.m., Jun Rao wrote: > > Thanks for that patch. A few comments below. > > > > Also, two common types of users are consumers and publishers. Currently, if > > you want to allow a user to consume from topic t in consumer group g, you > > have to grant (1) read permission

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

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

Re: Review Request 34492: Patch for KAFKA-2210

2015-07-21 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- (Updated July 22, 2015, 12:08 a.m.) Review request for kafka. Bugs: KAFKA-221

[jira] [Commented] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636040#comment-14636040 ] Gwen Shapira commented on KAFKA-2353: - I left comments in RB :) > SocketServer.Proces

[jira] [Commented] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14636020#comment-14636020 ] Jiangjie Qin commented on KAFKA-2353: - [~gwenshap] Can you help take a look at this pa

Re: Review Request 36664: Patch for KAFKA-2353

2015-07-21 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36664/#review92496 --- Thanks for looking into that. Exception handling was the most challe

[jira] [Commented] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14635987#comment-14635987 ] Jiangjie Qin commented on KAFKA-2353: - Created reviewboard https://reviews.apache.org/

[jira] [Updated] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2353: Status: Patch Available (was: Open) > SocketServer.Processor should catch exception and close the s

[jira] [Updated] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2353?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2353: Attachment: KAFKA-2353.patch > SocketServer.Processor should catch exception and close the socket pr

Review Request 36664: Patch for KAFKA-2353

2015-07-21 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36664/ --- Review request for kafka. Bugs: KAFKA-2353 https://issues.apache.org/jira/b

Re: [VOTE] Switch to GitHub pull requests for new contributions

2015-07-21 Thread Jay Kreps
+1 -Jay On Tue, Jul 21, 2015 at 4:28 AM, Ismael Juma wrote: > Hi all, > > I would like to start a vote on switching to GitHub pull requests for new > contributions. To be precise, the vote is on whether we should: > > * Update the documentation to tell users to use pull requests instead of > pa

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36652/#review92488 --- Ship it! Latest patch looks good to me. - Jiangjie Qin On July 2

[jira] [Resolved] (KAFKA-2354) setting log.dirs property makes tools fail if there is a comma

2015-07-21 Thread Michael Graff (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2354?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Graff resolved KAFKA-2354. -- Resolution: Not A Problem > setting log.dirs property makes tools fail if there is a comma > ---

[jira] [Commented] (KAFKA-2354) setting log.dirs property makes tools fail if there is a comma

2015-07-21 Thread Michael Graff (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14635915#comment-14635915 ] Michael Graff commented on KAFKA-2354: -- Closing as this now appears to be a local err

[jira] [Updated] (KAFKA-2351) Brokers are having a problem shutting down correctly

2015-07-21 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-2351: --- Attachment: KAFKA-2351_2015-07-21_14:58:13.patch > Brokers are having a problem shutting down

[jira] [Commented] (KAFKA-2351) Brokers are having a problem shutting down correctly

2015-07-21 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14635885#comment-14635885 ] Mayuresh Gharat commented on KAFKA-2351: Updated reviewboard https://reviews.apach

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36652/ --- (Updated July 21, 2015, 9:58 p.m.) Review request for kafka. Bugs: KAFKA-2351

[jira] [Commented] (KAFKA-2354) setting log.dirs property makes tools fail if there is a comma

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2354?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14635864#comment-14635864 ] Edward Ribeiro commented on KAFKA-2354: --- Hi [~Skandragon], unfortunately, I was unab

[jira] [Updated] (KAFKA-2354) setting log.dirs property makes tools fail if there is a comma

2015-07-21 Thread Michael Graff (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2354?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Graff updated KAFKA-2354: - Description: If one sets log.dirs=/u1/kafka,/u2/kafka, the tools fail to run: kafka-topics --desc

[jira] [Created] (KAFKA-2354) setting log.dirs property makes tools fail if there is a comma

2015-07-21 Thread Michael Graff (JIRA)
Michael Graff created KAFKA-2354: Summary: setting log.dirs property makes tools fail if there is a comma Key: KAFKA-2354 URL: https://issues.apache.org/jira/browse/KAFKA-2354 Project: Kafka

Re: Submitting a patch (Jira errors)

2015-07-21 Thread Mayuresh Gharat
Resolved this. Thanks, Mayuresh On Tue, Jul 21, 2015 at 12:59 PM, Mayuresh Gharat < gharatmayures...@gmail.com> wrote: > Yes. > > Thanks, > > Mayuresh > > On Tue, Jul 21, 2015 at 12:27 PM, Aditya Auradkar < > aaurad...@linkedin.com.invalid> wrote: > >> Did you setup your jira.ini? >> >> On Tue,

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Grant Henke
> On July 21, 2015, 8:26 p.m., Grant Henke wrote: > > core/src/main/scala/kafka/network/SocketServer.scala, line 266 > > > > > > What errors were seen that should be caught here? Can we catch a more > > specific exc

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Mayuresh Gharat
> On July 21, 2015, 8:26 p.m., Grant Henke wrote: > > core/src/main/scala/kafka/network/SocketServer.scala, line 266 > > > > > > What errors were seen that should be caught here? Can we catch a more > > specific exc

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Grant Henke
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36652/#review92470 --- core/src/main/scala/kafka/network/SocketServer.scala (line 266)

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Mayuresh Gharat
On July 21, 2015, 8:18 p.m., Mayuresh Gharat wrote: > > T Yes. Got it, I thought that we should be catching all exceptions and exit. But doing the above will catch the exception and exit when its shutting down and thats the only thing that this ticket considers. - Mayuresh

Re: Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Jiangjie Qin
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36652/#review92465 --- Thanks for the patch, some comments. core/src/main/scala/kafka/net

[jira] [Commented] (KAFKA-2351) Brokers are having a problem shutting down correctly

2015-07-21 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14635734#comment-14635734 ] Mayuresh Gharat commented on KAFKA-2351: Created reviewboard https://reviews.apach

[jira] [Updated] (KAFKA-2351) Brokers are having a problem shutting down correctly

2015-07-21 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-2351: --- Attachment: KAFKA-2351.patch > Brokers are having a problem shutting down correctly >

[jira] [Updated] (KAFKA-2351) Brokers are having a problem shutting down correctly

2015-07-21 Thread Mayuresh Gharat (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2351?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mayuresh Gharat updated KAFKA-2351: --- Status: Patch Available (was: Open) > Brokers are having a problem shutting down correctly >

Review Request 36652: Patch for KAFKA-2351

2015-07-21 Thread Mayuresh Gharat
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36652/ --- Review request for kafka. Bugs: KAFKA-2351 https://issues.apache.org/jira/b

Re: Submitting a patch (Jira errors)

2015-07-21 Thread Mayuresh Gharat
Yes. Thanks, Mayuresh On Tue, Jul 21, 2015 at 12:27 PM, Aditya Auradkar < aaurad...@linkedin.com.invalid> wrote: > Did you setup your jira.ini? > > On Tue, Jul 21, 2015 at 11:52 AM, Mayuresh Gharat < > gharatmayures...@gmail.com> wrote: > > > Hi, > > > > I had to clean up existing kafka repo on

Re: [VOTE] Switch to GitHub pull requests for new contributions

2015-07-21 Thread Parth Brahmbhatt
+1 (non-binding) Thanks Parth On 7/21/15, 10:24 AM, "Gwen Shapira" wrote: >+1 (binding) on using PRs. > >It sounds like we need additional discussion on how the transition >will happen. Maybe move that to a separate thread, to keep the vote >easy to follow. > >On Tue, Jul 21, 2015 at 4:28 AM, I

Re: Submitting a patch (Jira errors)

2015-07-21 Thread Aditya Auradkar
Did you setup your jira.ini? On Tue, Jul 21, 2015 at 11:52 AM, Mayuresh Gharat < gharatmayures...@gmail.com> wrote: > Hi, > > I had to clean up existing kafka repo on my linux box and start with a > fresh one. > > I followed the instructions here : > > > https://cwiki.apache.org/confluence/displa

[jira] [Created] (KAFKA-2353) SocketServer.Processor should catch exception and close the socket properly in configureNewConnections.

2015-07-21 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2353: --- Summary: SocketServer.Processor should catch exception and close the socket properly in configureNewConnections. Key: KAFKA-2353 URL: https://issues.apache.org/jira/browse/KAFKA-235

[jira] [Closed] (KAFKA-2299) kafka-patch-review tool does not correctly capture testing done

2015-07-21 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2299?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh closed KAFKA-2299. - > kafka-patch-review tool does not correctly capture testing done > --

[jira] [Updated] (KAFKA-2299) kafka-patch-review tool does not correctly capture testing done

2015-07-21 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2299?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ashish K Singh updated KAFKA-2299: -- Resolution: Won't Fix Status: Resolved (was: Patch Available) Moving to Github PRs, so

Submitting a patch (Jira errors)

2015-07-21 Thread Mayuresh Gharat
Hi, I had to clean up existing kafka repo on my linux box and start with a fresh one. I followed the instructions here : https://cwiki.apache.org/confluence/display/KAFKA/Patch+submission+and+review I am trying to upload a patch and I am getting these errors : Configuring reviewboard url to ht

[jira] [Commented] (KAFKA-2345) Attempt to delete a topic already marked for deletion throws ZkNodeExistsException

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14635596#comment-14635596 ] Edward Ribeiro commented on KAFKA-2345: --- [~singhashish], [~gwenshap], sorry, late to

Re: Kafka High level consumer rebalancing

2015-07-21 Thread Mayuresh Gharat
Not sure if you can do that with High level consumer. Thanks, Mayuresh On Tue, Jul 21, 2015 at 10:53 AM, Pranay Agarwal wrote: > Any ideas? > > On Mon, Jul 20, 2015 at 2:34 PM, Pranay Agarwal > wrote: > > > Hi all, > > > > Is there any way I can force Zookeeper/Kafka to rebalance new consumer

Re: Review Request 36578: Patch for KAFKA-2338

2015-07-21 Thread Edward Ribeiro
> On July 21, 2015, 6:57 a.m., Ewen Cheslack-Postava wrote: > > core/src/main/scala/kafka/admin/TopicCommand.scala, line 90 > > > > > > This format call isn't working because it's being called on the second > > strin

Re: Kafka High level consumer rebalancing

2015-07-21 Thread Pranay Agarwal
Any ideas? On Mon, Jul 20, 2015 at 2:34 PM, Pranay Agarwal wrote: > Hi all, > > Is there any way I can force Zookeeper/Kafka to rebalance new consumers > only for subset of total number of partitions. I have a situation where out > of 120 partitions 60 have been already consumed, but the zookeep

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-07-21 Thread Ben Kirwin
That's a fair point. I've added some imagined job logic to the KIP, so we can make sure the proposal stays in sync with the usages we're discussing. (The logic is just a quick sketch for now -- I expect I'll need to elaborate it as we get into more detail, or to address other concerns...) On Tue,

[jira] [Updated] (KAFKA-2342) KafkaConsumer rebalance with in-flight fetch can cause invalid position

2015-07-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2342?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2342: --- Summary: KafkaConsumer rebalance with in-flight fetch can cause invalid position (was: transi

[jira] [Updated] (KAFKA-2342) transient unit test failure in testConsumptionWithBrokerFailures

2015-07-21 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2342?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2342: --- Description: If a rebalance occurs with an in-flight fetch, the new KafkaConsumer can end up

Re: [VOTE] Switch to GitHub pull requests for new contributions

2015-07-21 Thread Gwen Shapira
+1 (binding) on using PRs. It sounds like we need additional discussion on how the transition will happen. Maybe move that to a separate thread, to keep the vote easy to follow. On Tue, Jul 21, 2015 at 4:28 AM, Ismael Juma wrote: > Hi all, > > I would like to start a vote on switching to GitHub

Re: [VOTE] Switch to GitHub pull requests for new contributions

2015-07-21 Thread Guozhang Wang
+1 On Tue, Jul 21, 2015 at 9:31 AM, Grant Henke wrote: > +1 non-binding > > On Tue, Jul 21, 2015 at 11:19 AM, Neha Narkhede wrote: > > > +1 (binding) > > > > Agree with Ismael. We may not want to rush to push the PR right away. > > Having said that, if it works well with say, 10 patches, I'd co

Re: [DISCUSS] KIP-27 - Conditional Publish

2015-07-21 Thread Yasuhiro Matsuda
In KV store usage, all instances are writers, aren't they? There is no leader or master, thus there is no fail over. The offset based CAS ensures an update is based on the latest value and doesn't care who is writing the new value. I think the idea of the offset based CAS is great. I think it work

Re: [VOTE] Switch to GitHub pull requests for new contributions

2015-07-21 Thread Grant Henke
+1 non-binding On Tue, Jul 21, 2015 at 11:19 AM, Neha Narkhede wrote: > +1 (binding) > > Agree with Ismael. We may not want to rush to push the PR right away. > Having said that, if it works well with say, 10 patches, I'd consider that > enough to require the new JIRAs to submit patches using th

Re: Review Request 36565: Patch for KAFKA-2345

2015-07-21 Thread Grant Henke
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/36565/#review92422 --- Ship it! Ship It! - Grant Henke On July 17, 2015, 5:21 p.m., Ash

[jira] [Updated] (KAFKA-2338) Warn users if they change max.message.bytes that they also need to update broker and consumer settings

2015-07-21 Thread Edward Ribeiro (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Ribeiro updated KAFKA-2338: -- Attachment: KAFKA-2338_2015-07-21_13:21:19.patch > Warn users if they change max.message.bytes t

  1   2   >