[jira] [Created] (KAFKA-1914) Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics

2015-02-02 Thread Aditya A Auradkar (JIRA)
Aditya A Auradkar created KAFKA-1914: Summary: Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics Key: KAFKA-1914 URL: https://issues.apache.org/jira/browse/KAFKA-1914

[jira] [Commented] (KAFKA-1809) Refactor brokers to allow listening on multiple ports and IPs

2015-02-02 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14301791#comment-14301791 ] Gwen Shapira commented on KAFKA-1809: - Updated reviewboard

[jira] [Updated] (KAFKA-1809) Refactor brokers to allow listening on multiple ports and IPs

2015-02-02 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1809?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-1809: Attachment: KAFKA-1809_2015-02-02_11:55:16.patch Refactor brokers to allow listening on multiple

[jira] [Commented] (KAFKA-1729) add doc for Kafka-based offset management in 0.8.2

2015-02-02 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14301864#comment-14301864 ] Joel Koshy commented on KAFKA-1729: --- Yes I will commit it today. Re: rolling back to ZK

[jira] [Commented] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

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

[jira] [Updated] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

2015-02-02 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-1886: - Attachment: KAFKA-1886.patch SimpleConsumer swallowing ClosedByInterruptException

Review Request 30527: Patch for KAFKA-1886

2015-02-02 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30527/ --- Review request for kafka. Bugs: KAFKA-1886

Re: Review Request 28769: Patch for KAFKA-1809

2015-02-02 Thread Gwen Shapira
Good idea :) Fixed this, rebased the patch and uploaded a new diff. Gwen On Thu, Jan 29, 2015 at 9:27 AM, Don Bosco Durai bo...@apache.org wrote: +1 I also feel, having security.* would be easy going forward. Thanks Bosco On 1/29/15, 6:08 AM, Jeff Holoman jeff.holo...@gmail.com wrote:

[jira] [Updated] (KAFKA-1914) Count TotalProduceRequestRate and TotalFetchRequestRate in BrokerTopicMetrics

2015-02-02 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1914?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar updated KAFKA-1914: --- Assignee: Aditya Auradkar Count TotalProduceRequestRate and TotalFetchRequestRate in

[jira] [Commented] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

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

[jira] [Commented] (KAFKA-1912) Create a simple request re-routing facility

2015-02-02 Thread Joe Stein (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1912?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14301956#comment-14301956 ] Joe Stein commented on KAFKA-1912: -- I think this makes sense. Do you want to fold

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-02 Thread Joe Stein
Huzzah! Thanks Jun for preparing the release candidates and getting this out to the community. - Joe Stein On Mon, Feb 2, 2015 at 2:27 PM, Jun Rao j...@confluent.io wrote: The following are the results of the votes. +1 binding = 3 votes +1 non-binding = 1 votes -1 = 0 votes 0 = 0 votes

[DISCUSS] ConfigDec Broker Changes on Trunk

2015-02-02 Thread Joe Stein
Hey, I wanted to start a quick convo around some changes on trunk. Not sure this requires a KIP since it is kind of internal and shouldn't affect users but we can decide if so and link this thread to that KIP if so (and keep the discussion going on the thread if makes sense). Before making any

Re: Review Request 30196: Patch for KAFKA-1886

2015-02-02 Thread Aditya Auradkar
On Jan. 26, 2015, 1:28 a.m., Neha Narkhede wrote: core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala, line 235 https://reviews.apache.org/r/30196/diff/1/?file=831148#file831148line235 what is the purpose of this sleep? Aditya Auradkar wrote: I wanted to make

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-02 Thread Neha Narkhede
Great! Thanks Jun for helping with the release and everyone involved for your contributions. On Mon, Feb 2, 2015 at 1:32 PM, Joe Stein joe.st...@stealth.ly wrote: Huzzah! Thanks Jun for preparing the release candidates and getting this out to the community. - Joe Stein On Mon, Feb 2,

Re: Review Request 30403: Patch for KAFKA-1906

2015-02-02 Thread Neha Narkhede
+1 on including a reasonable default for log.dirs that points to data/ in the installation directory. This is followed by some other systems and is probably a little better compared to /tmp. Also, I actually think we should default to production ready settings for most configs, at least those

Re: Review Request 30403: Patch for KAFKA-1906

2015-02-02 Thread Gwen Shapira
While I appreciate production-grade configs a lot, I prefer to ship with usable-out-of-the-box configs. I suspect that large part of the amazing adoption that projects like MySQL, MongoDB, Cassandra and Kafka had is how easy it is to install and get going on a local dev box. Working as DB

Re: [DISCUSS] ConfigDec Broker Changes on Trunk

2015-02-02 Thread Gwen Shapira
Strong +1 from me (obviously). Lots of good reasons to do it: consistency, code reuse, better validations, etc, etc. I had one comment on the patch in RB, but it can also be refactored as follow up JIRA to avoid blocking everyone who is waiting on this. Gwen On Mon, Feb 2, 2015 at 1:31 PM, Joe

Re: Review Request 28769: Patch for KAFKA-1809

2015-02-02 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/#review70642 --- It seems some of the recent trunk changes have crept into this

Re: Review Request 29831: Patch for KAFKA-1476

2015-02-02 Thread Onur Karaman
On Feb. 2, 2015, 5:42 p.m., Neha Narkhede wrote: core/src/main/scala/kafka/utils/ZkUtils.scala, line 758 https://reviews.apache.org/r/29831/diff/7/?file=841974#file841974line758 Let's add consumerGroupOwners inside ZKGroupDirs so we don't have to hardcode zk paths like this. I

Re: Review Request 30482: Add the coordinator to server

2015-02-02 Thread Onur Karaman
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30482/#review70535 --- core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala

Re: Cannot stop Kafka server if zookeeper is shutdown first

2015-02-02 Thread Jaikiran Pai
Thanks for pointing to that repo! I just had a look at it and it appears that the project isn't much active (going by the lack of activity). The latest contribution is from Gwen and that was around 3 months back. I haven't found release plans for that project or a place to ask about it

[jira] [Created] (KAFKA-1915) Integrate checkstyle for java code

2015-02-02 Thread Jay Kreps (JIRA)
Jay Kreps created KAFKA-1915: Summary: Integrate checkstyle for java code Key: KAFKA-1915 URL: https://issues.apache.org/jira/browse/KAFKA-1915 Project: Kafka Issue Type: Improvement

[jira] [Commented] (KAFKA-1729) add doc for Kafka-based offset management in 0.8.2

2015-02-02 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14302676#comment-14302676 ] Joel Koshy commented on KAFKA-1729: --- I committed the patch to trunk - but then realized

[jira] [Commented] (KAFKA-1870) Cannot commit with simpleConsumer on Zookeeper only with Java API

2015-02-02 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1870?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14302679#comment-14302679 ] Joel Koshy commented on KAFKA-1870: --- [~junrao] (at least on trunk) I think the change to

[jira] [Commented] (KAFKA-1756) never allow the replica fetch size to be less than the max message size

2015-02-02 Thread Tong Li (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1756?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14301373#comment-14301373 ] Tong Li commented on KAFKA-1756: I wonder if we really do need two parameters. Can't we

Re: Review Request 30547: Integrate checkstyle.

2015-02-02 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30547/#review70712 --- Looks like a useful tool. build.gradle

Re: Review Request 30547: Integrate checkstyle.

2015-02-02 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30547/#review70713 --- build.gradle https://reviews.apache.org/r/30547/#comment116071

[jira] [Resolved] (KAFKA-1871) Kafka Producer constructor hungs in case of wrong 'serializer.class' property

2015-02-02 Thread Igor Khomenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1871?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Khomenko resolved KAFKA-1871. -- Resolution: Invalid Fix Version/s: 0.8.1.1 It throws that exception in my demo app, but

[jira] [Closed] (KAFKA-1871) Kafka Producer constructor hungs in case of wrong 'serializer.class' property

2015-02-02 Thread Igor Khomenko (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1871?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Igor Khomenko closed KAFKA-1871. Kafka Producer constructor hungs in case of wrong 'serializer.class' property

[jira] [Created] (KAFKA-1913) App hungs when calls producer.send to wrong IP of Kafka broker

2015-02-02 Thread Igor Khomenko (JIRA)
Igor Khomenko created KAFKA-1913: Summary: App hungs when calls producer.send to wrong IP of Kafka broker Key: KAFKA-1913 URL: https://issues.apache.org/jira/browse/KAFKA-1913 Project: Kafka

Re: Review Request 28769: Patch for KAFKA-1809

2015-02-02 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/28769/ --- (Updated Feb. 2, 2015, 7:55 p.m.) Review request for kafka. Bugs: KAFKA-1809

[jira] [Updated] (KAFKA-1886) SimpleConsumer swallowing ClosedByInterruptException

2015-02-02 Thread Aditya A Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya A Auradkar updated KAFKA-1886: - Attachment: KAFKA-1886_2015-02-02_13:57:23.patch SimpleConsumer swallowing

Re: Review Request 30196: Patch for KAFKA-1886

2015-02-02 Thread Aditya Auradkar
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30196/ --- (Updated Feb. 2, 2015, 9:57 p.m.) Review request for kafka and Joel Koshy.

Re: Review Request 29831: Patch for KAFKA-1476

2015-02-02 Thread Neha Narkhede
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/29831/#review70581 --- Onur, we should be able to check in after these review comments are

Re: Cannot stop Kafka server if zookeeper is shutdown first

2015-02-02 Thread Gwen Shapira
I did! Thanks for clarifying :) The client that is part of Zookeeper itself actually does support timeouts. On Mon, Feb 2, 2015 at 9:54 AM, Guozhang Wang wangg...@gmail.com wrote: Hi Jaikiran, I think Gwen was talking about contributing to ZkClient project:

Re: Cannot stop Kafka server if zookeeper is shutdown first

2015-02-02 Thread Guozhang Wang
Hi Jaikiran, I think Gwen was talking about contributing to ZkClient project: https://github.com/sgroschupf/zkclient Guozhang On Sun, Feb 1, 2015 at 5:30 AM, Jaikiran Pai jai.forums2...@gmail.com wrote: Hi Gwen, Yes, the KafkaZkClient is a wrapper around ZkClient and not a complete

Re: [kafka-clients] Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-02 Thread Jay Kreps
Yay! -Jay On Mon, Feb 2, 2015 at 2:23 PM, Neha Narkhede n...@confluent.io wrote: Great! Thanks Jun for helping with the release and everyone involved for your contributions. On Mon, Feb 2, 2015 at 1:32 PM, Joe Stein joe.st...@stealth.ly wrote: Huzzah! Thanks Jun for preparing the

[jira] [Commented] (KAFKA-1915) Integrate checkstyle for java code

2015-02-02 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14302796#comment-14302796 ] Jay Kreps commented on KAFKA-1915: -- Created reviewboard

Review Request 30547: Integrate checkstyle.

2015-02-02 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/30547/ --- Review request for kafka. Bugs: KAFKA-1915

[jira] [Updated] (KAFKA-1915) Integrate checkstyle for java code

2015-02-02 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1915?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1915: - Status: Patch Available (was: Open) Integrate checkstyle for java code

[jira] [Updated] (KAFKA-1915) Integrate checkstyle for java code

2015-02-02 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1915?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1915: - Attachment: KAFKA-1915.patch Integrate checkstyle for java code --

[jira] [Commented] (KAFKA-1729) add doc for Kafka-based offset management in 0.8.2

2015-02-02 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1729?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14301736#comment-14301736 ] Jun Rao commented on KAFKA-1729: Joel, Thanks for the doc. Looks good. Just a few minor

Re: [VOTE] 0.8.2.0 Candidate 3

2015-02-02 Thread Jun Rao
The following are the results of the votes. +1 binding = 3 votes +1 non-binding = 1 votes -1 = 0 votes 0 = 0 votes The vote passes. I will release artifacts to maven central, update the dist svn and download site. Will send out an announce after that. Thanks everyone that contributed to the