[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051330#comment-15051330 ] Ismael Juma commented on KAFKA-2978: Thanks for checking Michal. > Topic partition is not sometimes

Re: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Ismael Juma
On Thu, Dec 10, 2015 at 5:24 PM, Dave Ariens wrote: > Absolutely, currently I'm hoping to get authentication working and then > ultimately work towards encryption. We're also testing performance of > more out-of-the-box Kafka components but I fear our message volume

[jira] [Commented] (KAFKA-2967) Move Kafka documentation to ReStructuredText

2015-12-10 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051372#comment-15051372 ] Gwen Shapira commented on KAFKA-2967: - Will do :) > Move Kafka documentation to ReStructuredText >

[jira] [Commented] (KAFKA-2578) Client Metadata internal state should be synchronized

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

Build failed in Jenkins: kafka_0.9.0_jdk7 #65

2015-12-10 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2578; Client Metadata internal state should be synchronized -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-4

[jira] [Commented] (KAFKA-2967) Move Kafka documentation to ReStructuredText

2015-12-10 Thread Jay Kreps (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051348#comment-15051348 ] Jay Kreps commented on KAFKA-2967: -- [~ewencp] Cool, can we see what the output would look like for

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

2015-12-10 Thread Becket Qin
Hey Jay, Thanks for the comments. Good point about the actions after when max.message.time.difference is exceeded. Rejection is a useful behavior although I cannot think of use case at LinkedIn at this moment. I think it makes sense to add a configuration. How about the following

[GitHub] kafka pull request: KAKFA-2980 Fix deadlock when ZookeeperConsumer...

2015-12-10 Thread becketqin
GitHub user becketqin opened a pull request: https://github.com/apache/kafka/pull/660 KAKFA-2980 Fix deadlock when ZookeeperConsumerConnector create messag… You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka

[GitHub] kafka pull request: KAFKA-2980 Fix deadlock when ZookeeperConsumer...

2015-12-10 Thread becketqin
Github user becketqin closed the pull request at: https://github.com/apache/kafka/pull/660 --- 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_0.9.0_jdk7 #66

2015-12-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2893: Add a simple non-negative partition seek check -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-1

[jira] [Updated] (KAFKA-2507) Replace ControlledShutdown{Request,Response} with org.apache.kafka.common.requests equivalent

2015-12-10 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2507?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2507: --- Status: In Progress (was: Patch Available) > Replace ControlledShutdown{Request,Response} with >

[jira] [Assigned] (KAFKA-2070) Replace OffsetRequest/response with ListOffsetRequest/response from org.apache.kafka.common.requests

2015-12-10 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke reassigned KAFKA-2070: -- Assignee: Grant Henke > Replace OffsetRequest/response with ListOffsetRequest/response from >

[jira] [Assigned] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson reassigned KAFKA-2978: -- Assignee: Jason Gustafson (was: Neha Narkhede) > Topic partition is not sometimes

[jira] [Updated] (KAFKA-2927) System tests: reduce storage footprint of collected logs

2015-12-10 Thread Geoff Anderson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Geoff Anderson updated KAFKA-2927: -- Reviewer: Ewen Cheslack-Postava Status: Patch Available (was: Open) > System tests:

[GitHub] kafka pull request: KAFKA-2980 Fix deadlock when ZookeeperConsumer...

2015-12-10 Thread becketqin
GitHub user becketqin reopened a pull request: https://github.com/apache/kafka/pull/660 KAFKA-2980 Fix deadlock when ZookeeperConsumerConnector create messag… You can merge this pull request into a Git repository by running: $ git pull https://github.com/becketqin/kafka

[jira] [Commented] (KAFKA-2980) ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a stream creation.

2015-12-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2980?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051512#comment-15051512 ] ASF GitHub Bot commented on KAFKA-2980: --- GitHub user becketqin reopened a pull request:

[jira] [Work started] (KAFKA-2980) ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a stream creation.

2015-12-10 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Work on KAFKA-2980 started by Jiangjie Qin. --- > ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a >

[jira] [Commented] (KAFKA-2980) ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a stream creation.

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

[GitHub] kafka pull request: MINOR: Fix typos in code comments

2015-12-10 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/662 MINOR: Fix typos in code comments You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka typo02/fix_typos_in_code_comments

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

2015-12-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2893: Add a simple non-negative partition seek check -- [...truncated 1389 lines...] kafka.log.LogTest > testIndexResizingAtTruncation PASSED kafka.log.LogTest >

[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051461#comment-15051461 ] Jason Gustafson commented on KAFKA-2978: [~tu...@avast.com] Thanks for the report. I'll assign

[jira] [Commented] (KAFKA-2507) Replace ControlledShutdown{Request,Response} with org.apache.kafka.common.requests equivalent

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

[GitHub] kafka pull request: KAFKA-2507: Replace ControlledShutdown{Request...

2015-12-10 Thread granthenke
Github user granthenke closed the pull request at: https://github.com/apache/kafka/pull/640 --- 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: test ktable state store creation

2015-12-10 Thread ymatsuda
GitHub user ymatsuda opened a pull request: https://github.com/apache/kafka/pull/661 MINOR: test ktable state store creation @guozhangwang * a test for ktable state store creation You can merge this pull request into a Git repository by running: $ git pull

[jira] [Updated] (KAFKA-2980) ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a stream creation.

2015-12-10 Thread Jiangjie Qin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jiangjie Qin updated KAFKA-2980: Status: Patch Available (was: In Progress) > ZookeeperConsumerConnector may enter deadlock if a

[jira] [Updated] (KAFKA-2929) Deprecate duplicate error mapping functionality

2015-12-10 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2929: --- Summary: Deprecate duplicate error mapping functionality (was: Remove duplicate error mapping

[GitHub] kafka pull request: KAFKA-2837: fix transient failure of kafka.api...

2015-12-10 Thread ZoneMayor
GitHub user ZoneMayor reopened a pull request: https://github.com/apache/kafka/pull/648 KAFKA-2837: fix transient failure of kafka.api.ProducerBounceTest > testBrokerFailure I can reproduced this transient failure, it seldom happen; code is like below: // rolling bounce

[GitHub] kafka pull request: KAFKA-2837: fix transient failure of kafka.api...

2015-12-10 Thread ZoneMayor
Github user ZoneMayor closed the pull request at: https://github.com/apache/kafka/pull/648 --- 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-2893: Add a simple non-negative partitio...

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

Jenkins build is back to normal : kafka-trunk-jdk8 #222

2015-12-10 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2967) Move Kafka documentation to ReStructuredText

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15050416#comment-15050416 ] Ewen Cheslack-Postava commented on KAFKA-2967: -- {quote} The big problem with the existing

[GitHub] kafka pull request: KAFKA-2070: Replace Offset{Request,Response} w...

2015-12-10 Thread granthenke
GitHub user granthenke opened a pull request: https://github.com/apache/kafka/pull/663 KAFKA-2070: Replace Offset{Request,Response} with o.a.k.c requests eq… …uivalent You can merge this pull request into a Git repository by running: $ git pull

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

2015-12-10 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2926; [MirrorMaker] InternalRebalancer calls wrong method of -- [...truncated 1447 lines...] kafka.log.BrokerCompressionTest > testBrokerSideCompression[17] PASSED

[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051764#comment-15051764 ] Jason Gustafson commented on KAFKA-2978: [~tu...@avast.com] I was able to reproduce this on the

[jira] [Resolved] (KAFKA-2926) [MirrorMaker] InternalRebalancer calls wrong method of external rebalancer

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2926?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-2926. -- Resolution: Fixed Fix Version/s: 0.9.1.0 0.9.0.1

[jira] [Commented] (KAFKA-2926) [MirrorMaker] InternalRebalancer calls wrong method of external rebalancer

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2926?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051658#comment-15051658 ] Ewen Cheslack-Postava commented on KAFKA-2926: -- [~gwenshap] 1 line down, 69,999 to go. >

[jira] [Commented] (KAFKA-2070) Replace OffsetRequest/response with ListOffsetRequest/response from org.apache.kafka.common.requests

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

[GitHub] kafka pull request: KAFKA-2926: [MirrorMaker] InternalRebalancer c...

2015-12-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/611 --- 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-2926) [MirrorMaker] InternalRebalancer calls wrong method of external rebalancer

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

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

2015-12-10 Thread Apache Jenkins Server
See Changes: [wangguoz] KAFKA-2893: Add a simple non-negative partition seek check -- [...truncated 2825 lines...] kafka.log.LogTest > testTimeBasedLogRollJitter PASSED kafka.log.LogTest >

[jira] [Updated] (KAFKA-2070) Replace OffsetRequest/response with ListOffsetRequest/response from org.apache.kafka.common.requests

2015-12-10 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2070: --- Status: Patch Available (was: Open) > Replace OffsetRequest/response with ListOffsetRequest/response

[jira] [Commented] (KAFKA-2653) Stateful operations in the KStream DSL layer

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

Jenkins build is back to normal : kafka-trunk-jdk8 #226

2015-12-10 Thread Apache Jenkins Server
See

[jira] [Updated] (KAFKA-2896) System test for partition re-assignment

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2896?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2896: - Reviewer: Ewen Cheslack-Postava > System test for partition re-assignment >

[GitHub] kafka pull request: KAFKA-2896: Added system test for partition re...

2015-12-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/655 --- 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-2896) System test for partition re-assignment

2015-12-10 Thread Anna Povzner (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2896?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Anna Povzner updated KAFKA-2896: Status: Patch Available (was: In Progress) > System test for partition re-assignment >

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

2015-12-10 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2928; system test: fix version sanity checks -- [...truncated 2798 lines...] kafka.log.LogTest > testTruncateTo PASSED kafka.log.LogTest > testCleanShutdownFile PASSED

[jira] [Commented] (KAFKA-2928) system tests: failures in version-related sanity checks

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

[jira] [Commented] (KAFKA-2896) System test for partition re-assignment

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

[jira] [Updated] (KAFKA-2896) System test for partition re-assignment

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2896?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2896: - Resolution: Fixed Fix Version/s: 0.9.1.0 Status: Resolved (was:

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

2015-12-10 Thread Apache Jenkins Server
See

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

2015-12-10 Thread Apache Jenkins Server
See

[GitHub] kafka pull request: KAFKA-2653 Phase I: Stateful Operation API Des...

2015-12-10 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/665 KAFKA-2653 Phase I: Stateful Operation API Design You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K2653r Alternatively

[GitHub] kafka pull request: KAFKA-2928: system test: fix version sanity ch...

2015-12-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/656 --- 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-2928) system tests: failures in version-related sanity checks

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2928: - Resolution: Fixed Fix Version/s: 0.9.1.0 Status: Resolved (was:

[jira] [Comment Edited] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051920#comment-15051920 ] Jason Gustafson edited comment on KAFKA-2978 at 12/11/15 12:42 AM: --- I

[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051920#comment-15051920 ] Jason Gustafson commented on KAFKA-2978: I think I see what's going on and the problem will affect

[jira] [Updated] (KAFKA-2929) Deprecate duplicate error mapping functionality

2015-12-10 Thread Grant Henke (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2929?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Grant Henke updated KAFKA-2929: --- Description: Kafka common and core both have a class that maps error codes and exceptions. To

RE: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Dave Ariens
Absolutely, currently I'm hoping to get authentication working and then ultimately work towards encryption. We're also testing performance of more out-of-the-box Kafka components but I fear our message volume will require us to maintain our custom producers and consumers.

Re: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Flavio Junqueira
Hi Dave, I apologize for the obvious question, but have you had a look at the documentation: http://kafka.apache.org/documentation.html#security It is possible that you're not aware that it is there, so I'm just confirming. But, if you

RE: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Dave Ariens
Yes, I have read through the documentation and I am attempting SASL authentication with Kerberos.I don't blame anyone for assuming I hadn't :) I have my keytab generated and I am launching with my client with a JAAS configuration accordingly. The missing piece for me was I was attempting

[jira] [Created] (KAFKA-2980) ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a stream creation.

2015-12-10 Thread Jiangjie Qin (JIRA)
Jiangjie Qin created KAFKA-2980: --- Summary: ZookeeperConsumerConnector may enter deadlock if a rebalance occurs during a stream creation. Key: KAFKA-2980 URL: https://issues.apache.org/jira/browse/KAFKA-2980

[jira] [Commented] (KAFKA-2893) Add Negative Partition Seek Check

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

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

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

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

2015-12-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051285#comment-15051285 ] ASF GitHub Bot commented on KAFKA-2837: --- GitHub user ZoneMayor reopened a pull request:

[jira] [Assigned] (KAFKA-2967) Move Kafka documentation to ReStructuredText

2015-12-10 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2967?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gwen Shapira reassigned KAFKA-2967: --- Assignee: Gwen Shapira > Move Kafka documentation to ReStructuredText >

RE: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Dave Ariens
> Interested to find out if the new producer and consumer fare > better Regarding that, what's the best way to use the new producer with as little overhead as possible? import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; producer =

Re: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Ismael Juma
On Thu, Dec 10, 2015 at 6:06 PM, Dave Ariens wrote: > Sure, the custom producer and consumer library I'm updating is Krackle: > https://github.com/blackberry/Krackle > ... > We found that neither the Kafka proper producers nor Kafka could keep up > with the stock packages

[jira] [Resolved] (KAFKA-2893) Add Negative Partition Seek Check

2015-12-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2893?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-2893. -- Resolution: Fixed Fix Version/s: 0.9.0.1 Issue resolved by pull request 628

[jira] [Updated] (KAFKA-2578) Client Metadata internal state should be synchronized

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2578?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2578: - Resolution: Fixed Fix Version/s: 0.9.1.0 0.9.0.1

[GitHub] kafka pull request: KAFKA-2578; Client Metadata internal state sho...

2015-12-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/659 --- 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-2927) System tests: reduce storage footprint of collected logs

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

[jira] [Updated] (KAFKA-2927) System tests: reduce storage footprint of collected logs

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2927?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2927: - Resolution: Fixed Fix Version/s: 0.9.1.0 Status: Resolved (was:

[GitHub] kafka pull request: KAFKA-2927: reduce system test storage footpri...

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

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

2015-12-10 Thread Apache Jenkins Server
See

[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15052087#comment-15052087 ] Guozhang Wang commented on KAFKA-2978: -- Thanks for the investigation, this makes sense. > Topic

[jira] [Created] (KAFKA-2979) Enable authorizer and ACLs in ducktape tests

2015-12-10 Thread Flavio Junqueira (JIRA)
Flavio Junqueira created KAFKA-2979: --- Summary: Enable authorizer and ACLs in ducktape tests Key: KAFKA-2979 URL: https://issues.apache.org/jira/browse/KAFKA-2979 Project: Kafka Issue Type:

[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Michal Turek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051313#comment-15051313 ] Michal Turek commented on KAFKA-2978: - Ismael, I afraid I have reproduced the issue using code from

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

2015-12-10 Thread Allen Wang
If there are no more comments I would like to call for a vote. On Sun, Nov 15, 2015 at 10:08 PM, Allen Wang wrote: > KIP is updated with more details and how to handle the situation where > rack information is incomplete. > > In the situation where rack information is

RE: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Dave Ariens
Sure, the custom producer and consumer library I'm updating is Krackle: https://github.com/blackberry/Krackle It's a very limited low overhead library that we're using to get syslog style messages from the Blackberry service provider infrastructure into Kafka. We have dozens of pools of

Jenkins build is back to normal : kafka-trunk-jdk8 #230

2015-12-10 Thread Apache Jenkins Server
See

RE: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Andrew Schofield
Wouldn't you use TLS to secure the connections? Encrypting just the credentials but not the connection seems brave. Andrew From: Dave Ariens To: "dev@kafka.apache.org" Date: 10/12/2015 15:43 Subject:RE: Interacting with a secured

[jira] [Created] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Michal Turek (JIRA)
Michal Turek created KAFKA-2978: --- Summary: Topic partition is not sometimes consumed after rebalancing of consumer group Key: KAFKA-2978 URL: https://issues.apache.org/jira/browse/KAFKA-2978 Project:

[jira] [Updated] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Michal Turek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Turek updated KAFKA-2978: Description: Hi there, we are evaluating Kafka 0.9 to find if it is stable enough and ready for

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

2015-12-10 Thread Apache Jenkins Server
See Changes: [me] KAFKA-2927; reduce system test storage footprint -- [...truncated 2869 lines...] kafka.log.LogTest > testTruncateTo PASSED kafka.log.LogTest > testCleanShutdownFile PASSED

[GitHub] kafka pull request: MINOR: test ktable state store creation

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

Windows support

2015-12-10 Thread PATTERSON Tony
Hello, My group produces system that have to run on both Linux and Windows. We're very interested in Kafka but are inhibited by the lack of windows support. I notice in your documentation you state, "We have seen a few issues running on Windows and Windows is not currently a well supported

[jira] [Commented] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

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

[GitHub] kafka pull request: KAFKA-2978: consumer stops fetching when consu...

2015-12-10 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/666 KAFKA-2978: consumer stops fetching when consumed and fetch positions get out of sync You can merge this pull request into a Git repository by running: $ git pull

[GitHub] kafka pull request: MINOR: Change SaslSetup workDir to be under th...

2015-12-10 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/664 --- 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-jdk8 #229

2015-12-10 Thread Apache Jenkins Server
See Changes: [me] MINOR: Change SaslSetup workDir to be under the build folder -- Started by an SCM change [EnvInject] - Loading node environment variables. Building remotely on ubuntu-2 (docker

[jira] [Updated] (KAFKA-2070) Replace OffsetRequest/response with ListOffsetRequest/response from org.apache.kafka.common.requests

2015-12-10 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2070: - Reviewer: Ewen Cheslack-Postava > Replace OffsetRequest/response with

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

2015-12-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2837?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15051021#comment-15051021 ] ASF GitHub Bot commented on KAFKA-2837: --- GitHub user ZoneMayor reopened a pull request:

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

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

[GitHub] kafka pull request: KAFKA-2837: fix transient failure of kafka.api...

2015-12-10 Thread ZoneMayor
Github user ZoneMayor closed the pull request at: https://github.com/apache/kafka/pull/648 --- 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-2837: fix transient failure of kafka.api...

2015-12-10 Thread ZoneMayor
GitHub user ZoneMayor reopened a pull request: https://github.com/apache/kafka/pull/648 KAFKA-2837: fix transient failure of kafka.api.ProducerBounceTest > testBrokerFailure I can reproduced this transient failure, it seldom happen; code is like below: // rolling bounce

Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Dave Ariens
Hi devs! I'm working on enhancing a custom 0.8.2.1 producer/consumer to support establishing connections a secured 0.9.0 cluster with strict ACLs on each topic. I'm pretty new to (read: first day working with) GSS-API/JAAS and not really sure how to approach this problem. Our existing

Re: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Ismael Juma
Hi Dave, Is there a reason why you are using GSS-API directly instead of via SASL? It should still work, but if you do the latter, you can potentially reuse the existing code (or at least use it as inspiration), see `org.apache.kafka.common.security.authenticator.SaslClientAuthenticator`. Also,

RE: Interacting with a secured Kafka cluster via GSS-API

2015-12-10 Thread Dave Ariens
> Is there a reason why you are using GSS-API directly instead of via SASL? There sure is--because I have no clue what I'm doing :) Our Kafka 0.9.0 cluster is currently only configured for SASL_PLAINTEXT so we're not encrypting anything at the moment. I'll take a look through

[jira] [Updated] (KAFKA-2978) Topic partition is not sometimes consumed after rebalancing of consumer group

2015-12-10 Thread Michal Turek (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2978?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michal Turek updated KAFKA-2978: Fix Version/s: 0.9.0.1 > Topic partition is not sometimes consumed after rebalancing of consumer

  1   2   >