Re: [DISCUSS] KIP-49: Fair Partition Assignment Strategy

2016-02-26 Thread Joel Koshy
Hi Andrew, Thanks for the wiki. Just a couple of comments: - The disruptive config change issue that you mentioned is pretty much a non-issue in the new consumer due to central assignment. - Optional: but it may be helpful to add a concrete example. - More of an orthogonal

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-02-26 Thread Ashish Singh
Parth, Thanks again for the awesome write up. Following our discussion from the JIRA, I think it will be easier to compare various alternatives if they are listed together. I am stating below a few alternatives along with a the current proposal. (Current proposal) Store Delegation Token, DT, on

Build failed in Jenkins: kafka-trunk-jdk7 #1071

2016-02-26 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3278: concatenate thread name to clientId when producer and -- [...truncated 1907 lines...] at

Re: Upgrading to Kafka 0.9.x

2016-02-26 Thread Joel Koshy
The 0.9 release still has the old consumer as Jay mentioned but this specific release is a little unusual in that it also provides a completely new consumer client. Based on what I understand, users of Kafka need to upgrade their brokers to > Kafka 0.9.x first, before they upgrade their clients

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

2016-02-26 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-3278: concatenate thread name to clientId when producer and -- [...truncated 1512 lines...] kafka.integration.AutoOffsetResetTest >

[GitHub] kafka pull request: MINOR: Add vagrant up wrapper for simple paral...

2016-02-26 Thread granders
GitHub user granders opened a pull request: https://github.com/apache/kafka/pull/982 MINOR: Add vagrant up wrapper for simple parallel bringup on aws The main impediment to bringing up aws machines in parallel using vagrant was the interaction between `vagrant-hostmanager` and

Re: Upgrading to Kafka 0.9.x

2016-02-26 Thread Mark Grover
Thanks Jay. Yeah, if we were able to use the old consumer API from 0.9 clients to work with 0.8 brokers that would have been super helpful here. I am just trying to avoid a scenario where Spark cares about new features from every new major release of Kafka (which is a good thing) but ends up

[jira] [Commented] (KAFKA-3299) KafkaConnect: DistributedHerder shouldn't wait forever to read configs after rebalance

2016-02-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15170172#comment-15170172 ] ASF GitHub Bot commented on KAFKA-3299: --- GitHub user gwenshap opened a pull request:

[GitHub] kafka pull request: KAFKA-3299: Ensure that reading config log on ...

2016-02-26 Thread gwenshap
GitHub user gwenshap opened a pull request: https://github.com/apache/kafka/pull/981 KAFKA-3299: Ensure that reading config log on rebalance doesn't hang the herder You can merge this pull request into a Git repository by running: $ git pull https://github.com/gwenshap/kafka

Build failed in Jenkins: kafka-trunk-jdk7 #1070

2016-02-26 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-3214: Added system tests for compressed topics -- [...truncated 5600 lines...] org.apache.kafka.connect.storage.KafkaStatusBackingStoreTest >

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

2016-02-26 Thread Apache Jenkins Server
See Changes: [cshapi] KAFKA-3214: Added system tests for compressed topics -- [...truncated 5025 lines...] org.apache.kafka.streams.processor.TopologyBuilderTest > testAddSinkWithSameName

[jira] [Commented] (KAFKA-3201) Add system test for KIP-31 and KIP-32 - Upgrade Test

2016-02-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15170091#comment-15170091 ] ASF GitHub Bot commented on KAFKA-3201: --- GitHub user apovzner opened a pull request:

[GitHub] kafka pull request: KAFKA-3201: Added rolling upgrade system tests...

2016-02-26 Thread apovzner
GitHub user apovzner opened a pull request: https://github.com/apache/kafka/pull/980 KAFKA-3201: Added rolling upgrade system tests from 0.8 and 0.9 to 0.10 Three main tests: 1. Setup: Producer (0.8) → Kafka Cluster → Consumer (0.8) First rolling bounce: Set

[jira] [Created] (KAFKA-3299) KafkaConnect: DistributedHerder shouldn't wait forever to read configs after rebalance

2016-02-26 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-3299: --- Summary: KafkaConnect: DistributedHerder shouldn't wait forever to read configs after rebalance Key: KAFKA-3299 URL: https://issues.apache.org/jira/browse/KAFKA-3299

[jira] [Commented] (KAFKA-3278) clientId is not unique in producer/consumer registration leads to mbean warning

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

[GitHub] kafka pull request: KAFKA-3278 concatenate thread name to clientId...

2016-02-26 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/978 --- 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] [Updated] (KAFKA-3278) clientId is not unique in producer/consumer registration leads to mbean warning

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

[DISCUSS] KIP-49: Fair Partition Assignment Strategy

2016-02-26 Thread Olson,Andrew
Here is a proposal for a new partition assignment strategy, https://cwiki.apache.org/confluence/display/KAFKA/KIP-49+-+Fair+Partition+Assignment+Strategy This KIP corresponds to these two pending pull requests, https://github.com/apache/kafka/pull/146 https://github.com/apache/kafka/pull/979

[DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-02-26 Thread Parth Brahmbhatt
Hi, I have filed KIP-48 so we can offer hadoop like delegation tokens in kafka. You can review the design https://cwiki.apache.org/confluence/display/KAFKA/KIP-48+Delegation+token+support+for+Kafka. This KIP depends on KIP-43 and we have also discussed an alternative to proposed design

[GitHub] kafka pull request: KAFKA-3214: Added system tests for compressed ...

2016-02-26 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/958 --- 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-3214) Add consumer system tests for compressed topics

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

Jenkins build is back to normal : kafka-trunk-jdk7 #1069

2016-02-26 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169673#comment-15169673 ] Jason Gustafson commented on KAFKA-3296: >From the log, it looks like the consumer cannot find the

[jira] [Resolved] (KAFKA-3298) Document unclean.leader.election.enable as a valid topic-level config

2016-02-26 Thread Joel Koshy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3298?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Koshy resolved KAFKA-3298. --- Resolution: Duplicate Thanks for the report. KAFKA-3234 addresses this and other such

[jira] [Updated] (KAFKA-3298) Document unclean.leader.election.enable as a valid topic-level config

2016-02-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3298?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-3298: Description: The http://kafka.apache.org/documentation.html#topic-config section does not

Re: Upgrading to Kafka 0.9.x

2016-02-26 Thread Jay Kreps
Hey, yeah, we'd really like to make this work well for you guys. I think there are actually maybe two questions here: 1. How should this work in steady state? 2. Given that there was a major reworking of the kafka consumer java library for 0.9 how does that impact things right now? (

[jira] [Commented] (KAFKA-1215) Rack-Aware replica assignment option

2016-02-26 Thread Allen Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1215?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169592#comment-15169592 ] Allen Wang commented on KAFKA-1215: --- [~aauradkar] Yes it is ready for review. > Rack-Aware replica

[jira] [Created] (KAFKA-3298) Document unclean.leader.election.enable as a valid topic-level config

2016-02-26 Thread Andrew Olson (JIRA)
Andrew Olson created KAFKA-3298: --- Summary: Document unclean.leader.election.enable as a valid topic-level config Key: KAFKA-3298 URL: https://issues.apache.org/jira/browse/KAFKA-3298 Project: Kafka

[GitHub] kafka pull request: KAFKA-3243: Fix Kafka basic ops documentation ...

2016-02-26 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/923 --- 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-3243) Fix Kafka basic ops documentation for Mirror maker, blacklist is not supported for new consumers

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

[jira] [Updated] (KAFKA-3243) Fix Kafka basic ops documentation for Mirror maker, blacklist is not supported for new consumers

2016-02-26 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3243?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-3243: - Resolution: Fixed Fix Version/s: 0.10.0.0 Status: Resolved

[jira] [Commented] (KAFKA-1995) JMS to Kafka: Inbuilt JMSAdaptor/JMSProxy/JMSBridge (Client can speak JMS but hit Kafka)

2016-02-26 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169426#comment-15169426 ] James Cheng commented on KAFKA-1995: Now that Kafka Connect has been released, a good way to do this

Upgrading to Kafka 0.9.x

2016-02-26 Thread Mark Grover
Hi Kafka devs, I come to you with a dilemma and a request. Based on what I understand, users of Kafka need to upgrade their brokers to Kafka 0.9.x first, before they upgrade their clients to Kafka 0.9.x. However, that presents a problem to other projects that integrate with Kafka (Spark, Flume,

[jira] [Commented] (KAFKA-2818) Clean up Controller Object on forced Resignation

2016-02-26 Thread Matthew Bruce (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2818?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169173#comment-15169173 ] Matthew Bruce commented on KAFKA-2818: -- [~fpj] You would definitely know that code better than me.

[jira] [Commented] (KAFKA-3297) More optimally balanced partition assignment strategy (new consumer)

2016-02-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169074#comment-15169074 ] ASF GitHub Bot commented on KAFKA-3297: --- GitHub user noslowerdna opened a pull request:

[GitHub] kafka pull request: KAFKA-3297: Fair consumer partition assignment...

2016-02-26 Thread noslowerdna
GitHub user noslowerdna opened a pull request: https://github.com/apache/kafka/pull/979 KAFKA-3297: Fair consumer partition assignment strategy (new consumer) Pull request for https://issues.apache.org/jira/browse/KAFKA-3297 You can merge this pull request into a Git repository by

[jira] [Work started] (KAFKA-3297) More optimally balanced partition assignment strategy (new consumer)

2016-02-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3297?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-3297 started by Andrew Olson. --- > More optimally balanced partition assignment strategy (new consumer) >

[jira] [Updated] (KAFKA-3297) More optimally balanced partition assignment strategy (new consumer)

2016-02-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3297?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-3297: Description: While the roundrobin partition assignment strategy is an improvement over the range

[jira] [Updated] (KAFKA-2435) More optimally balanced partition assignment strategy

2016-02-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2435?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-2435: Description: While the roundrobin partition assignment strategy is an improvement over the range

[jira] [Updated] (KAFKA-2435) More optimally balanced partition assignment strategy

2016-02-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2435?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-2435: Description: While the roundrobin partition assignment strategy is an improvement over the range

[jira] [Updated] (KAFKA-3297) More optimally balanced partition assignment strategy (new consumer)

2016-02-26 Thread Andrew Olson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3297?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Olson updated KAFKA-3297: Description: While the roundrobin partition assignment strategy is an improvement over the range

[jira] [Created] (KAFKA-3297) More optimally balanced partition assignment strategy (new consumer)

2016-02-26 Thread Andrew Olson (JIRA)
Andrew Olson created KAFKA-3297: --- Summary: More optimally balanced partition assignment strategy (new consumer) Key: KAFKA-3297 URL: https://issues.apache.org/jira/browse/KAFKA-3297 Project: Kafka

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169032#comment-15169032 ] Ismael Juma commented on KAFKA-3296: [~hachikuji] any thoughts? > All consumer reads hang

[jira] [Comment Edited] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169007#comment-15169007 ] Simon Cooper edited comment on KAFKA-3296 at 2/26/16 1:26 PM: -- Example code

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169007#comment-15169007 ] Simon Cooper commented on KAFKA-3296: - Example code that breaks that gets the start and end offset of

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15169000#comment-15169000 ] Simon Cooper commented on KAFKA-3296: - Looks like this breaks for every topic on the broker (only

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168995#comment-15168995 ] Ismael Juma commented on KAFKA-3296: Can you share your code please? > All consumer reads hang

[jira] [Comment Edited] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168952#comment-15168952 ] Simon Cooper edited comment on KAFKA-3296 at 2/26/16 1:07 PM: -- We've observed

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168952#comment-15168952 ] Simon Cooper commented on KAFKA-3296: - We've observed the same behaviour with a 0.9.0.0 consumer &

[jira] [Commented] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168944#comment-15168944 ] Ismael Juma commented on KAFKA-3296: Thanks for the report [~thecoop1984]. Can you please confirm that

[jira] [Updated] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3296?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Simon Cooper updated KAFKA-3296: Description: We've got several integration tests that bring up systems on VMs for testing. We've

[jira] [Created] (KAFKA-3296) All consumer reads hang indefinately

2016-02-26 Thread Simon Cooper (JIRA)
Simon Cooper created KAFKA-3296: --- Summary: All consumer reads hang indefinately Key: KAFKA-3296 URL: https://issues.apache.org/jira/browse/KAFKA-3296 Project: Kafka Issue Type: Bug Affects

[jira] [Commented] (KAFKA-3278) clientId is not unique in producer/consumer registration leads to mbean warning

2016-02-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168803#comment-15168803 ] ASF GitHub Bot commented on KAFKA-3278: --- GitHub user tomdearman opened a pull request:

[GitHub] kafka pull request: KAFKA-3278 concatenate thread name to clientId...

2016-02-26 Thread tomdearman
GitHub user tomdearman opened a pull request: https://github.com/apache/kafka/pull/978 KAFKA-3278 concatenate thread name to clientId when producer and consumers config is created @guozhangwang made the changes as requested, I reverted my original commit and that seems to have

[jira] [Commented] (KAFKA-3278) clientId is not unique in producer/consumer registration leads to mbean warning

2016-02-26 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168748#comment-15168748 ] ASF GitHub Bot commented on KAFKA-3278: --- Github user tomdearman closed the pull request at:

[GitHub] kafka pull request: KAFKA-3278 add thread number to clientId passe...

2016-02-26 Thread tomdearman
Github user tomdearman closed the pull request at: https://github.com/apache/kafka/pull/965 --- 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] [Comment Edited] (KAFKA-1995) JMS to Kafka: Inbuilt JMSAdaptor/JMSProxy/JMSBridge (Client can speak JMS but hit Kafka)

2016-02-26 Thread Christian Ferrari (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168649#comment-15168649 ] Christian Ferrari edited comment on KAFKA-1995 at 2/26/16 8:22 AM: --- A

[jira] [Comment Edited] (KAFKA-1995) JMS to Kafka: Inbuilt JMSAdaptor/JMSProxy/JMSBridge (Client can speak JMS but hit Kafka)

2016-02-26 Thread Christian Ferrari (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168649#comment-15168649 ] Christian Ferrari edited comment on KAFKA-1995 at 2/26/16 8:23 AM: --- A

[jira] [Commented] (KAFKA-1995) JMS to Kafka: Inbuilt JMSAdaptor/JMSProxy/JMSBridge (Client can speak JMS but hit Kafka)

2016-02-26 Thread Christian Ferrari (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1995?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15168649#comment-15168649 ] Christian Ferrari commented on KAFKA-1995: -- A JMS bridge would be very helpful to support this

[jira] [Created] (KAFKA-3295) test submit jira issue

2016-02-26 Thread andy (JIRA)
andy created KAFKA-3295: --- Summary: test submit jira issue Key: KAFKA-3295 URL: https://issues.apache.org/jira/browse/KAFKA-3295 Project: Kafka Issue Type: Test Affects Versions: 0.9.0.1