Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-18 Thread Apurva Mehta
Thanks Jason and Ismael. The message format problem is an acute one: if we enable idempotence by default, the UnsupportedVersionException when writing to topics with the older message format would mean that our prescribed upgrade steps would not work. I have detailed the problems and the

Re: Making sure all of you know about Kafka Summit

2017-08-18 Thread M. Manna
I guess It's kinda late since I am already in transit for work. Is there any plan to do something in Europe e.g. London or some other place? On 18 Aug 2017 4:41 pm, "Gwen Shapira" wrote: > Hi, > > I figured everyone in this list kinda cares about Kafka, so just making > sure

Making sure all of you know about Kafka Summit

2017-08-18 Thread Gwen Shapira
Hi, I figured everyone in this list kinda cares about Kafka, so just making sure you all know. Kafka Summit SF happens in about a week: https://kafka-summit.org/events/kafka-summit-sf/ August 28 in San Francisco. It is not too late to register. The talks are pretty great (and very relevant to

[GitHub] kafka pull request #3694: MINOR: Improve help doc of ConsumerGroupCommand

2017-08-18 Thread vahidhashemian
GitHub user vahidhashemian opened a pull request: https://github.com/apache/kafka/pull/3694 MINOR: Improve help doc of ConsumerGroupCommand You can merge this pull request into a Git repository by running: $ git pull https://github.com/vahidhashemian/kafka

[DISCUSS] KIP-190 Asynchronous Java Kafka Producer

2017-08-18 Thread Pavel Moukhataev
This KIP is to make java kafka producer client 100% async. This is very important for applications with low latency requirements. https://cwiki.apache.org/confluence/display/ARIES/KIP-190+-+ Make+java+kafka+producer+asynchronous But I put it into ARIES space because I don't have access rights for

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

2017-08-18 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5725; More failure testing -- [...truncated 2.57 MB...] org.apache.kafka.connect.runtime.distributed.DistributedHerderTest >

Re: Different Data Types under same topic

2017-08-18 Thread SenthilKumar K
+ dev experts for inputs. --Senthil On Fri, Aug 18, 2017 at 9:15 PM, SenthilKumar K wrote: > Hi Users , We have planned to use Kafka for one of the use to collect data > from different server and persist into Message Bus .. > > Flow Would Be : > Source --> Kafka -->

Re: [VOTE] KIP-180: Adding a new metric for brokers specifying the number of consumer group rebalances in progress

2017-08-18 Thread Colin McCabe
With +1s from Bill Bejeck, Apurva Mehta, Mickael Maison, Gwen Shapira, Ismael Juma, Guozhang Wang, and Jason Gustafson, and no +0 or -1s, the vote passes. cheers, Colin On Wed, Aug 16, 2017, at 13:08, Jason Gustafson wrote: > +1 > > On Sun, Aug 13, 2017 at 5:15 PM, Guozhang Wang

Re: [DISCUSS] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-18 Thread Jun Rao
Hi, Rajini, Thanks for the KIP. Looks good overall. Just a couple of minor comments. 1. "The final message from the broker will indicate if authentication succeeded or failed." Are we doing something special for the final message? I thought each token response will indicate success or failure in

Re: [DISCUSS] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-18 Thread Ismael Juma
Thanks Rajini. The changes look good. We have to update the motivation to take into account the improvements in 0.11: "With the current Kafka SASL implementation, broker closes the client connection if SASL authentication fails. Clients see this as a connection failure and do not get any feedback

[GitHub] kafka pull request #3693: KAFKA-2254: Use double quote to prevent globbing a...

2017-08-18 Thread omkreddy
GitHub user omkreddy opened a pull request: https://github.com/apache/kafka/pull/3693 KAFKA-2254: Use double quote to prevent globbing and word splitting in shell scripts You can merge this pull request into a Git repository by running: $ git pull

[jira] [Created] (KAFKA-5751) Kafka cannot start; corrupted index file(s)

2017-08-18 Thread Martin M (JIRA)
Martin M created KAFKA-5751: --- Summary: Kafka cannot start; corrupted index file(s) Key: KAFKA-5751 URL: https://issues.apache.org/jira/browse/KAFKA-5751 Project: Kafka Issue Type: Bug

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

2017-08-18 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5725; More failure testing -- Started by an SCM change Started by an SCM change [EnvInject] - Loading node environment variables. Building

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

2017-08-18 Thread Apache Jenkins Server
See

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

2017-08-18 Thread Apache Jenkins Server
See Changes: [damian.guy] KAFKA-5668; fetch across stores in CompositeReadOnlyWindowStore & -- [...truncated 914.26 KB...] kafka.integration.AutoOffsetResetTest >

[jira] [Resolved] (KAFKA-5725) Additional failure testing for streams with bouncing brokers

2017-08-18 Thread Damian Guy (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Damian Guy resolved KAFKA-5725. --- Resolution: Fixed > Additional failure testing for streams with bouncing brokers >

[GitHub] kafka pull request #3656: KAFKA-5725: More failure testing

2017-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3656 --- 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 #3685: KAFKA-5668: fetch across stores in CompositeReadOn...

2017-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3685 --- 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 #1920

2017-08-18 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5745; makeLeader should invoke `convertHWToLocalOffsetMetadata` -- [...truncated 4.28 MB...] kafka.integration.AutoOffsetResetTest >

Re: [DISCUSS] KIP-152 - Improve diagnostics for SASL authentication failures

2017-08-18 Thread Vahid S Hashemian
Hi Rajini, Thanks for the KIP. It looks good to me. It would be great if it can make it to 1.0.0. --Vahid From: Rajini Sivaram To: dev Date: 08/15/2017 01:23 PM Subject:Re: [DISCUSS] KIP-152 - Improve diagnostics for SASL

1.0.0 Release Update

2017-08-18 Thread Guozhang Wang
Hello folks, Just want to give you a quick heads up on 1.0.0. https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=71764913 * KIPs: we have a total of 19 KIPs accepted so far, and 10 of them have already merged to trunk. For those KIP contributors, just a kind reminder: please

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-18 Thread Jason Gustafson
> > so this change will break client backward compatibility while connecting > to 0.10.X brokers. > users need to change producer default settings while connecting older > brokers. At the moment, I think the answer is yes. The old broker will not support the InitProducerId request, so the

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-18 Thread Jason Gustafson
> > 1. We should only delay it if there's a concrete benefit (e.g. we agree > that we need to improve OutOfOrderSequence and we can't do it in time). I > doubt that we will get much additional testing from users if we keep it off > by default for another release cycle (i.e. 4 months). Not sure

[GitHub] kafka pull request #3692: KAFKA-5689:Add MeteredWindowStore and refactor sto...

2017-08-18 Thread dguy
GitHub user dguy opened a pull request: https://github.com/apache/kafka/pull/3692 KAFKA-5689:Add MeteredWindowStore and refactor store hierarchy Add MeteredWindowStore and ChangeLoggingWindowBytesStore. Refactor Store hierarchy such that Metered is always the outermost store

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

2017-08-18 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5745; makeLeader should invoke `convertHWToLocalOffsetMetadata` -- [...truncated 915.21 KB...] kafka.controller.ControllerFailoverTest >

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-18 Thread Manikumar
> > 3. The message format requirement is a good point. This should be mentioned > in the compatibility section. Users who are still using the old message > format will get an error after the upgrade, right? > so this change will break client backward compatibility while connecting to 0.10.X

[jira] [Resolved] (KAFKA-5745) Partition.makeLeader() should convert HW to OffsetMetadata before becoming the leader

2017-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5745. Resolution: Fixed Fix Version/s: 1.0.0 0.11.0.1 > Partition.makeLeader()

[GitHub] kafka pull request #3682: KAFKA-5745: makeLeader should invoke `convertHWToL...

2017-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3682 --- 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 #3691: HOTFIX: Cherrypicking state deadlock fix from 0.11

2017-08-18 Thread enothereska
GitHub user enothereska opened a pull request: https://github.com/apache/kafka/pull/3691 HOTFIX: Cherrypicking state deadlock fix from 0.11 Cherrypicking additional changes made as part of this PR https://github.com/apache/kafka/pull/3622, back to trunk. You can merge this pull

[jira] [Created] (KAFKA-5750) Elevate log messages for denials to WARN in SimpleAclAuthorizer class

2017-08-18 Thread Phillip Walker (JIRA)
Phillip Walker created KAFKA-5750: - Summary: Elevate log messages for denials to WARN in SimpleAclAuthorizer class Key: KAFKA-5750 URL: https://issues.apache.org/jira/browse/KAFKA-5750 Project: Kafka

[GitHub] kafka pull request #3490: MINOR: Pass over DSL documentation up to stateful ...

2017-08-18 Thread enothereska
GitHub user enothereska reopened a pull request: https://github.com/apache/kafka/pull/3490 MINOR: Pass over DSL documentation up to stateful transformations Added the following documentation: - creating source streams from Kafka - stateless transformations - moved

[jira] [Created] (KAFKA-5749) Refactor SessionStore hierarchy

2017-08-18 Thread Damian Guy (JIRA)
Damian Guy created KAFKA-5749: - Summary: Refactor SessionStore hierarchy Key: KAFKA-5749 URL: https://issues.apache.org/jira/browse/KAFKA-5749 Project: Kafka Issue Type: Sub-task

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

2017-08-18 Thread Apache Jenkins Server
See

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

2017-08-18 Thread Apache Jenkins Server
See Changes: [ismael] KAFKA-5744; ShellTest: add tests for attempting to run nonexistent -- [...truncated 911.61 KB...]

[jira] [Resolved] (KAFKA-5744) ShellTest: add tests for attempting to run nonexistent program, error return

2017-08-18 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5744?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-5744. Resolution: Fixed Fix Version/s: 1.0.0 > ShellTest: add tests for attempting to run

[GitHub] kafka pull request #3679: KAFKA-5744: ShellTest: add tests for attempting to...

2017-08-18 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/kafka/pull/3679 --- 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

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-18 Thread Rajini Sivaram
Perhaps we could register dynamically for now. If we find that the cost of retaining these is high, we can add the code to expire them later. Is that ok? Regards, Rajini On Fri, Aug 18, 2017 at 9:41 AM, Ismael Juma wrote: > Can we quantify the cost of having these metrics

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-18 Thread Ismael Juma
Can we quantify the cost of having these metrics around if they are dynamically registered? Given that the maximum is bounded at development time, is it really worth all the extra code? Ismael On Fri, Aug 18, 2017 at 9:34 AM, Rajini Sivaram wrote: > Jun, > > It feels

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-18 Thread Rajini Sivaram
Jun, It feels more consistent to add errors as yammer metrics similar to other request metrics. Perhaps we could add some code to track and remove these if unused? It is a bit more work, but it would keep the externals consistent. Ismael/Manikumar, Agree that version as a String attribute makes

Re: [DISCUSS] KIP-187 - Add cumulative count metric for all Kafka rate metrics

2017-08-18 Thread Rajini Sivaram
Hi Jun, I have added constructors for all the variations. Haven't included these in the KIP since it is an internal class: https://github.com/apache/kafka/pull/3686/files#diff-00ca7d83906c4fb8a5066e9af9c5c34b Thank you, Rajini On Fri, Aug 18, 2017 at 1:24 AM, Jun Rao

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-18 Thread Ismael Juma
Hi Apurva and Jason, A few thoughts: 1. We should only delay it if there's a concrete benefit (e.g. we agree that we need to improve OutOfOrderSequence and we can't do it in time). I doubt that we will get much additional testing from users if we keep it off by default for another release cycle

Re: [DISCUSS] KIP-185: Make exactly once in order delivery per partition the default producer setting

2017-08-18 Thread Manikumar
I agree with Jason/Apruva comments. we can wait some more time and enable it in 1.1.0 release. In general, we are in agreement with the change. On Fri, Aug 18, 2017 at 10:32 AM, Apurva Mehta wrote: > Hi Jason, > > You make some good points. > > First, I agree that we could

Re: [DISCUSS] KIP-188 - Add new metrics to support health checks

2017-08-18 Thread Manikumar
I agree it will be good if we can add "commit id/version" as an attribute value. It will be easy to parse. But as of now, KafkaMetric supports only numerical values. On Fri, Aug 18, 2017 at 5:49 AM, Ismael Juma wrote: > Hi Rajini, > > About the gauges, I was thinking that