[jira] [Updated] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deepthi updated KAFKA-2207: --- Attachment: KAFKA-2207.patch The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest

[jira] [Updated] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deepthi updated KAFKA-2207: --- Status: Patch Available (was: Open) KAFKA-2207 Patch has fixed the issue The testCannotSendToInternalTopic

Re: Review Request 33065: Patch for KAFKA-1928

2015-05-20 Thread Gwen Shapira
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/ --- (Updated May 20, 2015, 10:42 a.m.) Review request for kafka. Bugs: 1928 and

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

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

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

2015-05-20 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-05-20_13:41:37.patch Move kafka.network over to using the network

Re: Review Request 33065: Patch for KAFKA-1928

2015-05-20 Thread Gwen Shapira
On May 19, 2015, 11:34 p.m., Jun Rao wrote: core/src/main/scala/kafka/network/SocketServer.scala, line 431 https://reviews.apache.org/r/33065/diff/14/?file=963545#file963545line431 We probably just want to catch Exception. For unexpected error, we will just let it propagate and

[jira] [Issue Comment Deleted] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deepthi updated KAFKA-2207: --- Comment: was deleted (was: KAFKA-2207 Patch has fixed the issue) The testCannotSendToInternalTopic test

[jira] [Created] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
Deepthi created KAFKA-2207: -- Summary: The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception: Key: KAFKA-2207 URL:

[jira] [Updated] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deepthi updated KAFKA-2207: --- Status: Patch Available (was: Open) The testCannotSendToInternalTopic test method in

[jira] [Updated] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deepthi updated KAFKA-2207: --- Description: kafka.api.ProducerFailureHandlingTest testCannotSendToInternalTopic FAILED

[jira] [Updated] (KAFKA-2207) The testCannotSendToInternalTopic test method in ProducerFailureHandlingTest fails consistently with the following exception:

2015-05-20 Thread Deepthi (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2207?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Deepthi updated KAFKA-2207: --- Status: Open (was: Patch Available) The testCannotSendToInternalTopic test method in

[jira] [Updated] (KAFKA-1334) Coordinator should detect consumer failures

2015-05-20 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-1334: Summary: Coordinator should detect consumer failures (was: Add failure detection capability to the

Re: Review Request 34397: Patch for KAFKA-1374

2015-05-20 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34397/#review84545 --- Ship it! Assuming some integration tests will be added in the

Re: Review Request 34450: Fix KAFKA-2017; rebased

2015-05-20 Thread Onur Karaman
On May 20, 2015, 5:15 p.m., Onur Karaman wrote: I only did a brief skim. This optimization tries to switch consumers over to a new coordinator without a rebalance. From my understanding, the consumers would detect a coordinator failure, discover the new coordinator to work with, and

[jira] [Comment Edited] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552582#comment-14552582 ] Guozhang Wang edited comment on KAFKA-2017 at 5/20/15 4:18 PM:

Re: Review Request 34450: Fix KAFKA-2017; rebased

2015-05-20 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34450/ --- (Updated May 20, 2015, 4:13 p.m.) Review request for kafka. Summary

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552574#comment-14552574 ] Guozhang Wang commented on KAFKA-2017: -- Updated reviewboard

[jira] [Updated] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2017: - Attachment: KAFKA-2017_2015-05-20_09:13:39.patch Persist Coordinator State for Coordinator

[jira] [Updated] (KAFKA-1333) Add consumer co-ordinator module to the server

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1333?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-1333: - Resolution: Fixed Status: Resolved (was: Patch Available) Resolve this ticket as a

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

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-1334. -- Resolution: Fixed Resolve this ticket as a result of KAFKA-2160. Add failure detection

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552582#comment-14552582 ] Guozhang Wang commented on KAFKA-2017: -- This patch used the second ZK data structure

[jira] [Created] (KAFKA-2209) Change client quotas dynamically using DynamicConfigManager

2015-05-20 Thread Aditya Auradkar (JIRA)
Aditya Auradkar created KAFKA-2209: -- Summary: Change client quotas dynamically using DynamicConfigManager Key: KAFKA-2209 URL: https://issues.apache.org/jira/browse/KAFKA-2209 Project: Kafka

[jira] [Commented] (KAFKA-1335) Add rebalancing logic to the coordinator / consumer

2015-05-20 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552707#comment-14552707 ] Onur Karaman commented on KAFKA-1335: - Part 1 was handled by:

[jira] [Commented] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552712#comment-14552712 ] Guozhang Wang commented on KAFKA-1374: -- Committed to trunk. I think we need some

[jira] [Updated] (KAFKA-1334) Coordinator should detect consumer failures

2015-05-20 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1334?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Onur Karaman updated KAFKA-1334: Description: Add failure detection capability to the coordinator when group management is used.

[jira] [Issue Comment Deleted] (KAFKA-2201) Open file handle leak

2015-05-20 Thread Albert Visagie (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2201?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Albert Visagie updated KAFKA-2201: -- Comment: was deleted (was: Apologies, I forgot to add: the number of open file handles climbs

[jira] [Commented] (KAFKA-2201) Open file handle leak

2015-05-20 Thread Albert Visagie (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552565#comment-14552565 ] Albert Visagie commented on KAFKA-2201: --- Apologies, I forgot to add: the number of

[jira] [Commented] (KAFKA-2201) Open file handle leak

2015-05-20 Thread Albert Visagie (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552564#comment-14552564 ] Albert Visagie commented on KAFKA-2201: --- Apologies, I forgot to add: the number of

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

2015-05-20 Thread Joel Koshy
The fact that I understand the producer internals and am still struggling to understand the implications of the different settings, how I would set them, and how they potentially interact such that I could set invalid combinations seems like a red flag to me... Being able to say I want

[jira] [Commented] (KAFKA-2017) Persist Coordinator State for Coordinator Failover

2015-05-20 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552703#comment-14552703 ] Onur Karaman commented on KAFKA-2017: - I left a review. Persist Coordinator State

[jira] [Commented] (KAFKA-1334) Coordinator should detect consumer failures

2015-05-20 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552701#comment-14552701 ] Onur Karaman commented on KAFKA-1334: - This ticket originally was two things: 1.

[jira] [Created] (KAFKA-2208) Consumer should detect coordinator failures

2015-05-20 Thread Onur Karaman (JIRA)
Onur Karaman created KAFKA-2208: --- Summary: Consumer should detect coordinator failures Key: KAFKA-2208 URL: https://issues.apache.org/jira/browse/KAFKA-2208 Project: Kafka Issue Type: Sub-task

Re: Review Request 33204: Patch for KAFKA-1646 add test cases

2015-05-20 Thread Jay Kreps
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33204/#review84554 --- Ship it! This looks good to be, but it would be good to get a

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

2015-05-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552777#comment-14552777 ] Jay Kreps commented on KAFKA-1646: -- This looks good to me but it would be good to get a

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

2015-05-20 Thread Parth Brahmbhatt (JIRA)
Parth Brahmbhatt created KAFKA-2210: --- Summary: KafkaAuthorizer: Add all public entities, config changes and changes to KafkaAPI and kafkaServer to allow pluggable authorizer implementation. Key: KAFKA-2210

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

2015-05-20 Thread Jiangjie Qin
Hey Ewen, I agree with you that we should avoid any unnecessary configurations exposed to user. And the necessity is defined by use case. I also agree that the configurations should be named from users’ perspective and comply with the intuition - for example, like what Joel said, something like

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

2015-05-20 Thread Jiangjie Qin
I actually feel many [VOTE] threads eventually become [DISCUSS] as people just put tons of comments there :) On 5/20/15, 11:52 AM, Jay Kreps jay.kr...@gmail.com wrote: Makes sense. Honghai, want to do a [VOTE] thread just so everything is official? -Jay On Wed, May 20, 2015 at 11:22 AM, Gwen

[jira] [Commented] (KAFKA-2211) KafkaAuthorizer: Add simpleACLAuthorizer implementation.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2211?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553012#comment-14553012 ] Parth Brahmbhatt commented on KAFKA-2211: - Created reviewboard

[jira] [Updated] (KAFKA-2211) KafkaAuthorizer: Add simpleACLAuthorizer implementation.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-2211: Status: Patch Available (was: Open) KafkaAuthorizer: Add simpleACLAuthorizer

[jira] [Updated] (KAFKA-2211) KafkaAuthorizer: Add simpleACLAuthorizer implementation.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2211?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-2211: Attachment: KAFKA-2211.patch KafkaAuthorizer: Add simpleACLAuthorizer implementation.

[jira] [Commented] (KAFKA-2212) KafkaAuthorizer: Add CLI for Acl management.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553014#comment-14553014 ] Parth Brahmbhatt commented on KAFKA-2212: - Created reviewboard

Review Request 34494: Patch for KAFKA-2212

2015-05-20 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34494/ --- Review request for kafka. Bugs: KAFKA-2212

[jira] [Updated] (KAFKA-2212) KafkaAuthorizer: Add CLI for Acl management.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-2212: Attachment: KAFKA-2212.patch KafkaAuthorizer: Add CLI for Acl management.

[jira] [Updated] (KAFKA-2212) KafkaAuthorizer: Add CLI for Acl management.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt updated KAFKA-2212: Status: Patch Available (was: Open) KafkaAuthorizer: Add CLI for Acl management.

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

2015-05-20 Thread Parth Brahmbhatt
This vote is now Closed with 4 binding +1s and 4 non binding +1s. Thanks Parth On 5/20/15, 12:04 PM, Joel Koshy jjkosh...@gmail.com wrote: +1 On Fri, May 15, 2015 at 04:18:49PM +, Parth Brahmbhatt wrote: Hi, Opening the voting thread for KIP-11. Link to the KIP:

[jira] [Reopened] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reopened KAFKA-2187: I was just testing the merge script and it seems to work great. :) Introduce merge-kafka-pr.py script

[jira] [Created] (KAFKA-2213) Log cleaner should write compacted messages using configured compression type

2015-05-20 Thread Joel Koshy (JIRA)
Joel Koshy created KAFKA-2213: - Summary: Log cleaner should write compacted messages using configured compression type Key: KAFKA-2213 URL: https://issues.apache.org/jira/browse/KAFKA-2213 Project: Kafka

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-20 Thread Sriharsha Chintalapani
On May 15, 2015, 8:26 p.m., Michael Herstine wrote: clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java, lines 137-139 https://reviews.apache.org/r/33620/diff/5/?file=957064#file957064line137 This is interesting; I don't see a corresponding

[jira] [Assigned] (KAFKA-2168) New consumer poll() can block other calls like position(), commit(), and close() indefinitely

2015-05-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2168?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-2168: -- Assignee: Jason Gustafson (was: Ewen Cheslack-Postava) New consumer poll() can block

Review Request 34492: Patch for KAFKA-2210

2015-05-20 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34492/ --- Review request for kafka. Bugs: KAFKA-2210

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

2015-05-20 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.patch KafkaAuthorizer: Add all public entities, config changes and

Review Request 34493: Patch for KAFKA-2211

2015-05-20 Thread Parth Brahmbhatt
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34493/ --- Review request for kafka. Bugs: KAFKA-2211

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

2015-05-20 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553010#comment-14553010 ] Parth Brahmbhatt commented on KAFKA-2210: - Created reviewboard

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

2015-05-20 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: Status: Patch Available (was: Open) KafkaAuthorizer: Add all public entities, config

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-20 Thread Sriharsha Chintalapani
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33620/ --- (Updated May 20, 2015, 9:54 p.m.) Review request for kafka. Bugs: KAFKA-1690

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

2015-05-20 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-1690: -- Attachment: KAFKA-1690_2015-05-20_14:54:35.patch new java producer needs ssl

[jira] [Commented] (KAFKA-1335) Add rebalancing logic to the coordinator / consumer

2015-05-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553213#comment-14553213 ] Jason Gustafson commented on KAFKA-1335: Hey [~onurkaraman], I'm looking at this

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

2015-05-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1646?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-1646: - Reviewer: Jun Rao Improve consumer read performance for Windows

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

2015-05-20 Thread Gwen Shapira
For simple discussions, I completely agree. For those threads where there are few votes, and then more discussion, and then KIP changes few times... separate thread will help keep things clear for both voters and anyone who will try to figure out what happened in the future. On Wed, May 20, 2015

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

2015-05-20 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552918#comment-14552918 ] Jun Rao commented on KAFKA-2147: [~csallsop...@gmail.com], we probably won't patch 0.8.1.1

[jira] [Created] (KAFKA-2211) KafkaAuthorizer: Add simpleACLAuthorizer implementation.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
Parth Brahmbhatt created KAFKA-2211: --- Summary: KafkaAuthorizer: Add simpleACLAuthorizer implementation. Key: KAFKA-2211 URL: https://issues.apache.org/jira/browse/KAFKA-2211 Project: Kafka

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

2015-05-20 Thread Joel Koshy
+1 On Fri, May 15, 2015 at 04:18:49PM +, Parth Brahmbhatt wrote: Hi, Opening the voting thread for KIP-11. Link to the KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 Thanks

Re: Review Request 34450: Fix KAFKA-2017; rebased

2015-05-20 Thread Guozhang Wang
On May 20, 2015, 5:15 p.m., Onur Karaman wrote: I only did a brief skim. This optimization tries to switch consumers over to a new coordinator without a rebalance. From my understanding, the consumers would detect a coordinator failure, discover the new coordinator to work with, and

Build failed in Jenkins: KafkaPreCommit #105

2015-05-20 Thread Apache Jenkins Server
See https://builds.apache.org/job/KafkaPreCommit/105/changes Changes: [wangguoz] KAFKA-1374; Log cleaner should be able to handle compressed messages; reviewed by Guozhang Wang -- [...truncated 733 lines...] kafka.zk.ZKEphemeralTest

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

2015-05-20 Thread Joel Koshy
I think in general it is fine (even good) if a VOTE thread has a lot of discussion. The only issue I can think of is the one that Gwen made reference to: early votes - update KIP/whatever is being voted on due to more discussion - later votes as it then becomes unclear on what exactly each

Re: Review Request 34418: Patch for KAFKA-2191

2015-05-20 Thread Dong Lin
On May 19, 2015, 5:41 p.m., Dong Lin wrote: clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 64 https://reviews.apache.org/r/34418/diff/1/?file=963982#file963982line64 Hey Jay, My approach differs with your only in this line:

Re: Review Request 33065: Patch for KAFKA-1928

2015-05-20 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/#review84536 --- Thanks for the latest patch. Just two new minor comments and a

[jira] [Commented] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-05-20 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14552840#comment-14552840 ] Joel Koshy commented on KAFKA-1374: --- I just wanted the commit to record Manikumar as

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

2015-05-20 Thread Jay Kreps
Makes sense. Honghai, want to do a [VOTE] thread just so everything is official? -Jay On Wed, May 20, 2015 at 11:22 AM, Gwen Shapira gshap...@cloudera.com wrote: For simple discussions, I completely agree. For those threads where there are few votes, and then more discussion, and then KIP

[jira] [Created] (KAFKA-2212) KafkaAuthorizer: Add CLI for Acl management.

2015-05-20 Thread Parth Brahmbhatt (JIRA)
Parth Brahmbhatt created KAFKA-2212: --- Summary: KafkaAuthorizer: Add CLI for Acl management. Key: KAFKA-2212 URL: https://issues.apache.org/jira/browse/KAFKA-2212 Project: Kafka Issue

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

2015-05-20 Thread Jay Kreps
Hey all, How do people feel about these [DISCUSS] threads that basically accidentally turn into votes. Like basically everyone was +1 on this KIP already should we just skip the second vote? I find it kind of annoying to do both when this happens. -Jay On Mon, May 11, 2015 at 8:16 PM, Honghai

[jira] [Resolved] (KAFKA-1374) LogCleaner (compaction) does not support compressed topics

2015-05-20 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1374?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy resolved KAFKA-1374. --- Resolution: Fixed Please file a separate jira for that. Also, I'm going to amend this commit with

[jira] [Updated] (KAFKA-269) ./system_test/producer_perf/bin/run-test.sh without --async flag does not run

2015-05-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-269?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jay Kreps updated KAFKA-269: Reviewer: (was: Jay Kreps) ./system_test/producer_perf/bin/run-test.sh without --async flag does not run

[jira] [Commented] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553237#comment-14553237 ] Ismael Juma commented on KAFKA-2187: Updated reviewboard

Request for permission to edit pages in the wiki

2015-05-20 Thread Ismael Juma
Hi, In order to edit the Patch submission and review page with information on how to merge GitHub pull requests, it would be helpful to have edit permission for Kafka's Confluence pages. My Confluence account id is `ijuma`[1]. Thanks, Ismael [1]

[jira] [Commented] (KAFKA-2168) New consumer poll() can block other calls like position(), commit(), and close() indefinitely

2015-05-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553332#comment-14553332 ] Ewen Cheslack-Postava commented on KAFKA-2168: -- For option 1 it's probably

Re: [VOTE] KIP-21 Dynamic Configuration

2015-05-20 Thread Joel Koshy
+1 On Wed, May 20, 2015 at 05:33:31AM +, Aditya Auradkar wrote: Thanks Andrii. I'll make the changes. I've also updated KIP-21 to include the new config requests. Take a look and vote. https://cwiki.apache.org/confluence/display/KAFKA/KIP-21+-+Dynamic+Configuration Aditya

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

2015-05-20 Thread Craig Allsop (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553394#comment-14553394 ] Craig Allsop commented on KAFKA-2147: - re: version, I understand. re: interval, I

[jira] [Commented] (KAFKA-1778) Create new re-elect controller admin function

2015-05-20 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1778?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553429#comment-14553429 ] Joel Koshy commented on KAFKA-1778: --- I may be missing some detail, but (a), (b), (c)

[jira] [Resolved] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-2187. Resolution: Fixed Fix Version/s: 0.8.3 Issue resolved by pull request 3

[jira] [Commented] (KAFKA-1335) Add rebalancing logic to the coordinator / consumer

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553322#comment-14553322 ] Guozhang Wang commented on KAFKA-1335: -- Hi [~hachikuji], I have already started

[jira] [Commented] (KAFKA-2168) New consumer poll() can block other calls like position(), commit(), and close() indefinitely

2015-05-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2168?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553343#comment-14553343 ] Ewen Cheslack-Postava commented on KAFKA-2168: -- Actually, now I realize

Review Request 34502: Patch for KAFKA-2187

2015-05-20 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34502/ --- Review request for kafka. Bugs: KAFKA-2187

[jira] [Updated] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2187: --- Attachment: KAFKA-2187.patch Introduce merge-kafka-pr.py script --

[jira] [Updated] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2187: --- Status: Patch Available (was: Reopened) Introduce merge-kafka-pr.py script

[jira] [Commented] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553226#comment-14553226 ] Ismael Juma commented on KAFKA-2187: Created reviewboard

Re: Request for permission to edit pages in the wiki

2015-05-20 Thread Joel Koshy
Done On Wed, May 20, 2015 at 11:45:48PM +0100, Ismael Juma wrote: Hi, In order to edit the Patch submission and review page with information on how to merge GitHub pull requests, it would be helpful to have edit permission for Kafka's Confluence pages. My Confluence account id is

[jira] [Commented] (KAFKA-1335) Add rebalancing logic to the coordinator / consumer

2015-05-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553418#comment-14553418 ] Jason Gustafson commented on KAFKA-1335: Hey [~guozhang], sounds fine to me. Is

[jira] [Commented] (KAFKA-1479) Logs filling up while Kafka ReplicaFetcherThread tries to retrieve partition info for deleted topics

2015-05-20 Thread Greg Lloyd (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1479?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553460#comment-14553460 ] Greg Lloyd commented on KAFKA-1479: --- I am not able to locate the directories that Manasi

[jira] [Updated] (KAFKA-2187) Introduce merge-kafka-pr.py script

2015-05-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2187?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2187: --- Fix Version/s: (was: 0.8.3) Introduce merge-kafka-pr.py script

[jira] [Issue Comment Deleted] (KAFKA-1737) Document required ZkSerializer for ZkClient used with AdminUtils

2015-05-20 Thread Vivek Madani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1737?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vivek Madani updated KAFKA-1737: Comment: was deleted (was: Updated the patch. Removed the imports and made ZkStringSerializer

[jira] [Updated] (KAFKA-1737) Document required ZkSerializer for ZkClient used with AdminUtils

2015-05-20 Thread Vivek Madani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1737?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vivek Madani updated KAFKA-1737: Status: Patch Available (was: In Progress) Submitting the patch with a wrapper around ZkClient to

Re: Review Request 34502: Patch for KAFKA-2187

2015-05-20 Thread Ismael Juma
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34502/ --- (Updated May 20, 2015, 10:14 p.m.) Review request for kafka. Bugs:

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

2015-05-20 Thread Craig Allsop (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2147?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Craig Allsop updated KAFKA-2147: Attachment: craig-kafka-purgatory-queue-size-issue.png Unbalanced replication can cause extreme

Re: Review Request 34450: Fix KAFKA-2017; rebased

2015-05-20 Thread Jun Rao
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34450/#review84604 --- Thanks for the patch. A few comments below.

[jira] [Commented] (KAFKA-1335) Add rebalancing logic to the coordinator / consumer

2015-05-20 Thread Onur Karaman (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1335?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553458#comment-14553458 ] Onur Karaman commented on KAFKA-1335: - I'll write up the ticket with the details

[jira] [Comment Edited] (KAFKA-1479) Logs filling up while Kafka ReplicaFetcherThread tries to retrieve partition info for deleted topics

2015-05-20 Thread Greg Lloyd (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1479?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553460#comment-14553460 ] Greg Lloyd edited comment on KAFKA-1479 at 5/21/15 2:05 AM: I

Re: Review Request 33065: Patch for KAFKA-1928

2015-05-20 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/33065/#review84649 --- core/src/main/scala/kafka/network/RequestChannel.scala

Review Request 34524: Fix KAFKA-2208

2015-05-20 Thread Guozhang Wang
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34524/ --- Review request for kafka. Bugs: KAFKA-2208

[jira] [Commented] (KAFKA-2208) Consumer should detect coordinator failures

2015-05-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2208?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14553520#comment-14553520 ] Guozhang Wang commented on KAFKA-2208: -- Created reviewboard

  1   2   >