Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-10-20 Thread Kartik Paramasivam
Joel or Becket will probably respond back in more detail.. but here are my 2c. >From the standpoint of LinkedIN, the suggested proposal works.. in essence max.appenddelay can be used to turn "creationTime" into "logAppendTime". this does mean that at LinkedIn we won't be able to use

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

2015-10-20 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/333 KAFKA-2667: Fix transient error in KafkaBasedLogTest. The test required a specific sequence of events for each Consumer.poll() call, but the MockConsumer.waitForPollThen() method could not

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-10-20 Thread Joel Koshy
I’m in favor of adding the create-time in the message (although some would argue even that should really be an application-level header), but I don’t think it should be mutable after it leaves the client and I think we should avoid having the server use that for any server-side indexing. The

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

2015-10-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2667: - Component/s: copycat > Copycat KafkaBasedLogTest.testSendAndReadToEnd transient

[GitHub] kafka pull request: KAFKA-1686; Implement SASL/Kerberos

2015-10-20 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/334 KAFKA-1686; Implement SASL/Kerberos This PR implements SASL/Kerberos which was originally submitted by @harshach as https://github.com/apache/kafka/pull/191. I've been submitting PRs to

[jira] [Commented] (KAFKA-1686) Implement SASL/Kerberos

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

Jenkins build is back to normal : kafka_system_tests #113

2015-10-20 Thread ewen
See

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

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965052#comment-14965052 ] Ismael Juma commented on KAFKA-2472: I did a quick spike: * Introduced a `Config` interface with the

[jira] [Resolved] (KAFKA-2628) KafkaOffsetBackingStoreTest.testGetSet transient test failure

2015-10-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2628?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-2628. -- Resolution: Fixed Fix Version/s: 0.9.0.0 Closing because this test has

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

2015-10-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2667: - Reviewer: Guozhang Wang Fix Version/s: 0.9.0.0 Status: Patch

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

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14964788#comment-14964788 ] Ismael Juma commented on KAFKA-2472: [~junrao], the reason why we have the warnings is that we are

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

2015-10-20 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava reassigned KAFKA-2667: Assignee: Ewen Cheslack-Postava > Copycat

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

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

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

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14964788#comment-14964788 ] Ismael Juma edited comment on KAFKA-2472 at 10/20/15 8:27 AM: -- [~junrao], the

[jira] [Assigned] (KAFKA-2617) Move protocol field default values to Protocol

2015-10-20 Thread Jakub Nowak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2617?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jakub Nowak reassigned KAFKA-2617: -- Assignee: Jakub Nowak > Move protocol field default values to Protocol >

[DISCUSS] KIP-39 Pinning controller to a broker

2015-10-20 Thread Abhishek Nigam
Hi, Can we please discuss this KIP. The background for this is that it allows us to pin controller to a broker. This is useful in a couple of scenarios: a) If we want to do a rolling bounce we can reduce the number of controller moves down to 1. b) Again pick a designated broker and reduce the

[jira] [Comment Edited] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965331#comment-14965331 ] Ismael Juma edited comment on KAFKA-2672 at 10/20/15 4:22 PM: -- [~junrao],

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

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965373#comment-14965373 ] Ismael Juma commented on KAFKA-2472: [~jkreps], good point regarding Partitioner and other public

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

2015-10-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965532#comment-14965532 ] Jason Gustafson commented on KAFKA-2674: [~onurkaraman] Maybe this issue can be addressed when we

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

2015-10-20 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965456#comment-14965456 ] Ashish K Singh commented on KAFKA-2671: --- [~gwenshap] and [~jkreps] as long as there is an easy way

[jira] [Assigned] (KAFKA-2664) Adding a new metric with several pre-existing metrics is very expensive

2015-10-20 Thread Aditya Auradkar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2664?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aditya Auradkar reassigned KAFKA-2664: -- Assignee: Aditya Auradkar (was: Onur Karaman) > Adding a new metric with several

Re: Slow request log in Kafka

2015-10-20 Thread Aditya Auradkar
Fair points. Kafka doesn't really have slow queries. I was thinking about this kind of log in response to a request processing slowdown we had during an internal release.. it's unlikely a slow query log would have really helped since it slowed down requests from all entities (see KAFKA-2664 for

[jira] [Commented] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2015-10-20 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965321#comment-14965321 ] Jun Rao commented on KAFKA-2672: [~ijuma], the consumer shouldn't send any requests until the SSL

[jira] [Created] (KAFKA-2675) SASL/Kerberos follow-up

2015-10-20 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2675: -- Summary: SASL/Kerberos follow-up Key: KAFKA-2675 URL: https://issues.apache.org/jira/browse/KAFKA-2675 Project: Kafka Issue Type: Sub-task Reporter:

[jira] [Commented] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965331#comment-14965331 ] Ismael Juma commented on KAFKA-2672: [~junrao], that is right. As far as I can see, it does not:

Re: [DISCUSS] KIP-36 - Rack aware replica assignment

2015-10-20 Thread Aditya Auradkar
Hey Allen, 1. If we choose fail fast topic creation, we will have topic creation failures while upgrading the cluster. I really doubt we want this behavior. Ideally, this should be invisible to clients of a cluster. Currently, each broker is effectively its own rack. So we probably can use the

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

2015-10-20 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2472?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965348#comment-14965348 ] Jay Kreps commented on KAFKA-2472: -- [~gwenshap] This should work with plugins too. The config helper code

[GitHub] kafka pull request: KAFKA-2657; Kafka clients fail to start if one...

2015-10-20 Thread apakulov
GitHub user apakulov opened a pull request: https://github.com/apache/kafka/pull/336 KAFKA-2657; Kafka clients fail to start if one of broker isn't resolved by DNS You can merge this pull request into a Git repository by running: $ git pull https://github.com/apakulov/kafka

[jira] [Commented] (KAFKA-2657) Kafka clients fail to start if one of broker isn't resolved by DNS

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

[jira] [Created] (KAFKA-2676) AclCommandTest has wrong package name

2015-10-20 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2676: -- Summary: AclCommandTest has wrong package name Key: KAFKA-2676 URL: https://issues.apache.org/jira/browse/KAFKA-2676 Project: Kafka Issue Type: Bug

[jira] [Commented] (KAFKA-2645) Document potentially breaking changes in the release notes for 0.9.0

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

[jira] [Created] (KAFKA-2677) Coordinator disconnects not propagated to new consumer

2015-10-20 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-2677: -- Summary: Coordinator disconnects not propagated to new consumer Key: KAFKA-2677 URL: https://issues.apache.org/jira/browse/KAFKA-2677 Project: Kafka

[jira] [Updated] (KAFKA-2617) Move protocol field default values to Protocol

2015-10-20 Thread Jakub Nowak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2617?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jakub Nowak updated KAFKA-2617: --- Status: Patch Available (was: In Progress) > Move protocol field default values to Protocol >

[jira] [Assigned] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2015-10-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-2672: -- Assignee: Jason Gustafson (was: Neha Narkhede) > SendFailedException when new

Re: [DISCUSS] KIP-39 Pinning controller to a broker

2015-10-20 Thread Neha Narkhede
Agree with Jay on staying away from pinning roles to brokers. This is actually harder to operate and monitor. Regarding the problems you mentioned- 1. Reducing the controller moves during rolling bounce is useful but really something that should be handled by the tooling. The root cause is that

[GitHub] kafka pull request: KAFKA-1686; Implement SASL/Kerberos

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

[GitHub] kafka pull request: KAFKA-2645: Document potentially breaking chan...

2015-10-20 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/337 KAFKA-2645: Document potentially breaking changes in the release note… …s for 0.9.0 You can merge this pull request into a Git repository by running: $ git pull

[jira] [Resolved] (KAFKA-1686) Implement SASL/Kerberos

2015-10-20 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1686?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-1686. Resolution: Fixed Issue resolved by pull request 334 [https://github.com/apache/kafka/pull/334] >

[jira] [Commented] (KAFKA-1686) Implement SASL/Kerberos

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

[jira] [Commented] (KAFKA-1686) Implement SASL/Kerberos

2015-10-20 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965789#comment-14965789 ] Jun Rao commented on KAFKA-1686: [~sriharsha], thanks a lot of the patch. I committed the sasl patch using

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

2015-10-20 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965782#comment-14965782 ] Jiangjie Qin commented on KAFKA-2674: - [~hachikuji] I am reviewing KAFKA-2464 and also noticed this. I

[jira] [Commented] (KAFKA-2617) Move protocol field default values to Protocol

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

[GitHub] kafka pull request: KAFKA-2617: Move protocol field default values...

2015-10-20 Thread Mszak
GitHub user Mszak opened a pull request: https://github.com/apache/kafka/pull/338 KAFKA-2617: Move protocol field default values to Protocol. You can merge this pull request into a Git repository by running: $ git pull https://github.com/Mszak/kafka

[jira] [Work started] (KAFKA-2617) Move protocol field default values to Protocol

2015-10-20 Thread Jakub Nowak (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2617?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2617 started by Jakub Nowak. -- > Move protocol field default values to Protocol >

Re: Kafka KIP meeting Oct 20 at 11:00am PST

2015-10-20 Thread Jun Rao
The following are the notes from today's KIP discussion. * KIP-38: No concerns with this KIP. Flavio will initiate the voting on this. * KIP-37: There are questions on how ACL, configurations, etc will work, and whether we should support "move" or not. We will discuss the details more in the

Re: [DISCUSS] KIP-39 Pinning controller to a broker

2015-10-20 Thread Jay Kreps
This seems like a step backwards--we really don't want people to manually manage the location of the controller and try to manually balance partitions off that broker. I think it might make sense to consider directly fixing the things you actual want to fix: 1. Two many controller moves--we could

[jira] [Commented] (KAFKA-2672) SendFailedException when new consumer is run with SSL

2015-10-20 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2672?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965675#comment-14965675 ] Ismael Juma commented on KAFKA-2672: Thanks Jason. If this is harmless and since it's only logged at

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

2015-10-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965803#comment-14965803 ] Jason Gustafson commented on KAFKA-2674: [~becket_qin] I think the only problem with that is that

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

2015-10-20 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/339 KAFKA-2618; Disable SSL renegotiation for 0.9.0.0 You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-2618-disable-renegotiation

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

2015-10-20 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965865#comment-14965865 ] Jiangjie Qin edited comment on KAFKA-2674 at 10/20/15 10:15 PM: I think it

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

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

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

2015-10-20 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965865#comment-14965865 ] Jiangjie Qin commented on KAFKA-2674: - I think it would be clearer if rebalance callback is only

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

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

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

2015-10-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965913#comment-14965913 ] Guozhang Wang commented on KAFKA-2674: -- Today we are already calling ```commitOffsetSync``` upon

[jira] [Commented] (KAFKA-1686) Implement SASL/Kerberos

2015-10-20 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965975#comment-14965975 ] Jun Rao commented on KAFKA-1686: Also, [~sriharsha], could you write up a wiki of using SASL like you did

[jira] [Updated] (KAFKA-2645) Document potentially breaking changes in the release notes for 0.9.0

2015-10-20 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2645?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2645: - Reviewer: Jun Rao > Document potentially breaking changes in the release notes for 0.9.0 >

[GitHub] kafka pull request: KAFKA-2338: Warn on max.message.bytes change

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

[jira] [Commented] (KAFKA-1686) Implement SASL/Kerberos

2015-10-20 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14966076#comment-14966076 ] ASF GitHub Bot commented on KAFKA-1686: --- Github user harshach closed the pull request at:

[GitHub] kafka pull request: KAFKA-1686: Implement SASL/Kerberos.

2015-10-20 Thread harshach
Github user harshach closed the pull request at: https://github.com/apache/kafka/pull/191 --- 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

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

2015-10-20 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14965935#comment-14965935 ] Jason Gustafson commented on KAFKA-2674: Since LeaveGroup will cause a group rebalance, it doesn't

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

2015-10-20 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-1686; Implement SASL/Kerberos -- [...truncated 4564 lines...] org.apache.kafka.copycat.file.FileStreamSourceConnectorTest > testMultipleSourcesInvalid PASSED

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

2015-10-20 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2674?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14966003#comment-14966003 ] Jiangjie Qin commented on KAFKA-2674: - [~guozhang] The original reason we add consumer rebalance

Re: [DISCUSS] KIP-39 Pinning controller to a broker

2015-10-20 Thread Abhishek Nigam
Hi Jay/Neha, I just subscribed to the mailing list so I read your response but did not receive your email so adding the context into this email thread. " Agree with Jay on staying away from pinning roles to brokers. This is actually harder to operate and monitor. Regarding the problems you

[GitHub] kafka pull request: KAFKA 2480 Add backoff timeout and support rew...

2015-10-20 Thread Ishiihara
GitHub user Ishiihara opened a pull request: https://github.com/apache/kafka/pull/340 KAFKA 2480 Add backoff timeout and support rewinds You can merge this pull request into a Git repository by running: $ git pull https://github.com/Ishiihara/kafka backoff Alternatively you

[jira] [Commented] (KAFKA-1686) Implement SASL/Kerberos

2015-10-20 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1686?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14966059#comment-14966059 ] Sriharsha Chintalapani commented on KAFKA-1686: --- [~junrao] working on it. I'll post it on

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

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

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

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

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

2015-10-20 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2678: -- Summary: partition level lag metrics can be negative Key: KAFKA-2678 URL: https://issues.apache.org/jira/browse/KAFKA-2678 Project: Kafka Issue Type: Bug

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

2015-10-20 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2678?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-2678: Assignee: Dong Lin > partition level lag metrics can be negative >

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

2015-10-20 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2338; Warn on max.message.bytes change -- [...truncated 4569 lines...] org.apache.kafka.copycat.file.FileStreamSourceConnectorTest > testMultipleSourcesInvalid

[jira] [Created] (KAFKA-2679) Mirror Maker produces duplicated data

2015-10-20 Thread He Tianyi (JIRA)
He Tianyi created KAFKA-2679: Summary: Mirror Maker produces duplicated data Key: KAFKA-2679 URL: https://issues.apache.org/jira/browse/KAFKA-2679 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-39 Pinning controller to a broker

2015-10-20 Thread Neha Narkhede
> > I will update the KIP on how we can optimize the placement of controller > (pinning it to a preferred broker id (potentially config enabled) ) if that > sounds reasonable. The point I (and I think Jay too) was making is that pinning a controller to a broker through config is what we should

Re: [DISCUSS] KIP-39 Pinning controller to a broker

2015-10-20 Thread Aditya Auradkar
Hi Abhishek - Perhaps it would help if you explained the motivation behind your proposal. I know there was a bunch of discussion on KAFKA-1778, can you summarize? Currently, I'd agree with Neha and Jay that there isn't really a strong reason to pin the controller to a given broker or restricted

[GitHub] kafka pull request: Fix bash scripts to use `/usr/bin/env`.

2015-10-20 Thread aloiscochard
GitHub user aloiscochard opened a pull request: https://github.com/apache/kafka/pull/335 Fix bash scripts to use `/usr/bin/env`. Which makes them compatible with NixOS. You can merge this pull request into a Git repository by running: $ git pull

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

2015-10-20 Thread Michal Turek (JIRA)
Michal Turek created KAFKA-2674: --- Summary: ConsumerRebalanceListener.onPartitionsRevoked() is not called on consumer close Key: KAFKA-2674 URL: https://issues.apache.org/jira/browse/KAFKA-2674 Project: