Re: [DISCUSS] Apache Kafka 2.0.0 Release Plan

2018-05-10 Thread Rajini Sivaram
Hi all, A reminder that KIP freeze for 2.0.0 is May 22. I have updated the release page with all the approved KIPs: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=80448820 We have many KIPs still under discussion and/or ready for voting. Please participate in discussions and vot

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-10 Thread Manikumar
+1 (non-binding) Thanks. On Wed, May 9, 2018 at 10:09 PM, Mickael Maison wrote: > +1, thanks for the KIP! > > On Wed, May 9, 2018 at 4:41 PM, Ted Yu wrote: > > +1 > > > > On Wed, May 9, 2018 at 8:28 AM, Rajini Sivaram > > wrote: > > > >> Hi all, > >> > >> Since there have been no objections o

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-05-10 Thread Rajini Sivaram
Hi Richard, Thanks for the KIP. +1 (binding) Regards, Rajini On Wed, May 9, 2018 at 10:54 PM, Guozhang Wang wrote: > +1 from me, thanks! > > > Guozhang > > On Wed, May 9, 2018 at 10:46 AM, Jason Gustafson > wrote: > > > Thanks for the KIP, +1 (binding). > > > > One small correction: the KIP

Re: [VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-10 Thread Rajini Sivaram
Hi Dhruvil, Thanks for the KIP! +1 (binding) Regards, Rajini On Wed, May 9, 2018 at 9:28 PM, Dhruvil Shah wrote: > Thanks for the feedback, Jason and Ismael. I renamed the config to > "message.downconversion.enable". > > Also, as an update, I found a potential problem with one of the suggesti

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-05-10 Thread Mickael Maison
+1 (non binding) Thanks On Thu, May 10, 2018 at 9:39 AM, Rajini Sivaram wrote: > Hi Richard, Thanks for the KIP. > > +1 (binding) > > Regards, > > Rajini > > On Wed, May 9, 2018 at 10:54 PM, Guozhang Wang wrote: > >> +1 from me, thanks! >> >> >> Guozhang >> >> On Wed, May 9, 2018 at 10:46 AM, Ja

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-05-10 Thread Manikumar
+1 (non-binding). Thanks. On Thu, May 10, 2018 at 2:33 PM, Mickael Maison wrote: > +1 (non binding) > Thanks > > On Thu, May 10, 2018 at 9:39 AM, Rajini Sivaram > wrote: > > Hi Richard, Thanks for the KIP. > > > > +1 (binding) > > > > Regards, > > > > Rajini > > > > On Wed, May 9, 2018 at 10:54

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-10 Thread Edoardo Comar
+1 (non-binding) On 10 May 2018 at 09:36, Manikumar wrote: > +1 (non-binding) > > Thanks. > > On Wed, May 9, 2018 at 10:09 PM, Mickael Maison > wrote: > > > +1, thanks for the KIP! > > > > On Wed, May 9, 2018 at 4:41 PM, Ted Yu wrote: > > > +1 > > > > > > On Wed, May 9, 2018 at 8:28 AM, Rajini

Re: [VOTE] KIP-283: Efficient Memory Usage for Down-Conversion

2018-05-10 Thread Edoardo Comar
+1 (non-binding) On 10 May 2018 at 09:56, Rajini Sivaram wrote: > Hi Dhruvil, Thanks for the KIP! > > +1 (binding) > > Regards, > > Rajini > > On Wed, May 9, 2018 at 9:28 PM, Dhruvil Shah wrote: > > > Thanks for the feedback, Jason and Ismael. I renamed the config to > > "message.downconversion

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-05-10 Thread zhenya Sun
+1 non-binding > 在 2018年5月10日,下午5:19,Manikumar 写道: > > +1 (non-binding). > Thanks. > > On Thu, May 10, 2018 at 2:33 PM, Mickael Maison > wrote: > >> +1 (non binding) >> Thanks >> >> On Thu, May 10, 2018 at 9:39 AM, Rajini Sivaram >> wrote: >>> Hi Richard, Thanks for the KIP. >>> >>> +1 (bi

Re: [VOTE] KIP-266: Add TimeoutException for KafkaConsumer#position

2018-05-10 Thread Edoardo Comar
+1 (non-binding) On 10 May 2018 at 10:29, zhenya Sun wrote: > +1 non-binding > > > 在 2018年5月10日,下午5:19,Manikumar 写道: > > > > +1 (non-binding). > > Thanks. > > > > On Thu, May 10, 2018 at 2:33 PM, Mickael Maison < > mickael.mai...@gmail.com> > > wrote: > > > >> +1 (non binding) > >> Thanks > >>

Re: Use of a formatter like Scalafmt

2018-05-10 Thread Joan Goyeau
Thanks guys for your feedback. Matthias, we can change the config to use JavaDoc (1 space) instead of the Scala doc (2 space), that would limit the change indeed. When I say we can apply this change module by module I mean we can specify folders, so we could breakdown core too. I updated the PR

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-10 Thread Andy Coates
Rather than having name and pattern fields on the ResourceFilter, where it’s only valid for one to be set, and we want to restrict the character set in case future enhancements need them, we could instead add a new integer ‘nameType’ field, and use constants to indicate how the name field should

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

2018-05-10 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6870 Concurrency conflicts in SampledStat (#4985) -- [...truncated 64.07 KB...] withAuthorizer(opts) { authorizer =>

RE: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-05-10 Thread Skrzypek, Jonathan
Hi, Have implemented the changes discussed. bootstrap.reverse.dns.lookup is disabled by default. When enabled, the client will perform reverse dns lookup regardless of the security protocol used. https://github.com/apache/kafka/pull/4485 Jonathan Skrzypek -Original Message- From: Sk

Re: [VOTE] KIP-235 Add DNS alias support for secured connection

2018-05-10 Thread Rajini Sivaram
Thanks Jonathan. You have binding votes from me and Gwen. One more binding vote is required for this KIP to be approved. On Thu, May 10, 2018 at 1:14 PM, Skrzypek, Jonathan < jonathan.skrzy...@gs.com> wrote: > Hi, > > Have implemented the changes discussed. > bootstrap.reverse.dns.lookup is disab

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

2018-05-10 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6870 Concurrency conflicts in SampledStat (#4985) -- Started by an SCM change [EnvInject] - Loading node environment variables. Building

[jira] [Created] (KAFKA-6892) Kafka Streams memory usage grows

2018-05-10 Thread Dawid Kulig (JIRA)
Dawid Kulig created KAFKA-6892: -- Summary: Kafka Streams memory usage grows Key: KAFKA-6892 URL: https://issues.apache.org/jira/browse/KAFKA-6892 Project: Kafka Issue Type: Bug Compone

Build failed in Jenkins: kafka-trunk-jdk10 #95

2018-05-10 Thread Apache Jenkins Server
See Changes: [rajinisivaram] KAFKA-6870 Concurrency conflicts in SampledStat (#4985) -- [...truncated 1.50 MB...] kafka.network.SocketServerTest > testGracefulClose STARTED

[jira] [Resolved] (KAFKA-6870) Concurrency conflicts in SampledStat

2018-05-10 Thread Chia-Ping Tsai (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6870?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chia-Ping Tsai resolved KAFKA-6870. --- Resolution: Fixed Reviewer: Rajini Sivaram Thanks for the reviews. [~rsivaram] > Concur

[jira] [Resolved] (KAFKA-6828) Index files are no longer sparse in Java 9/10 due to OpenJDK regression

2018-05-10 Thread Ismael Juma (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6828?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma resolved KAFKA-6828. Resolution: Fixed Great! Let's close this then. > Index files are no longer sparse in Java 9/10 due

[jira] [Resolved] (KAFKA-6825) DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG is private

2018-05-10 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6825?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-6825. -- Resolution: Fixed Assignee: Guozhang Wang Fix Version/s: 2.0.0 > DEFAULT_PRODUCTION_EXCE

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-10 Thread Guozhang Wang
Thanks for your thoughts Matthias. I think if we do want to bring KIP-244 into 2.0 then we need to keep its scope small and well defined. For that I'm proposing: 1. Make the inheritance implementation of headers consistent with what we had with other record context fields. I.e. pass through the re

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-10 Thread Matthias J. Sax
Thanks Guozhang! Sounds good to me! -Matthias On 5/10/18 7:55 AM, Guozhang Wang wrote: > Thanks for your thoughts Matthias. I think if we do want to bring KIP-244 > into 2.0 then we need to keep its scope small and well defined. For that > I'm proposing: > > 1. Make the inheritance implementatio

Re: [DISCUSS] KIP-290: Support for wildcard suffixed ACLs

2018-05-10 Thread Colin McCabe
Hi Andy, The issue that I was trying to solve here is the Java API. Right now, someone can write "new ResourceFilter(ResourceType.TRANSACTIONAL_ID, "foo*") and have a ResourceFilter that applies to a Transactional ID named "foo*". This has to continue to work, or else we have broken compatibi

Re: [VOTE] KIP-255: OAuth Authentication via SASL/OAUTHBEARER

2018-05-10 Thread Ron Dagostino
HI again, everyone. Still looking for 2 more binding votes. PR is now available at https://github.com/apache/kafka/pull/4994. Ron On Tue, May 8, 2018 at 9:45 AM, Ron Dagostino wrote: > HI everyone. Can we get 2 more binding votes on this KIP (and non-binding > votes, too)? > > Ron > > On Fri

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

2018-05-10 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Update dynamic broker configuration doc for truststore update -- [...truncated 422.89 KB...] kafka.admin.ResetConsumerGroupOffsetTest > testRe

Jenkins build is back to normal : kafka-trunk-jdk10 #96

2018-05-10 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-6141) Errors logs when running integration/kafka/tools/MirrorMakerIntegrationTest

2018-05-10 Thread Manikumar (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6141?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Manikumar resolved KAFKA-6141. -- Resolution: Fixed Closing this as log level changed to debug in ZOOKEEPER-2795 / Zookeeper 3.4.12 > Err

Can anyone take a look at this KIP and Jira?

2018-05-10 Thread qingjun wu
Dear Kafka Developers, I opened a KIP and also a Jira ticket related to this. Can you please take a look? It should be simple change to Kafka, but it should improve the performance a lot. https://cwiki.apache.org/confluence/display/KAFKA/KIP-NEXT%3A+Get+rid+of+unnecessary+read+lock https://iss

[jira] [Created] (KAFKA-6893) Processors created after acceptor started which can cause in a brief refusal to accept connections

2018-05-10 Thread Ryan P (JIRA)
Ryan P created KAFKA-6893: - Summary: Processors created after acceptor started which can cause in a brief refusal to accept connections Key: KAFKA-6893 URL: https://issues.apache.org/jira/browse/KAFKA-6893 P

Re: Can anyone take a look at this KIP and Jira?

2018-05-10 Thread Ted Yu
Since the change is internal to *SensorAccess class, looks like KIP is not required.* On Thu, May 10, 2018 at 11:54 AM, qingjun wu wrote: > Dear Kafka Developers, > > I opened a KIP and also a Jira ticket related to this. Can you please take > a look? It should be simple change to Kafka, but it

[jira] [Created] (KAFKA-6894) Cannot access GlobalKTable store from KStream.transform()

2018-05-10 Thread Robert Yokota (JIRA)
Robert Yokota created KAFKA-6894: Summary: Cannot access GlobalKTable store from KStream.transform() Key: KAFKA-6894 URL: https://issues.apache.org/jira/browse/KAFKA-6894 Project: Kafka Issue

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

2018-05-10 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-6893) Processors created after acceptor started which can cause in a brief refusal to accept connections

2018-05-10 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6893?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-6893. Resolution: Fixed Fix Version/s: 1.1.1 > Processors created after acceptor started wh

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-10 Thread Jakub Scholz
+1 (non-binding) On Thu, May 10, 2018 at 11:24 AM, Edoardo Comar wrote: > +1 (non-binding) > > On 10 May 2018 at 09:36, Manikumar wrote: > > > +1 (non-binding) > > > > Thanks. > > > > On Wed, May 9, 2018 at 10:09 PM, Mickael Maison < > mickael.mai...@gmail.com> > > wrote: > > > > > +1, thanks f

[jira] [Created] (KAFKA-6895) Schema Inferencing for JsonConverter

2018-05-10 Thread Allen Tang (JIRA)
Allen Tang created KAFKA-6895: - Summary: Schema Inferencing for JsonConverter Key: KAFKA-6895 URL: https://issues.apache.org/jira/browse/KAFKA-6895 Project: Kafka Issue Type: New Feature

Request to create KIP

2018-05-10 Thread Anurag Jain
Hi, I would like to create KIP for *https://issues.apache.org/jira/browse/KAFKA-2200 *. Can you please give me permission to create KIP for this? Thanks, Anurag

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

2018-05-10 Thread Apache Jenkins Server
See

[jira] [Resolved] (KAFKA-6894) Improve error message when connecting processor with a global store

2018-05-10 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-6894?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang resolved KAFKA-6894. -- Resolution: Fixed Fix Version/s: 2.0.0 > Improve error message when connecting processor

Re: Request to create KIP

2018-05-10 Thread Matthias J. Sax
What is your Wiki ID ? -Matthias On 5/10/18 3:36 AM, Anurag Jain wrote: > Hi, > > I would like to create KIP for > *https://issues.apache.org/jira/browse/KAFKA-2200 > *. > > Can you please give me permission to create KIP for this? > > Thanks,

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-10 Thread Jun Rao
Hi, Rajini, Thanks for the KIP. +1 Could you document in the wiki how to set ssl.endpoint.identification.algorithm to empty in the server property file and through dynamic config? It's not obvious how to do that. Jun On Wed, May 9, 2018 at 8:28 AM, Rajini Sivaram wrote: > Hi all, > > Since th

Re: [VOTE] KIP-294 - Enable TLS hostname verification by default

2018-05-10 Thread Ismael Juma
Thanks for the KIP, +1 (binding) from me. Ismael On Wed, May 9, 2018 at 8:29 AM Rajini Sivaram wrote: > Hi all, > > Since there have been no objections on this straightforward KIP, I would > like to initiate the voting process. KIP-294 proposes to use a secure > default value for endpoint ident

Re: [VOTE] KIP-281: ConsumerPerformance: Increase Polling Loop Timeout and Make It Reachable by the End User

2018-05-10 Thread Ismael Juma
Thanks for the KIP, +1 (binding). A few suggestions: 1. We normally include the time unit in configs. Not sure if we do it for command line parameters though, so can we please verify and make it consistent? 2. The KIP mentions --polling-loop-timeout and --timeout. Which is it? 3. Can we include th

[VOTE] KIP-278: Add version option to Kafka's commands

2018-05-10 Thread Sasaki Toru
Hi all, I would like to start the vote on KIP-278: Add version option to Kafka's commands. The link to this KIP is here: The discussion thread is here:

Re: [DISCUSS] KIP-278: Add version option to Kafka's commands

2018-05-10 Thread Sasaki Toru
I started voting thread for this KIP. Thanks, Jason and Colin. From: Colin McCabe Date: 2018-05-10 2:53 GMT+09:00 Subject: Re: [DISCUSS] KIP-278: Add version option to Kafka's commands To: dev@kafka.apache.org +1. Thanks, Sasaki. Colin On Wed, May 9, 2018, at 09:15, Jason Gustafson wrote:

Re: [VOTE] KIP-278: Add version option to Kafka's commands

2018-05-10 Thread Ted Yu
+1 On Thu, May 10, 2018 at 6:42 PM, Sasaki Toru wrote: > Hi all, > > I would like to start the vote on KIP-278: Add version option to Kafka's > commands. > > The link to this KIP is here: > +Add+version+option+to+Kafka%27s+commands>

[jira] [Created] (KAFKA-6896) add producer metrics exporting in KafkaStreams.java

2018-05-10 Thread Boyang Chen (JIRA)
Boyang Chen created KAFKA-6896: -- Summary: add producer metrics exporting in KafkaStreams.java Key: KAFKA-6896 URL: https://issues.apache.org/jira/browse/KAFKA-6896 Project: Kafka Issue Type: Imp

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

2018-05-10 Thread Apache Jenkins Server
See

Re: [DISCUSS] KIP-275 - Indicate "isClosing" in the SinkTaskContext

2018-05-10 Thread Matt Farmer
Given that the conversation has lingered for a bit, I've gone ahead and opened up a PR with the initial implementation. Let me know your thoughts! https://github.com/apache/kafka/pull/5002 Also, voting is open - so if you like this idea please send me some binding +1's before May 22nd so we can g

[jira] [Created] (KAFKA-6897) Mirrormaker waits to shut down forever on produce failure with abort.on.send.failure=true

2018-05-10 Thread Koelli Mungee (JIRA)
Koelli Mungee created KAFKA-6897: Summary: Mirrormaker waits to shut down forever on produce failure with abort.on.send.failure=true Key: KAFKA-6897 URL: https://issues.apache.org/jira/browse/KAFKA-6897

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

2018-05-10 Thread Apache Jenkins Server
See Changes: [github] MINOR: A few small cleanups in AdminClient from KAFKA-6299 (#4989) -- [...truncated 421.81 KB...] kafka.admin.ResetConsumerGroupOffsetTest > testRes