[GitHub] kafka pull request: Pass the correct principal to `RequestChannel....

2015-11-17 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/541 Pass the correct principal to `RequestChannel.Session` Also change `Authorizer` to use `Principal` instead of `KafkaPrincipal` since this is what `KafkaChannel` returns. The other option would be

[jira] [Updated] (KAFKA-2849) Pass `KafkaChannel.principal` to `RequestChannel.Session`

2015-11-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2849: --- Status: Patch Available (was: Open) Pull request: https://github.com/apache/kafka/pull/541 > Pass

[jira] [Created] (KAFKA-2849) Pass `KafkaChannel.principal` to `RequestChannel.Session`

2015-11-17 Thread Ismael Juma (JIRA)
Ismael Juma created KAFKA-2849: -- Summary: Pass `KafkaChannel.principal` to `RequestChannel.Session` Key: KAFKA-2849 URL: https://issues.apache.org/jira/browse/KAFKA-2849 Project: Kafka Issue

Is a Kafka 0.9 broker supposed to connect to itself?

2015-11-17 Thread Damian Guy
I would think not I'm bringing up a new 0.9 cluster and i'm getting the below Exception (and the same thing on all nodes) - the IP address is the IP for the host the broker is running on. I think DNS is a bit stuffed on these machines and maybe that is the cause, but... any ideas? [2015-11-17

[jira] [Updated] (KAFKA-2847) remove principal.builder.class from client configs

2015-11-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2847: --- Reviewer: Jun Rao Status: Patch Available (was: Open) > remove principal.builder.class from

[jira] [Created] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

2015-11-17 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-2850: --- Summary: SslTransportLayerTest.testInvalidEndpointIdentification fails consistently Key: KAFKA-2850 URL: https://issues.apache.org/jira/browse/KAFKA-2850

[jira] [Updated] (KAFKA-2847) remove principal.builder.class from client configs

2015-11-17 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2847: --- Resolution: Fixed Fix Version/s: 0.9.0.0 Status: Resolved (was: Patch Available) Issue

[jira] [Commented] (KAFKA-2849) Pass `KafkaChannel.principal` to `RequestChannel.Session`

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

[GitHub] kafka pull request: KAFKA-2849; Pass the correct principal to `Req...

2015-11-17 Thread ijuma
Github user ijuma closed the pull request at: https://github.com/apache/kafka/pull/541 --- 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-2849) Pass `KafkaChannel.principal` to `RequestChannel.Session`

2015-11-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-2849: --- Resolution: Not A Problem Fix Version/s: (was: 0.9.0.0) Status: Resolved (was:

[GitHub] kafka pull request: KAFKA-2847; Remove principal builder class fro...

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/542 --- 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-2847) remove principal.builder.class from client configs

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

[GitHub] kafka pull request: KAFKA-2847; Remove principal builder class fro...

2015-11-17 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/542 KAFKA-2847; Remove principal builder class from client configs Also mark `PrincipalBuilder` as `Unstable` and tweak docs. You can merge this pull request into a Git repository by running: $ git

[jira] [Commented] (KAFKA-2847) remove principal.builder.class from client configs

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

issue with javaapi consumer

2015-11-17 Thread Kudumula, Surender
Hi all Iam currently using kafka 0.8.2.1 consumer which has kafka java api consumer but I had to upgrade to kafka 0.8.2.3 and when I add the jars for kafka 0.8.2.3 my consumer code doesn't compile. consumerConnector = ConsumerConfig.createJavaConsumerConnector(consumerConfig); Do I have to

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

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2847; Remove principal builder class from client configs -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-5

[GitHub] kafka pull request: MINOR: remove the group id from a restore cons...

2015-11-17 Thread ymatsuda
GitHub user ymatsuda opened a pull request: https://github.com/apache/kafka/pull/543 MINOR: remove the group id from a restore consumer @guozhangwang A restore consumer does not belong to a consumer group. You can merge this pull request into a Git repository by running:

Build failed in Jenkins: kafka_0.9.0_jdk7 #29

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2847; Remove principal builder class from client configs -- [...truncated 1518 lines...] at

[jira] [Created] (KAFKA-2851) system tests: error copying keytab file

2015-11-17 Thread Geoff Anderson (JIRA)
Geoff Anderson created KAFKA-2851: - Summary: system tests: error copying keytab file Key: KAFKA-2851 URL: https://issues.apache.org/jira/browse/KAFKA-2851 Project: Kafka Issue Type: Bug

[jira] [Assigned] (KAFKA-2851) system tests: error copying keytab file

2015-11-17 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2851?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner reassigned KAFKA-2851: --- Assignee: Anna Povzner > system tests: error copying keytab file >

[jira] [Work started] (KAFKA-2851) system tests: error copying keytab file

2015-11-17 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2851?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2851 started by Anna Povzner. --- > system tests: error copying keytab file >

[jira] [Created] (KAFKA-2852) Kafka Authroizer CLI should use consistent way to specify multiple values for all config options.

2015-11-17 Thread Parth Brahmbhatt (JIRA)
Parth Brahmbhatt created KAFKA-2852: --- Summary: Kafka Authroizer CLI should use consistent way to specify multiple values for all config options. Key: KAFKA-2852 URL:

[jira] [Created] (KAFKA-2853) Transient Failure in kafka.api.SaslSslConsumerTest.testPartitionReassignmentCallback

2015-11-17 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-2853: Summary: Transient Failure in kafka.api.SaslSslConsumerTest.testPartitionReassignmentCallback Key: KAFKA-2853 URL: https://issues.apache.org/jira/browse/KAFKA-2853

[GitHub] kafka pull request: MINOR: Introduce `producer.config` property to...

2015-11-17 Thread ijuma
GitHub user ijuma opened a pull request: https://github.com/apache/kafka/pull/544 MINOR: Introduce `producer.config` property to `ConsoleProducer` This makes it easier to pass security properties in the same way to `ConsoleConsumer` and `ConsoleProducer`. You can merge this

[GitHub] kafka pull request: KAFKA-2824: MiniKDC based tests don't run in V...

2015-11-17 Thread benstopford
GitHub user benstopford reopened a pull request: https://github.com/apache/kafka/pull/520 KAFKA-2824: MiniKDC based tests don't run in VirtualBox This is a hack which works. Is there a better way? Build of the replication_test.py running here:

[jira] [Commented] (KAFKA-2824) MiniKDC based tests don't run in VirtualBox

2015-11-17 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2824?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009255#comment-15009255 ] ASF GitHub Bot commented on KAFKA-2824: --- GitHub user benstopford reopened a pull request:

[GitHub] kafka pull request: Kafka-2852:Updating the Authorizer CLI to use ...

2015-11-17 Thread Parth-Brahmbhatt
GitHub user Parth-Brahmbhatt opened a pull request: https://github.com/apache/kafka/pull/545 Kafka-2852:Updating the Authorizer CLI to use a consistent way to specify a list of values for a config options. …ecify a list of values for a config options. You can merge this pull

[jira] [Commented] (KAFKA-2843) when consumer got empty messageset, fetchResponse.highWatermark != current_offset?

2015-11-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009268#comment-15009268 ] Guozhang Wang commented on KAFKA-2843: -- Is the simple consumer consuming from only one

[jira] [Created] (KAFKA-2856) add KTable

2015-11-17 Thread Yasuhiro Matsuda (JIRA)
Yasuhiro Matsuda created KAFKA-2856: --- Summary: add KTable Key: KAFKA-2856 URL: https://issues.apache.org/jira/browse/KAFKA-2856 Project: Kafka Issue Type: Sub-task Components:

[jira] [Commented] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

2015-11-17 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009790#comment-15009790 ] Flavio Junqueira commented on KAFKA-2850: - works for me, thanks [~rsivaram]. >

[jira] [Commented] (KAFKA-2854) Make KerberosName implement PrincipalToLocal plugin so authorizer and authenticator can share this.

2015-11-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009322#comment-15009322 ] Ismael Juma commented on KAFKA-2854: It would also be interesting to think how these relate to

[jira] [Commented] (KAFKA-2853) Transient Failure in kafka.api.SaslSslConsumerTest.testPartitionReassignmentCallback

2015-11-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2853?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009329#comment-15009329 ] Rajini Sivaram commented on KAFKA-2853: --- This could be the same issue as KAFKA-2718 > Transient

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

2015-11-17 Thread Alan Braithwaite (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2678?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009342#comment-15009342 ] Alan Braithwaite commented on KAFKA-2678: - Isn't it more accurate for it to be negative? You're

[jira] [Comment Edited] (KAFKA-2854) Make KerberosName implement PrincipalToLocal plugin so authorizer and authenticator can share this.

2015-11-17 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2854?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009322#comment-15009322 ] Ismael Juma edited comment on KAFKA-2854 at 11/17/15 7:26 PM: -- It would also

[jira] [Commented] (KAFKA-2843) when consumer got empty messageset, fetchResponse.highWatermark != current_offset?

2015-11-17 Thread netcafe (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2843?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009307#comment-15009307 ] netcafe commented on KAFKA-2843: This topic only one partition. > when consumer got empty messageset,

[GitHub] kafka pull request: Kafka 2746

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/534 --- 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-2334) Prevent HW from going back during leader failover

2015-11-17 Thread Albert Strasheim (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009364#comment-15009364 ] Albert Strasheim commented on KAFKA-2334: - Hitting this issue in production with 0.8.2.1 on a

[jira] [Updated] (KAFKA-2853) Reuse of temporary server data.dir could lead to transient failures.

2015-11-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2853?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2853: - Summary: Reuse of temporary server data.dir could lead to transient failures. (was: Transient

[jira] [Assigned] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

2015-11-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-2850: - Assignee: Rajini Sivaram > SslTransportLayerTest.testInvalidEndpointIdentification fails

Re: [gradle build] The wrapper should be in the repository

2015-11-17 Thread jeanbaptiste lespiau
Thank you :) 2015-11-17 7:26 GMT+01:00 Ewen Cheslack-Postava : > Hi, > > Those instructions simply fell out of date -- you can see in the README in > the repository that the first step after checkout is to bootstrap the > gradle wrapper. The wrapper is not included due to

[GitHub] kafka pull request: MINOR: fix shutdownHook in ConsoleConsumer

2015-11-17 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/548 MINOR: fix shutdownHook in ConsoleConsumer You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka HFConsoleConsumer

[jira] [Resolved] (KAFKA-2852) Kafka Authroizer CLI should use consistent way to specify multiple values for all config options.

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

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

2015-11-17 Thread Apache Jenkins Server
See Changes: [confluent] MINOR: add KStream merge operator [confluent] MINOR: remove the group id from a restore consumer [junrao] KAFKA-2852; Updating the Authorizer CLI to use a consistent way to [junrao] MINOR: fix shutdownHook

Re: Is a Kafka 0.9 broker supposed to connect to itself?

2015-11-17 Thread Jun Rao
There is inter-broker communication. It seems that the broker got a request more than the default allowed size (~10MB). How many topic/partitions do you have on this cluster? Do you have clients running on the broker host? Thanks, Jun On Tue, Nov 17, 2015 at 4:10 AM, Damian Guy

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

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

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

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

[GitHub] kafka pull request: KAFKA-2841: safe group metadata cache loading/...

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/530 --- 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-2854: Making KerberosShortNamer implemen...

2015-11-17 Thread Parth-Brahmbhatt
GitHub user Parth-Brahmbhatt opened a pull request: https://github.com/apache/kafka/pull/547 Kafka-2854: Making KerberosShortNamer implement an interface and making it pluggable. You can merge this pull request into a Git repository by running: $ git pull

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

2015-11-17 Thread Apache Jenkins Server
See Changes: [confluent] MINOR: add KStream merge operator -- [...truncated 1818 lines...] kafka.admin.AddPartitionsTest > testTopicDoesNotExist PASSED kafka.admin.AddPartitionsTest >

[jira] [Updated] (KAFKA-2854) Make KerberosName implement PrincipalToLocal plugin so authorizer and authenticator can share this.

2015-11-17 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2854?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2854: --- Fix Version/s: (was: 0.9.0.0) > Make KerberosName implement PrincipalToLocal plugin so authorizer and >

Re: [VOTE] 0.9.0.0 Candiate 2

2015-11-17 Thread Jun Rao
We hit a blocker (KAFKA-2841) in RC2. It's being addressed now. Will roll out RC3 once it's fixed. Thanks, Jun On Fri, Nov 13, 2015 at 6:28 PM, Jun Rao wrote: > This is the second candidate for release of Apache Kafka 0.9.0.0. This a > major release that includes (1)

[GitHub] kafka pull request: MINOR: add KStream merge operator

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/536 --- 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: MINOR: remove the group id from a restore cons...

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/543 --- 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-2852:Updating the Authorizer CLI to use ...

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/545 --- 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: MINOR: fix shutdownHook in ConsoleConsumer

2015-11-17 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/548 --- 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: MINOR: Introduce `producer.config` property to...

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

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

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] trivial fix to 0.9.0 docs -- [...truncated 1461 lines...] kafka.log.LogTest > testParseTopicPartitionNameForMissingPartition PASSED kafka.log.LogTest >

[VOTE] 0.9.0.0 Candiate 3

2015-11-17 Thread Jun Rao
This is the third 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,

Subscription Request

2015-11-17 Thread saurabh pratap singh
Hi I have recently started using kafka and I want to subscribe to mailing list . Thanks Saurabh

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

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2841; safe group metadata cache loading/unloading [junrao] trivial fix to 0.9.0 docs -- Started by an SCM change [EnvInject] - Loading node environment variables.

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

2015-11-17 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2851) system tests: error copying keytab file

2015-11-17 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15010231#comment-15010231 ] Anna Povzner commented on KAFKA-2851: - Pull request: https://github.com/apache/kafka/pull/518 >

Build failed in Jenkins: kafka_0.9.0_jdk7 #32

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2841; safe group metadata cache loading/unloading [junrao] trivial fix to 0.9.0 docs -- Started by an SCM change [EnvInject] - Loading node environment variables.

Re: Compatibility between 0.9 and 0.8.x

2015-11-17 Thread Martin Liesenberg
there are exemplary tests in the kafka repo, described in this issue: https://issues.apache.org/jira/browse/KAFKA-2845 best regards Stephan Ewen schrieb am Di., 17. Nov. 2015 um 11:13 Uhr: > Hi! > > Sorry if this is question has been asked before, but I have not found a >

Build failed in Jenkins: kafka_0.9.0_jdk7 #28

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2848; Use client SSL/SASL config utilities in Kafka Connect to -- [...truncated 2890 lines...] kafka.integration.AutoOffsetResetTest >

Compatibility between 0.9 and 0.8.x

2015-11-17 Thread Stephan Ewen
Hi! Sorry if this is question has been asked before, but I have not found a complete answer so far. I found that a 0.9.0 broker will be able to work with 0.8.x consumer clients. Does the opposite also hold? Will a new 0.9.0 consumer (new consumer API) be able to work with 0.8.x brokers? Thanks,

Re: Compatibility between 0.9 and 0.8.x

2015-11-17 Thread Stephan Ewen
Thanks, that clarifies it. Not the answer I hoped for ;-) but at least a definite answer. On Tue, Nov 17, 2015 at 11:15 AM, Martin Liesenberg < martin.liesenb...@gmail.com> wrote: > there are exemplary tests in the kafka repo, described in this issue: >

[jira] [Created] (KAFKA-2854) Make KerberosName implement PrincipalToLocal plugin so authorizer and authenticator can share this.

2015-11-17 Thread Parth Brahmbhatt (JIRA)
Parth Brahmbhatt created KAFKA-2854: --- Summary: Make KerberosName implement PrincipalToLocal plugin so authorizer and authenticator can share this. Key: KAFKA-2854 URL:

[jira] [Updated] (KAFKA-2718) Reuse of temporary directories leading to transient unit test failures

2015-11-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2718?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2718: - Affects Version/s: (was: 0.9.0.0) 0.9.1.0 > Reuse of temporary

[jira] [Updated] (KAFKA-2718) Reuse of temporary directories leading to transient unit test failures

2015-11-17 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2718?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2718: - Issue Type: Sub-task (was: Bug) Parent: KAFKA-2054 > Reuse of temporary directories

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

2015-11-17 Thread Apache Jenkins Server
See Changes: [junrao] KAFKA-2746; Add support for using ConsumerGroupCommand on secure install -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on

[jira] [Created] (KAFKA-2855) Run Multiple Ducktape Tests from the Command Line / Branch Builder

2015-11-17 Thread Ben Stopford (JIRA)
Ben Stopford created KAFKA-2855: --- Summary: Run Multiple Ducktape Tests from the Command Line / Branch Builder Key: KAFKA-2855 URL: https://issues.apache.org/jira/browse/KAFKA-2855 Project: Kafka

[jira] [Commented] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

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

[jira] [Commented] (KAFKA-2850) SslTransportLayerTest.testInvalidEndpointIdentification fails consistently

2015-11-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15009601#comment-15009601 ] Rajini Sivaram commented on KAFKA-2850: --- [~fpj] Do you mind testing the fix in the PR since I cannot

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

2015-11-17 Thread Apache Jenkins Server
See

[GitHub] kafka pull request: KAFKA-2850: Fix SSL invalid endpoint validatio...

2015-11-17 Thread rajinisivaram
GitHub user rajinisivaram opened a pull request: https://github.com/apache/kafka/pull/546 KAFKA-2850: Fix SSL invalid endpoint validation test Use invalid hostname to ensure that test works in all environments You can merge this pull request into a Git repository by running: $