[jira] [Commented] (KAFKA-2841) Group metadata cache loading is not safe when reloading a partition

2015-11-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15005174#comment-15005174 ] Jason Gustafson commented on KAFKA-2841: [~guozhang] Yeah, I think that addresses

[jira] [Updated] (KAFKA-2841) Group metadata cache loading is not safe when reloading a partition

2015-11-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2841: --- Fix Version/s: 0.9.0.0 > Group metadata cache loading is not safe when reloading a partition > ---

[jira] [Commented] (KAFKA-2841) Group metadata cache loading is not safe when reloading a partition

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15005170#comment-15005170 ] Guozhang Wang commented on KAFKA-2841: -- [~hachikuji] Is this solvable in KAFKA-2721?

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

2015-11-13 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-2841) Group metadata cache loading is not safe when reloading a partition

2015-11-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2841?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2841: --- Description: If the coordinator receives a leaderAndIsr request which includes a higher leade

[jira] [Created] (KAFKA-2841) Group metadata cache loading is not safe when reloading a partition

2015-11-13 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-2841: -- Summary: Group metadata cache loading is not safe when reloading a partition Key: KAFKA-2841 URL: https://issues.apache.org/jira/browse/KAFKA-2841 Project: Kafka

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] trivial 0.9.0 doc changes to fix links -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-2 (docker Ubuntu ubuntu) in

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2838; Allow comma in super users, allow comma in CLI authz prop… -- [...truncated 2763 lines...] kafka.log.LogTest > testParseTopicPartitionNameForMissingPartition

Build failed in Jenkins: kafka_0.9.0_jdk7 #23

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2838; Allow comma in super users, allow comma in CLI authz prop… [junrao] trivial 0.9.0 doc changes to AclCommand [junrao] trivial 0.9.0 doc changes to fix links -

[VOTE] 0.9.0.0 Candiate 2

2015-11-13 Thread Jun Rao
This is the second candidate for release of Apache Kafka 0.9.0.0. This a major release that includes (1) authentication (through SSL and SASL) and authorization, (2) a new java consumer, (3) a Kafka connect framework for data ingestion and egression, and (4) quotas. Since this is a major release, w

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2838; Allow comma in super users, allow comma in CLI authz prop… [junrao] trivial 0.9.0 doc changes to AclCommand -- Started by an SCM change [EnvInject] - Loading

Re: [kafka-clients] Re: [VOTE] 0.9.0.0 Candiate 1

2015-11-13 Thread Jun Rao
Thanks for identifying the issues. Will roll out RC2. Jun On Tue, Nov 10, 2015 at 12:42 PM, Gwen Shapira wrote: > BTW. I created a Jenkins job for the 0.9 branch: > https://builds.apache.org/job/kafka_0.9.0_jdk7/ > > Right now its pretty much identical to trunk, but since they may diverge, I >

[jira] [Resolved] (KAFKA-2838) allow comma when specifying superusers

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

[jira] [Commented] (KAFKA-2838) allow comma when specifying superusers

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

[GitHub] kafka pull request: KAFKA-2838: Allow comma in super users, allow ...

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/529 --- 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 #152

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] additional trivial 0.9.0 doc changes -- [...truncated 4410 lines...] org.apache.kafka.connect.json.JsonConverterTest > nullToConnect PASSED org.apache.kafka.connect.json.Js

[jira] [Commented] (KAFKA-2840) Support security in kafka-consumer-groups.sh

2015-11-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2840?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004943#comment-15004943 ] Jason Gustafson commented on KAFKA-2840: Sure is. Thanks for noting. > Support se

[jira] [Resolved] (KAFKA-2840) Support security in kafka-consumer-groups.sh

2015-11-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2840?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-2840. Resolution: Duplicate > Support security in kafka-consumer-groups.sh > -

[jira] [Commented] (KAFKA-2840) Support security in kafka-consumer-groups.sh

2015-11-13 Thread Ashish K Singh (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2840?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004941#comment-15004941 ] Ashish K Singh commented on KAFKA-2840: --- Duplicate of KAFKA-2746? > Support securit

[gradle build] The wrapper should be in the repository

2015-11-13 Thread jeanbaptiste lespiau
Hi everyone, When following the setup page [ https://cwiki.apache.org/confluence/display/KAFKA/Developer+Setup] running : ./gradlew eclipse I hit an error : Could not find or load main class org.gradle.wrapper.GradleWrapperMain When looking at the gradle documentation [ https://docs.gradle.org

[jira] [Created] (KAFKA-2840) Support security in kafka-consumer-groups.sh

2015-11-13 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-2840: -- Summary: Support security in kafka-consumer-groups.sh Key: KAFKA-2840 URL: https://issues.apache.org/jira/browse/KAFKA-2840 Project: Kafka Issue Type: Bu

[jira] [Comment Edited] (KAFKA-2831) kafka-consumer-groups requires zookeeper url when using the new-consumer option

2015-11-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004890#comment-15004890 ] Ismael Juma edited comment on KAFKA-2831 at 11/13/15 11:20 PM: -

[jira] [Updated] (KAFKA-2831) kafka-consumer-groups requires zookeeper url when using the new-consumer option

2015-11-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2831: --- Reviewer: Jun Rao Status: Patch Available (was: In Progress) I submitted a PR https://github.co

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

2015-11-13 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2838) allow comma when specifying superusers

2015-11-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004864#comment-15004864 ] ASF GitHub Bot commented on KAFKA-2838: --- GitHub user Parth-Brahmbhatt opened a pull

[GitHub] kafka pull request: KAFKA-2838: Allow comma in super users, allow ...

2015-11-13 Thread Parth-Brahmbhatt
GitHub user Parth-Brahmbhatt opened a pull request: https://github.com/apache/kafka/pull/529 KAFKA-2838: Allow comma in super users, allow comma in CLI authz prop… …erties. You can merge this pull request into a Git repository by running: $ git pull https://github.com/Parth

[jira] [Work started] (KAFKA-2831) kafka-consumer-groups requires zookeeper url when using the new-consumer option

2015-11-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2831 started by Ismael Juma. -- > kafka-consumer-groups requires zookeeper url when using the new-consumer > option

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2833; print only group offset / metadata according to the -- [...truncated 37 lines...] :core:clean :examples:clean :log4j-appender:clean :streams:clean :tools:clean :

[jira] [Resolved] (KAFKA-2833) OffsetsMessageFormatter hits unexpected exception

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

[GitHub] kafka pull request: KAFKA-2833: print only group offset / metadata...

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/527 --- 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-2833) OffsetsMessageFormatter hits unexpected exception

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

[jira] [Created] (KAFKA-2839) Kafka connect log test failing

2015-11-13 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2839: --- Summary: Kafka connect log test failing Key: KAFKA-2839 URL: https://issues.apache.org/jira/browse/KAFKA-2839 Project: Kafka Issue Type: Sub-task R

[jira] [Commented] (KAFKA-2838) allow comma when specifying superusers

2015-11-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004774#comment-15004774 ] Jun Rao commented on KAFKA-2838: [~parth.brahmbhatt], do you think this is an issue? An ea

[jira] [Created] (KAFKA-2838) allow comma when specifying superusers

2015-11-13 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2838: -- Summary: allow comma when specifying superusers Key: KAFKA-2838 URL: https://issues.apache.org/jira/browse/KAFKA-2838 Project: Kafka Issue Type: Bug Reporter

[GitHub] kafka pull request: Do not use ZKUtils in `ConsumerGroupCommand` i...

2015-11-13 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/528 Do not use ZKUtils in `ConsumerGroupCommand` if `new-consumer` is used You can merge this pull request into a Git repository by running: $ git pull https://github.com/ijuma/kafka kafka-2831-cons

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2768: AdminClient ignore member list for non-stable groups -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H11 (Ub

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

2015-11-13 Thread Apache Jenkins Server
See -- [...truncated 4452 lines...] org.apache.kafka.connect.json.JsonConverterTest > nullToConnect PASSED org.apache.kafka.connect.json.JsonConverterTest > timeToJson PASSED org.apache.kafka.connect.js

[jira] [Updated] (KAFKA-2835) FAILING TEST: LogCleaner

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2835?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2835: - Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 > FAILING TEST: LogCleaner > -

[jira] [Updated] (KAFKA-2836) FAILING TEST: SaslSslTopicMetadataTest

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2836: - Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 > FAILING TEST: SaslSslTopicMetadataTe

[jira] [Updated] (KAFKA-2836) FAILING TEST: SaslSslTopicMetadataTest

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2836?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2836: - Labels: newbiee (was: ) > FAILING TEST: SaslSslTopicMetadataTest > --

[jira] [Updated] (KAFKA-2837) FAILING TEST: kafka.api.ProducerBounceTest > testBrokerFailure

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2837: - Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 > FAILING TEST: kafka.api.ProducerBoun

[jira] [Updated] (KAFKA-2837) FAILING TEST: kafka.api.ProducerBounceTest > testBrokerFailure

2015-11-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira updated KAFKA-2837: Summary: FAILING TEST: kafka.api.ProducerBounceTest > testBrokerFailure (was: TEST FAILURE: kafka.

[jira] [Comment Edited] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-11-13 Thread Ralph Tice (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004110#comment-15004110 ] Ralph Tice edited comment on KAFKA-2082 at 11/13/15 9:12 PM: -

[jira] [Commented] (KAFKA-2834) kafka-merge-pr.py should run unit tests before pushing it to trunk

2015-11-13 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004729#comment-15004729 ] Sriharsha Chintalapani commented on KAFKA-2834: --- [~guozhang] Its good wait t

[jira] [Created] (KAFKA-2837) TEST FAILURE: kafka.api.ProducerBounceTest > testBrokerFailure

2015-11-13 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2837: --- Summary: TEST FAILURE: kafka.api.ProducerBounceTest > testBrokerFailure Key: KAFKA-2837 URL: https://issues.apache.org/jira/browse/KAFKA-2837 Project: Kafka

[jira] [Created] (KAFKA-2836) FAILING TEST: SaslSslTopicMetadataTest

2015-11-13 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2836: --- Summary: FAILING TEST: SaslSslTopicMetadataTest Key: KAFKA-2836 URL: https://issues.apache.org/jira/browse/KAFKA-2836 Project: Kafka Issue Type: Bug

[jira] [Created] (KAFKA-2835) FAILING TEST: LogCleaner

2015-11-13 Thread Gwen Shapira (JIRA)
Gwen Shapira created KAFKA-2835: --- Summary: FAILING TEST: LogCleaner Key: KAFKA-2835 URL: https://issues.apache.org/jira/browse/KAFKA-2835 Project: Kafka Issue Type: Bug Reporter: Gw

[jira] [Commented] (KAFKA-2500) Make logEndOffset available in the 0.8.3 Consumer

2015-11-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004707#comment-15004707 ] Jason Gustafson commented on KAFKA-2500: [~wushujames] Yeah, that's right. It uses

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2791; removed deprecated properties [junrao] KAFKA-2821; fix deadlock in group metadata write callback -- [...truncated 1408 lines...] kafka.log.LogTest > testParseT

[jira] [Resolved] (KAFKA-2768) New-consumer sends invalid describeGroupResponse while restabilizing

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2768?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2768. -- Resolution: Fixed Fix Version/s: 0.9.0.0 Issue resolved by pull request 447 [https://gith

[jira] [Commented] (KAFKA-2768) New-consumer sends invalid describeGroupResponse while restabilizing

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

[GitHub] kafka pull request: KAFKA-2768: New-consumer sends invalid describ...

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/447 --- 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-2834) kafka-merge-pr.py should run unit tests before pushing it to trunk

2015-11-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004666#comment-15004666 ] Gwen Shapira commented on KAFKA-2834: - The queue is longer than normal because of some

[GitHub] kafka pull request: KAFKA-2833: print only group offset / metadata...

2015-11-13 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/527 KAFKA-2833: print only group offset / metadata according to the formatter You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka

[jira] [Commented] (KAFKA-2833) OffsetsMessageFormatter hits unexpected exception

2015-11-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004649#comment-15004649 ] ASF GitHub Bot commented on KAFKA-2833: --- GitHub user guozhangwang opened a pull requ

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [cshapi] Minor: Missing License [junrao] trivial doc changes [junrao] KAFKA-2791; removed deprecated properties [junrao] KAFKA-2821; fix deadlock in group metadata write callback

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2830; Change default fix version to 0.9.1.0 in kafka-merge-pr.py -- [...truncated 3348 lines...] org.apache.kafka.clients.producer.internals.BufferPoolTest > test

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

2015-11-13 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2500) Make logEndOffset available in the 0.8.3 Consumer

2015-11-13 Thread James Cheng (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004639#comment-15004639 ] James Cheng commented on KAFKA-2500: [~hachikuji] Thanks for the status update and the

[jira] [Commented] (KAFKA-2834) kafka-merge-pr.py should run unit tests before pushing it to trunk

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004638#comment-15004638 ] Guozhang Wang commented on KAFKA-2834: -- The normal pattern of merging should wait for

[jira] [Commented] (KAFKA-2833) OffsetsMessageFormatter hits unexpected exception

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2833?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004580#comment-15004580 ] Guozhang Wang commented on KAFKA-2833: -- I missed in while doing KAFKA-2017, my bad. S

[jira] [Commented] (KAFKA-2834) kafka-merge-pr.py should run tests before pushing it to trunk

2015-11-13 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004573#comment-15004573 ] Sriharsha Chintalapani commented on KAFKA-2834: --- [~guozhang] unit tests actu

[jira] [Updated] (KAFKA-2834) kafka-merge-pr.py should run unit tests before pushing it to trunk

2015-11-13 Thread Sriharsha Chintalapani (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2834?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sriharsha Chintalapani updated KAFKA-2834: -- Summary: kafka-merge-pr.py should run unit tests before pushing it to trunk (wa

[jira] [Assigned] (KAFKA-2833) OffsetsMessageFormatter hits unexpected exception

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2833?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang reassigned KAFKA-2833: Assignee: Guozhang Wang > OffsetsMessageFormatter hits unexpected exception > -

[jira] [Commented] (KAFKA-2834) kafka-merge-pr.py should run tests before pushing it to trunk

2015-11-13 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2834?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004567#comment-15004567 ] Guozhang Wang commented on KAFKA-2834: -- Do you mean unit tests or integration tests?

Jenkins build is back to normal : kafka_0.9.0_jdk7 #19

2015-11-13 Thread Apache Jenkins Server
See

[GitHub] kafka pull request: KAFKA-2811: add standby tasks

2015-11-13 Thread ymatsuda
GitHub user ymatsuda opened a pull request: https://github.com/apache/kafka/pull/526 KAFKA-2811: add standby tasks @guozhangwang * added a new config param "num.standby.replicas" (the default value is 0). * added a new abstract class AbstractTask * added StandbyTask as a

[jira] [Commented] (KAFKA-2811) Add standby tasks

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

[jira] [Commented] (KAFKA-2821) Deadlock in group metadata persistence callback

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

[jira] [Resolved] (KAFKA-2821) Deadlock in group metadata persistence callback

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

[GitHub] kafka pull request: KAFKA-2821: fix deadlock in group metadata wri...

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/519 --- 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-2791) remove deprecated producer properties from console-producer

2015-11-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao resolved KAFKA-2791. Resolution: Fixed Fix Version/s: 0.9.0.0 Issue resolved by pull request 525 [https://github.com/apach

[GitHub] kafka pull request: KAFKA-2791: removed deprecated properties

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/525 --- 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-2791) remove deprecated producer properties from console-producer

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

[jira] [Created] (KAFKA-2834) kafka-merge-pr.py should run tests before pushing it to trunk

2015-11-13 Thread Sriharsha Chintalapani (JIRA)
Sriharsha Chintalapani created KAFKA-2834: - Summary: kafka-merge-pr.py should run tests before pushing it to trunk Key: KAFKA-2834 URL: https://issues.apache.org/jira/browse/KAFKA-2834 Project

[jira] [Commented] (KAFKA-2500) Make logEndOffset available in the 0.8.3 Consumer

2015-11-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004454#comment-15004454 ] Jason Gustafson commented on KAFKA-2500: [~willf] Apologies for the lack of progre

[GitHub] kafka pull request: Missing License.

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/524 --- 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-2830) Change default fix version to 0.9.1.0 in kafka-merge-pr.py

2015-11-13 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2830?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004441#comment-15004441 ] Gwen Shapira commented on KAFKA-2830: - I completely forgot requesting it, but thank yo

[jira] [Commented] (KAFKA-2791) remove deprecated producer properties from console-producer

2015-11-13 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2791?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004430#comment-15004430 ] ASF GitHub Bot commented on KAFKA-2791: --- GitHub user benstopford opened a pull reque

[GitHub] kafka pull request: KAFKA-2791: removed deprecated properties

2015-11-13 Thread benstopford
GitHub user benstopford opened a pull request: https://github.com/apache/kafka/pull/525 KAFKA-2791: removed deprecated properties Removed support for BLOCK_ON_BUFFER_FULL_CONFIG (block.on.buffer.full) Removed support for METADATA_FETCH_TIMEOUT_CONFIG Removed support for TIME

[jira] [Commented] (KAFKA-2830) Change default fix version to 0.9.1.0 in kafka-merge-pr.py

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

[jira] [Updated] (KAFKA-2830) Change default fix version to 0.9.1.0 in kafka-merge-pr.py

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

[GitHub] kafka pull request: KAFKA-2830; Change default fix version to 0.9....

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/523 --- 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: Missing License.

2015-11-13 Thread harshach
GitHub user harshach opened a pull request: https://github.com/apache/kafka/pull/524 Missing License. You can merge this pull request into a Git repository by running: $ git pull https://github.com/harshach/kafka missing-license Alternatively you can review and apply these ch

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2817; Check if socketChannel is connected in -- [...truncated 1401 lines...] kafka.log.LogTest > testParseTopicPartitionNameForMissingPartition PASSED kafka.log.LogT

[jira] [Created] (KAFKA-2833) OffsetsMessageFormatter hits unexpected exception

2015-11-13 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2833: -- Summary: OffsetsMessageFormatter hits unexpected exception Key: KAFKA-2833 URL: https://issues.apache.org/jira/browse/KAFKA-2833 Project: Kafka Issue Type: Bug

[jira] [Assigned] (KAFKA-2791) remove deprecated producer properties from console-producer

2015-11-13 Thread Ben Stopford (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2791?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ben Stopford reassigned KAFKA-2791: --- Assignee: Ben Stopford > remove deprecated producer properties from console-producer > --

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

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2817; Check if socketChannel is connected in -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on H11 (Ubuntu ubuntu) in

[jira] [Created] (KAFKA-2832) support exclude.internal.topics in new consumer

2015-11-13 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2832: -- Summary: support exclude.internal.topics in new consumer Key: KAFKA-2832 URL: https://issues.apache.org/jira/browse/KAFKA-2832 Project: Kafka Issue Type: New Feature

Build failed in Jenkins: kafka_0.9.0_jdk7 #18

2015-11-13 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2817; Check if socketChannel is connected in -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-1 (docker Ubuntu

[jira] [Commented] (KAFKA-2817) Closing an unconnected SslTransport get an invalid close state exception

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

[GitHub] kafka pull request: KAFKA-2817; Check if socketChannel is connecte...

2015-11-13 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/511 --- 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-2817) Closing an unconnected SslTransport get an invalid close state exception

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

[jira] [Assigned] (KAFKA-2831) kafka-consumer-groups requires zookeeper url when using the new-consumer option

2015-11-13 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma reassigned KAFKA-2831: -- Assignee: Ismael Juma > kafka-consumer-groups requires zookeeper url when using the new-consume

[jira] [Updated] (KAFKA-2831) kafka-consumer-groups requires zookeeper url when using the new-consumer option

2015-11-13 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2831?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2831: --- Description: bin/kafka-consumer-groups.sh --bootstrap-server localhost:9092 --group test-consumer-group --new

[jira] [Created] (KAFKA-2831) kafka-consumer-groups requires zookeeper url when using the new-consumer option

2015-11-13 Thread Jun Rao (JIRA)
Jun Rao created KAFKA-2831: -- Summary: kafka-consumer-groups requires zookeeper url when using the new-consumer option Key: KAFKA-2831 URL: https://issues.apache.org/jira/browse/KAFKA-2831 Project: Kafka

[jira] [Commented] (KAFKA-2150) FetcherThread backoff need to grab lock before wait on condition.

2015-11-13 Thread Ralph Tice (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2150?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004126#comment-15004126 ] Ralph Tice commented on KAFKA-2150: --- linking so it's clear on both issues 1461 raised a

[jira] [Commented] (KAFKA-2082) Kafka Replication ends up in a bad state

2015-11-13 Thread Ralph Tice (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15004110#comment-15004110 ] Ralph Tice commented on KAFKA-2082: --- I just ran into this error on 0.8.2.1 and one other

Re: Kafka Scalability & Partition

2015-11-13 Thread Prabhjot Bharaj
Hi, Having such a model will not scale. I think it it's mentioned in earlier posts as well as some wikis available out there Kafka works very well if you have more partitions, so you can massively parallel writes to Kafka. Also, your application need not send partition ids/numbers, unless you've

[jira] [Comment Edited] (KAFKA-2500) Make logEndOffset available in the 0.8.3 Consumer

2015-11-13 Thread Will Funnell (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2500?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15003924#comment-15003924 ] Will Funnell edited comment on KAFKA-2500 at 11/13/15 12:28 PM:

  1   2   >