[jira] [Comment Edited] (KAFKA-369) remove ZK dependency on producer

2015-10-21 Thread Jiahongchao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14589225#comment-14589225 ] Jiahongchao edited comment on KAFKA-369 at 10/21/15 9:43 AM: - S

[GitHub] kafka pull request: KAFKA-2658: Add PLAIN mechanism to SASL implem...

2015-10-21 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/341 KAFKA-2658: Add PLAIN mechanism to SASL implementation Implementation and unit tests for SASL/PLAIN. A simple login module and SaslServer implementation for PLAIN mechanism are included in the

[jira] [Commented] (KAFKA-2658) Implement SASL/PLAIN

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14966652#comment-14966652 ] ASF GitHub Bot commented on KAFKA-2658: --- GitHub user rajinisivaram opened a pull req

[jira] [Updated] (KAFKA-2658) Implement SASL/PLAIN

2015-10-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2658?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram updated KAFKA-2658: -- Status: Patch Available (was: Open) PR adds a new option to specify SASL mechanism and adds sup

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1499#comment-1499 ] Rajini Sivaram commented on KAFKA-2644: --- [~ijuma] Do you have any ideas on the simpl

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14966749#comment-14966749 ] Ismael Juma commented on KAFKA-2644: Good question [~rsivaram]. I'm not sure, [~harsha

[jira] [Assigned] (KAFKA-2456) Disable SSLv3 for ssl.enabledprotocols config on client & broker side

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-2456: -- Assignee: Ismael Juma (was: Sriharsha Chintalapani) > Disable SSLv3 for ssl.enabledprotocols c

[jira] [Work started] (KAFKA-2456) Disable SSLv3 for ssl.enabledprotocols config on client & broker side

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2456 started by Ismael Juma. -- > Disable SSLv3 for ssl.enabledprotocols config on client & broker side > ---

[jira] [Created] (KAFKA-2680) Zookeeper SASL check prevents any SASL code being run with IBM JDK

2015-10-21 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-2680: - Summary: Zookeeper SASL check prevents any SASL code being run with IBM JDK Key: KAFKA-2680 URL: https://issues.apache.org/jira/browse/KAFKA-2680 Project: Kafka

[jira] [Commented] (KAFKA-2456) Disable SSLv3 for ssl.enabledprotocols config on client & broker side

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2456?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14966882#comment-14966882 ] ASF GitHub Bot commented on KAFKA-2456: --- GitHub user ijuma opened a pull request:

[jira] [Updated] (KAFKA-2456) Disable SSLv3 for ssl.enabledprotocols config on client & broker side

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2456: --- Reviewer: Jun Rao Status: Patch Available (was: In Progress) > Disable SSLv3 for ssl.enabledpro

[jira] [Updated] (KAFKA-2472) Fix kafka ssl configs to not throw warnings

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2472?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2472: --- Reviewer: Jun Rao Status: Patch Available (was: In Progress) Pull request: https://github.com/

[GitHub] kafka pull request: KAFKA-2456 KAFKA-2472; SSL clean-ups

2015-10-21 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/342 KAFKA-2456 KAFKA-2472; SSL clean-ups You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-2472-fix-kafka-ssl-config-warnings Alterna

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14966892#comment-14966892 ] Sriharsha Chintalapani commented on KAFKA-2644: --- [~rsivaram] we already have

[VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Flavio Junqueira
Thanks everyone for the feedback so far. At this point, I'd like to start a vote for KIP-38. Summary: Add support for ZooKeeper authentication KIP page: https://cwiki.apache.org/confluence/display/KAFKA/KIP-38%3A+ZooKeeper+Authentication

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Onur Karaman
+1 On Wed, Oct 21, 2015 at 8:17 AM, Flavio Junqueira wrote: > Thanks everyone for the feedback so far. At this point, I'd like to start > a vote for KIP-38. > > Summary: Add support for ZooKeeper authentication > KIP page: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-38%3A+ZooKeeper+A

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967298#comment-14967298 ] Rajini Sivaram commented on KAFKA-2644: --- [~ijuma] [~harsha_ch] Thank you both for y

Java high level consumer providing duplicate messages when auto commit is off

2015-10-21 Thread Cliff Rhyne
Hi, My team and I are looking into a problem where the Java high level consumer provides duplicate messages if we turn auto commit off (using version 0.8.2.1 of the server and Java client). The expected sequence of events are: 1. Start high-level consumer and initialize a KafkaStream to get a Co

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Grant Henke
+1 Is it worth mentioning the follow up steps that were discussed in the KIP call in this KIP document? Some of them were: - Adding SSL support for Zookeeper - Removing the "world readable" assumption Thank you, Grant On Wed, Oct 21, 2015 at 10:23 AM, Onur Karaman < okara...@linkedin.com.

[jira] [Commented] (KAFKA-2658) Implement SASL/PLAIN

2015-10-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967340#comment-14967340 ] Rajini Sivaram commented on KAFKA-2658: --- [~ijuma] [~harsha_ch] [~junrao] Do you have

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Flavio Junqueira
Bringing the points Grant brought to this thread: > Is it worth mentioning the follow up steps that were discussed in the KIP > call in this KIP document? Some of them were: > > - Adding SSL support for Zookeeper > - Removing the "world readable" assumption > Grant, how would you do it? I s

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Ismael Juma
On Wed, Oct 21, 2015 at 5:04 PM, Flavio Junqueira wrote: > Bringing the points Grant brought to this thread: > > > Is it worth mentioning the follow up steps that were discussed in the KIP > > call in this KIP document? Some of them were: > > > > - Adding SSL support for Zookeeper > > - Remov

Re: Java high level consumer providing duplicate messages when auto commit is off

2015-10-21 Thread James Cheng
Do you have multiple consumers in a consumer group? I think that when a new consumer joins the consumer group, that the existing consumers will stop consuming during the group rebalance, and then when they start consuming again, that they will consume from the last committed offset. You should

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Todd Palino
While this is a great idea, is it really ready for vote? I don't see any detail in the wiki about what trees will be secured, and whether or not that is configurable. I also don't see anything about how the use of admin tools is going to be addressed. -Todd On Wed, Oct 21, 2015 at 8:48 AM, Grant

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967415#comment-14967415 ] Ewen Cheslack-Postava commented on KAFKA-2644: -- [~rsivaram] To be honest, I d

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Flavio Junqueira
Todd, There is a discuss thread for this KIP and we talked about it during the KIP call yesterday. I'm more than happy to add detail to the KIP if you bring it up in the discuss thread. I'd rather not stop the vote thread, though. Thanks, -Flavio > On 21 Oct 2015, at 17:41, Todd Palino wrote:

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Todd Palino
There seems to be a bit of detail lacking in the KIP. Specifically, I'd like to understand: 1) What znodes are the brokers going to secure? Is this configurable? How? 2) What ACL is the broker going to apply? Is this configurable? 3) How will the admin tools (such as preferred replica election and

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Todd Palino
I've added a reply on the discuss thread already. However, the point is that if there were changes as a result of the KIP call (which I often can't make on Tuesdays), it should be updated on the wiki page so everyone is aware of what is being voted on. -Todd On Wed, Oct 21, 2015 at 9:47 AM, Flav

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Flavio Junqueira
> On 21 Oct 2015, at 17:47, Todd Palino wrote: > > There seems to be a bit of detail lacking in the KIP. Specifically, I'd > like to understand: > > 1) What znodes are the brokers going to secure? Is this configurable? How? Currently it is securing all paths here except the consumers one: htt

[GitHub] kafka pull request: MINOR: Update to Gradle 2.8

2015-10-21 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/343 MINOR: Update to Gradle 2.8 There have been a number of improvements between the version we are currently using (2.4) and the current version (2.8): https://gradle.org/docs/2.5/release-notes

[GitHub] kafka pull request: KAFKA-2459: connection backoff, timeouts and r...

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/290 --- 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 enabl

[jira] [Updated] (KAFKA-2459) Connection backoff/blackout period should start when a connection is disconnected, not when the connection attempt was initiated

2015-10-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2459: - Resolution: Fixed Fix Version/s: 0.9.0.0 Status: Resolved (was: Patch Available)

[jira] [Commented] (KAFKA-2459) Connection backoff/blackout period should start when a connection is disconnected, not when the connection attempt was initiated

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967459#comment-14967459 ] ASF GitHub Bot commented on KAFKA-2459: --- Github user asfgit closed the pull request

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Jay Kreps
+1 -Jay On Wed, Oct 21, 2015 at 8:17 AM, Flavio Junqueira wrote: > Thanks everyone for the feedback so far. At this point, I'd like to start > a vote for KIP-38. > > Summary: Add support for ZooKeeper authentication > KIP page: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-38%3A+ZooKe

[jira] [Updated] (KAFKA-1687) SASL unit tests

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1687?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1687: --- Summary: SASL unit tests (was: SASL tests) > SASL unit tests > --- > > Ke

[jira] [Updated] (KAFKA-1687) SASL unit tests

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1687?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-1687: --- Assignee: sriharsha chintalapani (was: Gwen Shapira) > SASL unit tests > --- > >

[jira] [Resolved] (KAFKA-1687) SASL unit tests

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1687?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-1687. Resolution: Fixed This was done as part of KAFKA-1686. > SASL unit tests > --- > >

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Jay Kreps
Yeah let's definitely get a complete description of the user-facing impact, especially the changes to the command-line tools. As much as anything the purpose of these KIPs is to fully capture what the user's life will be like in the next release. Also, if I understand correctly we aren't securing

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Parth Brahmbhatt
I have 2 suggestions: 1) We need to document how does one move from secure to non secure environment: 1) change the config on all brokers to zookeeper.set.acl = false and do a rolling upgrade. 2) Run the migration script with the jass config file so it is sasl authenticated with

Re: Java high level consumer providing duplicate messages when auto commit is off

2015-10-21 Thread Cliff Rhyne
Hi James, There are two scenarios we run: 1. Multiple partitions with one consumer per partition. This rarely has starting/stopping of consumers, so the pool is very static. There is a configured consumer timeout, which is causing the ConsumerTimeoutException to get thrown prior to the test sta

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Todd Palino
Comments inline. In addition, the documentation on the migration path is good, but do we really need a separate utility? Would it be better to have checking and setting the ACLs be a function of the controller, possibly as a separate thread either only at controller startup or periodically, with i

[jira] [Commented] (KAFKA-1695) Authenticate connection to Zookeeper

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967487#comment-14967487 ] Ismael Juma commented on KAFKA-1695: This ticket has been broken down into KAFKA-2639,

[jira] [Created] (KAFKA-2681) SASL authentication in official docs

2015-10-21 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2681: -- Summary: SASL authentication in official docs Key: KAFKA-2681 URL: https://issues.apache.org/jira/browse/KAFKA-2681 Project: Kafka Issue Type: Sub-task

[jira] [Created] (KAFKA-2682) Authorization section in official docs

2015-10-21 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2682: -- Summary: Authorization section in official docs Key: KAFKA-2682 URL: https://issues.apache.org/jira/browse/KAFKA-2682 Project: Kafka Issue Type: Sub-task

[jira] [Commented] (KAFKA-2681) SASL authentication in official docs

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967493#comment-14967493 ] Ismael Juma commented on KAFKA-2681: cc [~harsha_ch] > SASL authentication in officia

[jira] [Commented] (KAFKA-2682) Authorization section in official docs

2015-10-21 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2682?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967491#comment-14967491 ] Ismael Juma commented on KAFKA-2682: cc [~parth.brahmbhatt] > Authorization section i

[jira] [Assigned] (KAFKA-2682) Authorization section in official docs

2015-10-21 Thread Parth Brahmbhatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2682?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Parth Brahmbhatt reassigned KAFKA-2682: --- Assignee: Parth Brahmbhatt > Authorization section in official docs > ---

[jira] [Assigned] (KAFKA-2681) SASL authentication in official docs

2015-10-21 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2681?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani reassigned KAFKA-2681: - Assignee: Sriharsha Chintalapani > SASL authentication in official docs >

Question about compressed messages...

2015-10-21 Thread Robert Thille
I’m working on a Twisted Python Kafka client and I was wondering what the ‘key’ on a gzip’d block of messages “means”. That is, if the client has a batch of messages to send, with a mix of keys, would it be a bug to batch them together and gzip into a single message? Or is the key on the ou

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Geoff Anderson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967527#comment-14967527 ] Geoff Anderson commented on KAFKA-2644: --- [~rsivaram] Keep in mind that service class

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Flavio Junqueira
> On 21 Oct 2015, at 18:07, Parth Brahmbhatt > wrote: > > I have 2 suggestions: > > 1) We need to document how does one move from secure to non secure > environment: > 1) change the config on all brokers to zookeeper.set.acl = false and do > a > rolling upgrade. > 2) Run the migr

[GitHub] kafka pull request: KAFKA-2365: Handle null keys and value validat...

2015-10-21 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/344 KAFKA-2365: Handle null keys and value validation properly in OffsetStorageWriter. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka

[jira] [Commented] (KAFKA-2365) Copycat checklist

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967537#comment-14967537 ] ASF GitHub Bot commented on KAFKA-2365: --- GitHub user ewencp opened a pull request:

[GitHub] kafka pull request: KAFKA-2626: Handle null keys and value validat...

2015-10-21 Thread ewencp
Github user ewencp closed the pull request at: https://github.com/apache/kafka/pull/344 --- 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 enabl

[jira] [Commented] (KAFKA-2626) Null offsets in copycat causes exception in OffsetStorageWriter

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2626?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967543#comment-14967543 ] ASF GitHub Bot commented on KAFKA-2626: --- Github user ewencp closed the pull request

[GitHub] kafka pull request: KAFKA-2626: Handle null keys and value validat...

2015-10-21 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/345 KAFKA-2626: Handle null keys and value validation properly in OffsetStorageWriter. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ewencp/kafka

[jira] [Commented] (KAFKA-2626) Null offsets in copycat causes exception in OffsetStorageWriter

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2626?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967544#comment-14967544 ] ASF GitHub Bot commented on KAFKA-2626: --- GitHub user ewencp opened a pull request:

[jira] [Updated] (KAFKA-2626) Null offsets in copycat causes exception in OffsetStorageWriter

2015-10-21 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2626?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2626: - Reviewer: Guozhang Wang Status: Patch Available (was: Open) > Null offsets

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967560#comment-14967560 ] Rajini Sivaram commented on KAFKA-2644: --- [~ewencp] [~geoffra] Thank you both for you

[jira] [Commented] (KAFKA-2644) Run relevant ducktape tests with SASL_PLAINTEXT and SASL_SSL

2015-10-21 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2644?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967569#comment-14967569 ] Sriharsha Chintalapani commented on KAFKA-2644: --- [~rsivaram] as long as you

[GitHub] kafka pull request: KAFKA-2667: Fix transient error in KafkaBasedL...

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/333 --- 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 enabl

[jira] [Updated] (KAFKA-2667) Copycat KafkaBasedLogTest.testSendAndReadToEnd transient failure

2015-10-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2667: - Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request

[jira] [Commented] (KAFKA-2667) Copycat KafkaBasedLogTest.testSendAndReadToEnd transient failure

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967584#comment-14967584 ] ASF GitHub Bot commented on KAFKA-2667: --- Github user asfgit closed the pull request

[GitHub] kafka pull request: MINOR: Update to Gradle 2.8

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/343 --- 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 enabl

Build failed in Jenkins: kafka-trunk-jdk8 #45

2015-10-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2459: Mark last committed timestamp to fix connection backoff -- [...truncated 4611 lines...] org.apache.kafka.copycat.file.FileStreamSourceConnectorTest > testSourc

[jira] [Commented] (KAFKA-2674) ConsumerRebalanceListener.onPartitionsRevoked() is not called on consumer close

2015-10-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967638#comment-14967638 ] Guozhang Wang commented on KAFKA-2674: -- This is a good point, especially if the user

[GitHub] kafka pull request: KAFKA-2464: client-side assignment for new con...

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/165 --- 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 enabl

[jira] [Resolved] (KAFKA-2464) Client-side assignment and group generalization

2015-10-21 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2464. -- Resolution: Fixed Issue resolved by pull request 165 [https://github.com/apache/kafka/pull/165]

[jira] [Commented] (KAFKA-2464) Client-side assignment and group generalization

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2464?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967682#comment-14967682 ] ASF GitHub Bot commented on KAFKA-2464: --- Github user asfgit closed the pull request

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-21 Thread Ashish Singh
In last KIP hangout following questions were raised. 1. *Whether or not to support move command? If yes, how do we support it.* I think *move* command will be essential, once we start supporting directories. However, implementation might be a bit convoluted. A few things required f

Re: Java high level consumer providing duplicate messages when auto commit is off

2015-10-21 Thread Kris K
Hi Cliff, One other case I observed in my environment is - when there were gc pauses on one of our high level consumer in the group. Thanks, Kris On Wed, Oct 21, 2015 at 10:12 AM, Cliff Rhyne wrote: > Hi James, > > There are two scenarios we run: > > 1. Multiple partitions with one consumer pe

[jira] [Commented] (KAFKA-1843) Metadata fetch/refresh in new producer should handle all node connection states gracefully

2015-10-21 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967773#comment-14967773 ] Eno Thereska commented on KAFKA-1843: - https://github.com/apache/kafka/pull/290 addres

Build failed in Jenkins: kafka-trunk-jdk8 #46

2015-10-21 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2667: Fix transient error in KafkaBasedLogTest. [wangguoz] MINOR: Update to Gradle 2.8 [wangguoz] KAFKA-2464: client-side assignment for new consumer -- [...truncat

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Jun Rao
Parth, For 2), in your approach, the broker/controller will then always have the overhead of resetting the ACL on startup after zookeeper.set.acl is set to true. The benefit of using a separate migration tool is that you paid the cost only once during upgrade. It is an extra step during the upgrad

[GitHub] kafka pull request: KAFKA-2454: Deadlock between log segment delet...

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/153 --- 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 enabl

[jira] [Commented] (KAFKA-2454) Dead lock between delete log segment and shutting down.

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967809#comment-14967809 ] ASF GitHub Bot commented on KAFKA-2454: --- Github user asfgit closed the pull request

[jira] [Updated] (KAFKA-2454) Dead lock between delete log segment and shutting down.

2015-10-21 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy updated KAFKA-2454: -- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request 153 [h

[jira] [Commented] (KAFKA-369) remove ZK dependency on producer

2015-10-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-369?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967853#comment-14967853 ] Jun Rao commented on KAFKA-369: --- We are adding a new java consumer (org.apache.kafka.clients.

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Todd Palino
Thanks for the clarification on that, Jun. Obviously, we haven't been doing much with ZK authentication around here yet. There is still a small concern there, mostly in that you should not share credentials any more than is necessary, which would argue for being able to use a different ACL than the

[GitHub] kafka pull request: KAFKA-2678; partition level lag metrics can be...

2015-10-21 Thread lindong28
GitHub user lindong28 opened a pull request: https://github.com/apache/kafka/pull/346 KAFKA-2678; partition level lag metrics can be negative You can merge this pull request into a Git repository by running: $ git pull https://github.com/lindong28/kafka KAFKA-2678 Alternative

[jira] [Commented] (KAFKA-2678) partition level lag metrics can be negative

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2678?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967925#comment-14967925 ] ASF GitHub Bot commented on KAFKA-2678: --- GitHub user lindong28 opened a pull request

Re: Java high level consumer providing duplicate messages when auto commit is off

2015-10-21 Thread Cliff Rhyne
Hi Kris, Thanks for the tip. I'm going to investigate this further. I checked and we have fairly short zk timeouts and run with a smaller memory allocation on the two environments we encounter this issue. I'll let you all know what I find. I saw this ticket https://issues.apache.org/jira/brows

[GitHub] kafka pull request: MINOR: fix checkstyle failures

2015-10-21 Thread SinghAsDev
GitHub user SinghAsDev opened a pull request: https://github.com/apache/kafka/pull/347 MINOR: fix checkstyle failures @guozhangwang could you take a look at this. These failures are a bit annoying as it never leads to a successful build. You can merge this pull request into a Git r

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-21 Thread Jay Kreps
Gwen, It's a good question of what the producer semantics are--would we only allow you to produce to a partition or first level directory or would we hash over whatever subtree you supply? Actually not sure which makes more sense... Ashish, here are some thoughts: 1. I think we can do this online.

[jira] [Commented] (KAFKA-1690) Add SSL support to Kafka Broker, Producer and Consumer

2015-10-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1690?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967968#comment-14967968 ] Jun Rao commented on KAFKA-1690: [~sriharsha], I am wondering why we need a ssl.protocol a

[jira] [Commented] (KAFKA-2671) Enable starting Kafka server with a Properties object

2015-10-21 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14967999#comment-14967999 ] Ashish K Singh commented on KAFKA-2671: --- [~gwenshap], [~guozhang] I have updated the

[GitHub] kafka pull request: KAFKA-2618; Disable SSL renegotiation for 0.9....

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/339 --- 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 enabl

[GitHub] kafka pull request: MINOR: Clean-up MemoryRecords variables and AP...

2015-10-21 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/348 MINOR: Clean-up MemoryRecords variables and APIs You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka MemoryRecordsCapacity A

[jira] [Updated] (KAFKA-2618) Disable SSL renegotiation for 0.9.0.0

2015-10-21 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2618?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2618: --- Resolution: Fixed Status: Resolved (was: Patch Available) Issue resolved by pull request 339 [https:/

[jira] [Commented] (KAFKA-2618) Disable SSL renegotiation for 0.9.0.0

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14968006#comment-14968006 ] ASF GitHub Bot commented on KAFKA-2618: --- Github user asfgit closed the pull request

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Dong Lin
+1 On Wed, Oct 21, 2015 at 5:01 PM, Jay Kreps wrote: > +1 > > -Jay > > On Wed, Oct 21, 2015 at 8:17 AM, Flavio Junqueira wrote: > > > Thanks everyone for the feedback so far. At this point, I'd like to start > > a vote for KIP-38. > > > > Summary: Add support for ZooKeeper authentication > > KI

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Flavio Junqueira
> On 21 Oct 2015, at 21:54, Todd Palino wrote: > > Thanks for the clarification on that, Jun. Obviously, we haven't been doing > much with ZK authentication around here yet. There is still a small concern > there, mostly in that you should not share credentials any more than is > necessary, whic

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Joel Koshy
+1 binding On Wed, Oct 21, 2015 at 8:17 AM, Flavio Junqueira wrote: > Thanks everyone for the feedback so far. At this point, I'd like to start > a vote for KIP-38. > > Summary: Add support for ZooKeeper authentication > KIP page: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-38%3A+Zoo

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Todd Palino
On Wed, Oct 21, 2015 at 3:38 PM, Flavio Junqueira wrote: > > > On 21 Oct 2015, at 21:54, Todd Palino wrote: > > > > Thanks for the clarification on that, Jun. Obviously, we haven't been > doing > > much with ZK authentication around here yet. There is still a small > concern > > there, mostly in

Build failed in Jenkins: kafka-trunk-jdk8 #47

2015-10-21 Thread Apache Jenkins Server
See Changes: [jjkoshy] KAFKA-2454; Deadlock between log segment deletion and server shutdown. [junrao] KAFKA-2618; Disable SSL renegotiation for 0.9.0.0 -- [...truncated 4996 lines...] kafka.coo

Re: [DISCUSS] KIP-37 - Add namespaces in Kafka

2015-10-21 Thread Ashish Singh
On Wed, Oct 21, 2015 at 2:22 PM, Jay Kreps wrote: > Gwen, It's a good question of what the producer semantics are--would we > only allow you to produce to a partition or first level directory or would > we hash over whatever subtree you supply? Actually not sure which makes > more sense... > > As

Re: [VOTE] KIP-38: ZooKeeper authentication

2015-10-21 Thread Ismael Juma
+1 (non-binding) On Wed, Oct 21, 2015 at 4:17 PM, Flavio Junqueira wrote: > Thanks everyone for the feedback so far. At this point, I'd like to start > a vote for KIP-38. > > Summary: Add support for ZooKeeper authentication > KIP page: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-38%

Re: [DISCUSS] KIP-38: ZooKeeper Authentication

2015-10-21 Thread Flavio Junqueira
Ok, thanks for the feedback, Todd. I have updated the KIP with some of the points discussed here. There is more to add based on these last comments, though. -Flavio > On 21 Oct 2015, at 23:43, Todd Palino wrote: > > On Wed, Oct 21, 2015 at 3:38 PM, Flavio Junqueira >

[GitHub] kafka pull request: KAFKA-2209 - Change quotas dynamically using D...

2015-10-21 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/298 --- 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 enabl

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

2015-10-21 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2209?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14968139#comment-14968139 ] ASF GitHub Bot commented on KAFKA-2209: --- Github user asfgit closed the pull request

  1   2   >