[jira] [Commented] (KAFKA-4254) Questionable handling of unknown partitions in KafkaProducer

2016-10-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4254?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15547518#comment-15547518 ] Ismael Juma commented on KAFKA-4254: Do we really want to have more cases where we fai

[jira] [Created] (KAFKA-4254) Questionable handling of unknown partitions in KafkaProducer

2016-10-04 Thread Jason Gustafson (JIRA)
Jason Gustafson created KAFKA-4254: -- Summary: Questionable handling of unknown partitions in KafkaProducer Key: KAFKA-4254 URL: https://issues.apache.org/jira/browse/KAFKA-4254 Project: Kafka

[jira] [Updated] (KAFKA-3838) Bump zkclient and Zookeeper versions

2016-10-04 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3838?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-3838: --- Fix Version/s: 0.10.1.0 > Bump zkclient and Zookeeper versions >

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

2016-10-04 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4251: fix test driver not launching in Vagrant 1.8.6 -- [...truncated 2421 lines...] kafka.api.AdminClientTest > testListGroups STARTED kafka.api.AdminClientTest > testL

Re: Snazzy new look to our website

2016-10-04 Thread Jason Gustafson
Huge improvement. Thanks Derrick and Gwen! On Tue, Oct 4, 2016 at 5:54 PM, Becket Qin wrote: > Much fancier now :) > > On Tue, Oct 4, 2016 at 5:51 PM, Ali Akhtar wrote: > > > Just noticed this on pulling up the documentation. Oh yeah! This new look > > is fantastic. > > > > On Wed, Oct 5, 2016

Re: [DISCUSS] Fault injection tests for Kafka

2016-10-04 Thread radai
for "small" failures (local failures on a single node, like socket disconnection, disk read errors, out of memory etc) I've used byteman before - http://byteman.jboss.org/ On Tue, Oct 4, 2016 at 5:46 PM, Joel Koshy wrote: > Hi Gwen, > > I've also seen suggestions of using Jepsen for fault inject

Re: Snazzy new look to our website

2016-10-04 Thread Joel Koshy
Looks great! On Tue, Oct 4, 2016 at 4:38 PM, Guozhang Wang wrote: > The new look is great, thanks Derrick and Gwen! > > I'm wondering if we should still consider breaking "document.html" into > multiple pages and indexed as sub-topics on the left bar? > > > Guozhang > > > On Tue, Oct 4, 2016 at

Re: [DISCUSS] Fault injection tests for Kafka

2016-10-04 Thread Joel Koshy
Hi Gwen, I've also seen suggestions of using Jepsen for fault injection, but > I'm not familiar with this framework. > > What do you guys think? Write our own failure injection? or write > Kafka tests in Jepsen? > This would definitely add a lot of value and save a lot on release validation overh

Jenkins build is back to normal : kafka-0.10.1-jdk7 #46

2016-10-04 Thread Apache Jenkins Server
See

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

2016-10-04 Thread Apache Jenkins Server
See Changes: [me] KAFKA-4251: fix test driver not launching in Vagrant 1.8.6 -- [...truncated 14049 lines...] org.apache.kafka.streams.kstream.internals.KeyValuePrinterProcessorTest > testPrintK

[jira] [Commented] (KAFKA-3559) Task creation time taking too long in rebalance callback

2016-10-04 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15547089#comment-15547089 ] Guozhang Wang commented on KAFKA-3559: -- Here are some more thoughts on this issue and

Re: Snazzy new look to our website

2016-10-04 Thread Guozhang Wang
The new look is great, thanks Derrick and Gwen! I'm wondering if we should still consider breaking "document.html" into multiple pages and indexed as sub-topics on the left bar? Guozhang On Tue, Oct 4, 2016 at 4:13 PM, Gwen Shapira wrote: > Hi Team Kafka, > > I just merged PR 20 to our websi

Re: Snazzy new look to our website

2016-10-04 Thread Vahid S Hashemian
+1 Thank you for the much needed new design. At first glance, it looks great, and more professional. --Vahid From: Gwen Shapira To: dev@kafka.apache.org, Users Cc: Derrick Or Date: 10/04/2016 04:13 PM Subject:Snazzy new look to our website Hi Team Kafka, I just merg

Re: Streams support for Serdes

2016-10-04 Thread Hojjat Jafarpour
Hi Jeyhun, You are right. As long as the types in Tuple-n are known types(e.g., primitives) and the serdes for them are available you can remove the need for providing serdes and infer them. Indeed we have a project that we use similar approach to eliminate the need to have user to provide the ser

Snazzy new look to our website

2016-10-04 Thread Gwen Shapira
Hi Team Kafka, I just merged PR 20 to our website - which gives it a new (and IMO pretty snazzy) look and feel. Thanks to Derrick Or for contributing the update. I had to do a hard-refresh (shift-f5 on my mac) to get the new look to load properly - so if stuff looks off, try it. Comments and con

Re: [VOTE] 0.10.1.0 RC0

2016-10-04 Thread Jason Gustafson
One clarification: this is a minor release, not a major one. -Jason On Tue, Oct 4, 2016 at 4:01 PM, Jason Gustafson wrote: > Hello Kafka users, developers and client-developers, > > This is the first candidate for release of Apache Kafka 0.10.1.0. This is > a major release that includes great n

[jira] [Commented] (KAFKA-4244) Update our website look & feel

2016-10-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546983#comment-15546983 ] Gwen Shapira commented on KAFKA-4244: - #3 is done - merged PR 20 to the site with the

[VOTE] 0.10.1.0 RC0

2016-10-04 Thread Jason Gustafson
Hello Kafka users, developers and client-developers, This is the first candidate for release of Apache Kafka 0.10.1.0. This is a major release that includes great new features including throttled replication, secure quotas, time-based log searching, and queryable state for Kafka Streams. A full li

[GitHub] kafka-site pull request #20: new design

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

[jira] [Commented] (KAFKA-3182) Failure in kafka.network.SocketServerTest.testSocketsCloseOnShutdown

2016-10-04 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546925#comment-15546925 ] Vahid Hashemian commented on KAFKA-3182: Another incident: https://builds.apache.

[GitHub] kafka-site issue #20: new design

2016-10-04 Thread gwenshap
Github user gwenshap commented on the issue: https://github.com/apache/kafka-site/pull/20 LGTM --- 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 fe

Build failed in Jenkins: kafka-0.10.1-jdk7 #45

2016-10-04 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4165; Add 0.10.0.1 as a source for compatibility tests -- [...truncated 6285 lines...] kafka.api.AuthorizerIntegrationTest > testPatternSubscriptionWithNoTopicAccess

Re: [UPDATE] 0.10.1 Release Progress

2016-10-04 Thread Becket Qin
Thanks, Jason! On Tue, Oct 4, 2016 at 1:57 PM, Guozhang Wang wrote: > Thanks for running the release Jason! > > On Mon, Oct 3, 2016 at 9:06 PM, Jason Gustafson > wrote: > > > Hi Everyone, > > > > The code freeze is upon us! We've made incredible progress fixing bugs > and > > improving testing.

[GitHub] kafka pull request #1969: MINOR: missing fullstop in doc for `max.partition....

2016-10-04 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/1969 MINOR: missing fullstop in doc for `max.partition.fetch.bytes` You can merge this pull request into a Git repository by running: $ git pull https://github.com/shikhar/kafka patch-2 Alternativ

[GitHub] kafka pull request #1968: MINOR: missing whitespace in doc for `ssl.cipher.s...

2016-10-04 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/1968 MINOR: missing whitespace in doc for `ssl.cipher.suites` You can merge this pull request into a Git repository by running: $ git pull https://github.com/shikhar/kafka patch-1 Alternatively yo

Re: [DISCUSS] KIP-82 - Add Record Headers

2016-10-04 Thread radai
another potential benefit of headers is it would reduce the number of API changes required to support future features (as they could be implemented as plugins). that would greatly accelerate the rate with which kafka can be extended. On Mon, Oct 3, 2016 at 12:46 PM, Michael Pearce wrote: > Oppos

[jira] [Commented] (KAFKA-4244) Update our website look & feel

2016-10-04 Thread Gwen Shapira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546873#comment-15546873 ] Gwen Shapira commented on KAFKA-4244: - Update: Change #1 is pending review and change

[jira] [Updated] (KAFKA-4253) Fix Kafka Stream thread shutting down process ordering

2016-10-04 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4253?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-4253: - Assignee: (was: Matthias J. Sax) > Fix Kafka Stream thread shutting down process ordering > --

[jira] [Created] (KAFKA-4253) Fix Kafka Stream thread shutting down process ordering

2016-10-04 Thread Guozhang Wang (JIRA)
Guozhang Wang created KAFKA-4253: Summary: Fix Kafka Stream thread shutting down process ordering Key: KAFKA-4253 URL: https://issues.apache.org/jira/browse/KAFKA-4253 Project: Kafka Issue Ty

[jira] [Commented] (KAFKA-4244) Update our website look & feel

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

[GitHub] kafka pull request #1967: cherry-picking KAFKA-4244 to 0.10.1.0 branch

2016-10-04 Thread gwenshap
GitHub user gwenshap opened a pull request: https://github.com/apache/kafka/pull/1967 cherry-picking KAFKA-4244 to 0.10.1.0 branch fixing few minor conflicts You can merge this pull request into a Git repository by running: $ git pull https://github.com/gwenshap/kafka KAFKA-424

[jira] [Commented] (KAFKA-4244) Update our website look & feel

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

[jira] [Resolved] (KAFKA-4251) Test driver not launching in Vagrant 1.8.6

2016-10-04 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4251?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava resolved KAFKA-4251. -- Resolution: Fixed Fix Version/s: 0.10.2.0 0.10.1.0 Iss

[jira] [Commented] (KAFKA-4251) Test driver not launching in Vagrant 1.8.6

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

[GitHub] kafka pull request #1966: KAFKA-4244: fixing formating issues in docs. missi...

2016-10-04 Thread gwenshap
GitHub user gwenshap opened a pull request: https://github.com/apache/kafka/pull/1966 KAFKA-4244: fixing formating issues in docs. missing headers and lots of paragrap… …h misformatting You can merge this pull request into a Git repository by running: $ git pull https://gi

[GitHub] kafka pull request #1962: KAFKA-4251: fix test driver not launching in Vagra...

2016-10-04 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1962 --- 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 enab

[jira] [Commented] (KAFKA-4176) Node stopped receiving heartbeat responses once another node started within the same group

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

[GitHub] kafka pull request #1965: KAFKA-4176: Only call printStream.flush for System...

2016-10-04 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/1965 KAFKA-4176: Only call printStream.flush for System.out You can merge this pull request into a Git repository by running: $ git pull https://github.com/guozhangwang/kafka K4176-only-flush

Re: [DISCUSS] KIP-80: Kafka REST Server

2016-10-04 Thread Edoardo Comar
Harsha thanks for opening the discussion on this KIP. While I understand he founding members' stand that the Kafka project can not expand its surface to a large number of clients, I strongly agree with your well explained points below and support your KIP. A REST API is not just on the same lev

[jira] [Commented] (KAFKA-4010) ConfigDef.toRst() should create sections for each group

2016-10-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4010?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546720#comment-15546720 ] ASF GitHub Bot commented on KAFKA-4010: --- GitHub user shikhar opened a pull request:

[GitHub] kafka pull request #1964: KAFKA-4010: add ConfigDef toEnrichedRst() for addi...

2016-10-04 Thread shikhar
GitHub user shikhar opened a pull request: https://github.com/apache/kafka/pull/1964 KAFKA-4010: add ConfigDef toEnrichedRst() for additional fields in output followup on https://github.com/apache/kafka/pull/1696 cc @rekhajoshm You can merge this pull request into a Git re

[jira] [Commented] (KAFKA-3985) Transient system test failure ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade.security_protocol

2016-10-04 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546706#comment-15546706 ] Rajini Sivaram commented on KAFKA-3985: --- [~fpj] Would it be possible to attach the f

[jira] [Commented] (KAFKA-4176) Node stopped receiving heartbeat responses once another node started within the same group

2016-10-04 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4176?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546686#comment-15546686 ] Guozhang Wang commented on KAFKA-4176: -- [~MarekSvitok] Thanks for filing this issue.

[GitHub] kafka pull request #1963: MINOR: Update documentation for 0.10.1 release

2016-10-04 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1963 MINOR: Update documentation for 0.10.1 release You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka update-docs-for-0.10.1 Alterna

Re: [UPDATE] 0.10.1 Release Progress

2016-10-04 Thread Guozhang Wang
Thanks for running the release Jason! On Mon, Oct 3, 2016 at 9:06 PM, Jason Gustafson wrote: > Hi Everyone, > > The code freeze is upon us! We've made incredible progress fixing bugs and > improving testing. If your feature or bug fix didn't get in this time, > don't worry since the next release

[jira] [Updated] (KAFKA-4252) Missing ProducerRequestPurgatory

2016-10-04 Thread Narendra Bidari (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4252?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Narendra Bidari updated KAFKA-4252: --- Attachment: Screen Shot 2016-10-04 at 1.34.00 PM.png > Missing ProducerRequestPurgatory >

[jira] [Created] (KAFKA-4252) Missing ProducerRequestPurgatory

2016-10-04 Thread Narendra Bidari (JIRA)
Narendra Bidari created KAFKA-4252: -- Summary: Missing ProducerRequestPurgatory Key: KAFKA-4252 URL: https://issues.apache.org/jira/browse/KAFKA-4252 Project: Kafka Issue Type: Bug

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

2016-10-04 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Tweak upgrade note on KIP-62 to include request.timeout.ms [jason] KAFKA-4165; Add 0.10.0.1 as a source for compatibility tests -- [...truncated 14050 lines...] org.a

Build failed in Jenkins: kafka-0.10.1-jdk7 #44

2016-10-04 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Tweak upgrade note on KIP-62 to include request.timeout.ms -- [...truncated 3814 lines...] kafka.coordinator.GroupMetadataManagerTest > testExpireGroup STARTED kafka

[jira] [Commented] (KAFKA-4251) Test driver not launching in Vagrant 1.8.6

2016-10-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4251?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546304#comment-15546304 ] ASF GitHub Bot commented on KAFKA-4251: --- GitHub user xvrl opened a pull request:

[GitHub] kafka pull request #1962: KAFKA-4251: fix test driver not launching in Vagra...

2016-10-04 Thread xvrl
GitHub user xvrl opened a pull request: https://github.com/apache/kafka/pull/1962 KAFKA-4251: fix test driver not launching in Vagrant 1.8.6 custom ip resolver in test driver makes incorrect assumption when calling vm.communicate.execute, causing driver to fail launching with Vagran

[jira] [Created] (KAFKA-4251) Test driver not launching in Vagrant 1.8.6

2016-10-04 Thread JIRA
Xavier Léauté created KAFKA-4251: Summary: Test driver not launching in Vagrant 1.8.6 Key: KAFKA-4251 URL: https://issues.apache.org/jira/browse/KAFKA-4251 Project: Kafka Issue Type: Bug

[jira] [Comment Edited] (KAFKA-4246) Discretionary partition assignment on the consumer side not functional

2016-10-04 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546229#comment-15546229 ] Vahid Hashemian edited comment on KAFKA-4246 at 10/4/16 6:29 PM: ---

[jira] [Commented] (KAFKA-4246) Discretionary partition assignment on the consumer side not functional

2016-10-04 Thread Vahid Hashemian (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546229#comment-15546229 ] Vahid Hashemian commented on KAFKA-4246: I'll try to run this in my environment to

[jira] [Commented] (KAFKA-4250) make ProducerRecord and ConsumerRecord extensible

2016-10-04 Thread radai rosenblatt (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4250?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15546226#comment-15546226 ] radai rosenblatt commented on KAFKA-4250: - PR up - https://github.com/apache/kafka

[jira] [Created] (KAFKA-4250) make ProducerRecord and ConsumerRecord extensible

2016-10-04 Thread radai rosenblatt (JIRA)
radai rosenblatt created KAFKA-4250: --- Summary: make ProducerRecord and ConsumerRecord extensible Key: KAFKA-4250 URL: https://issues.apache.org/jira/browse/KAFKA-4250 Project: Kafka Issue T

[jira] [Commented] (KAFKA-4165) Add 0.10.0.1 as a source for compatibility tests where relevant

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

[jira] [Resolved] (KAFKA-4165) Add 0.10.0.1 as a source for compatibility tests where relevant

2016-10-04 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4165?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-4165. Resolution: Fixed Issue resolved by pull request 1959 [https://github.com/apache/kafka/pull/

[GitHub] kafka pull request #1959: KAFKA-4165: Add 0.10.0.1 as a source for compatibi...

2016-10-04 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1959 --- 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 enab

[GitHub] kafka pull request #1960: MINOR: Tweak upgrade note on KIP-62 to include req...

2016-10-04 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/1960 --- 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 enab

[GitHub] kafka pull request #1961: make ProducerRecord and ConsumerRecord extensible

2016-10-04 Thread radai-rosenblatt
GitHub user radai-rosenblatt opened a pull request: https://github.com/apache/kafka/pull/1961 make ProducerRecord and ConsumerRecord extensible Signed-off-by: radai-rosenblatt You can merge this pull request into a Git repository by running: $ git pull https://github.com/radai

[GitHub] kafka pull request #1960: MINOR: Tweak upgrade note on KIP-62 to include req...

2016-10-04 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1960 MINOR: Tweak upgrade note on KIP-62 to include request.timeout.ms You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka add-note-on-

[GitHub] kafka pull request #1959: KAFKA-4165: Add 0.10.0.1 as a source for compatibi...

2016-10-04 Thread hachikuji
GitHub user hachikuji opened a pull request: https://github.com/apache/kafka/pull/1959 KAFKA-4165: Add 0.10.0.1 as a source for compatibility tests You can merge this pull request into a Git repository by running: $ git pull https://github.com/hachikuji/kafka KAFKA-4165 Alter

[jira] [Commented] (KAFKA-4165) Add 0.10.0.1 as a source for compatibility tests where relevant

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

[jira] [Commented] (KAFKA-3985) Transient system test failure ZooKeeperSecurityUpgradeTest.test_zk_security_upgrade.security_protocol

2016-10-04 Thread Flavio Junqueira (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15545834#comment-15545834 ] Flavio Junqueira commented on KAFKA-3985: - We have been observing in our system te

[jira] [Created] (KAFKA-4249) Document how to customize GC logging options for broker

2016-10-04 Thread Jim Hoagland (JIRA)
Jim Hoagland created KAFKA-4249: --- Summary: Document how to customize GC logging options for broker Key: KAFKA-4249 URL: https://issues.apache.org/jira/browse/KAFKA-4249 Project: Kafka Issue Typ

[DISCUSS] KIP-84: Support SASL/SCRAM mechanisms

2016-10-04 Thread Rajini Sivaram
Hi all, I have just created KIP-84 to add SCRAM-SHA-1 and SCRAM-SHA-256 SASL mechanisms to Kafka: https://cwiki.apache.org/confluence/display/KAFKA/KIP-84%3A+Support+SASL+SCRAM+mechanisms Comments and suggestions are welcome. Thank you... Regards, Rajini

[jira] [Commented] (KAFKA-4246) Discretionary partition assignment on the consumer side not functional

2016-10-04 Thread Alexandru Ionita (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4246?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15545134#comment-15545134 ] Alexandru Ionita commented on KAFKA-4246: - There's nothing special about the consu

[GitHub] kafka pull request #1958: MINOR: trivial doc cleanup

2016-10-04 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/1958 MINOR: trivial doc cleanup You can merge this pull request into a Git repository by running: $ git pull https://github.com/omkreddy/kafka MINOR-DOC-CHAGE Alternatively you can review and app

[jira] [Commented] (KAFKA-4074) Deleting a topic can make it unavailable even if delete.topic.enable is false

2016-10-04 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4074?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15545035#comment-15545035 ] ASF GitHub Bot commented on KAFKA-4074: --- Github user omkreddy closed the pull reques

[GitHub] kafka pull request #1784: KAFKA-4074: Deleting a topic can make it unavailab...

2016-10-04 Thread omkreddy
Github user omkreddy closed the pull request at: https://github.com/apache/kafka/pull/1784 --- 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 en

[jira] [Commented] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2016-10-04 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15545015#comment-15545015 ] Eno Thereska commented on KAFKA-4222: - This should now be fixed. > Transient failure

[jira] [Updated] (KAFKA-1843) Metadata fetch/refresh in new producer should handle all node connection states gracefully

2016-10-04 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1843?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska updated KAFKA-1843: Assignee: (was: Eno Thereska) > Metadata fetch/refresh in new producer should handle all node co

[jira] [Resolved] (KAFKA-4222) Transient failure in QueryableStateIntegrationTest.queryOnRebalance

2016-10-04 Thread Eno Thereska (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4222?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eno Thereska resolved KAFKA-4222. - Resolution: Fixed > Transient failure in QueryableStateIntegrationTest.queryOnRebalance >

[GitHub] kafka pull request #1957: MINOR: Add Replication Quotas Test Rig

2016-10-04 Thread benstopford
GitHub user benstopford opened a pull request: https://github.com/apache/kafka/pull/1957 MINOR: Add Replication Quotas Test Rig This test rig lives in the other.kafka package so isn't part of our standard tests. It provides a convenient mechanism for measuring throttling performanc

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

2016-10-04 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-4019; Update log cleaner to handle max message size of topics [jason] MINOR: Add upgrade notes for KIP-62 -- [...truncated 7687 lines...] kafka.server.HighwatermarkPer

Build failed in Jenkins: kafka-0.10.1-jdk7 #43

2016-10-04 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Add upgrade notes for KIP-62 -- [...truncated 4669 lines...] kafka.api.AuthorizerIntegrationTest > testSimpleConsumeWithOffsetLookupAndNoGroupAccess PASSED kafka.api

[jira] [Assigned] (KAFKA-4195) support throttling on request rate

2016-10-04 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4195?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram reassigned KAFKA-4195: - Assignee: Rajini Sivaram > support throttling on request rate > -

[jira] [Comment Edited] (KAFKA-4176) Node stopped receiving heartbeat responses once another node started within the same group

2016-10-04 Thread Marek Svitok (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4176?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15544601#comment-15544601 ] Marek Svitok edited comment on KAFKA-4176 at 10/4/16 7:22 AM: --

[jira] [Commented] (KAFKA-4176) Node stopped receiving heartbeat responses once another node started within the same group

2016-10-04 Thread Marek Svitok (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4176?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15544601#comment-15544601 ] Marek Svitok commented on KAFKA-4176: - 1. No 2. The second node starts / rebalance ver

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

2016-10-04 Thread Apache Jenkins Server
See