Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-17 Thread Navinder Brar
Hi John, Thanks for looking into it.  On using constructors rather than static factory methods I was coming from the convention on the classes currently available to users such as LagInfo and KeyQueryMetadata. Let me know if it's still favorable to change StoreQueryParams into static factory met

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [matthias] MINOR: move "Added/Removed sensor" log messages to TRACE (#7502) -- [...truncated 2.98 MB...] kafka.controller.PartitionLeaderElectionAlgorithmsTest

Re: [DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-17 Thread John Roesler
Thanks, Navinder! I took a look at the KIP. We tend to use static factory methods instead of public constructors, and also builders for optional parameters. Given that, I think it would be more typical to have a factory method: storeQueryParams() and also builders for setting the optional par

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2020-01-17 Thread John Roesler
Thanks, Vito. I've just cast my vote. -John On Fri, Jan 17, 2020, at 21:32, Vito Jeng wrote: > Hi, folks, > > Just update the KIP, please take a look. > > Thanks! > > --- > Vito > > > On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng wrote: > > > Thanks Bill, John and Matthias. Glad you guys joined

Re: [VOTE] KIP-216: IQ should throw different exceptions for different errors

2020-01-17 Thread John Roesler
Thanks for the KIP! I'm +1 (binding) Thanks, -John On Thu, Jan 16, 2020, at 08:46, Bill Bejeck wrote: > Thanks for the KIP. > > +1 (binding) > > -Bill > > On Tue, Jan 14, 2020 at 9:41 AM Navinder Brar > wrote: > > > +1 (non-binding) With a small comment which was mentioned by Vinoth as > >

Re: [DISCUSS]KIP-216: IQ should throw different exceptions for different errors

2020-01-17 Thread Vito Jeng
Hi, folks, Just update the KIP, please take a look. Thanks! --- Vito On Fri, Jan 17, 2020 at 9:12 AM Vito Jeng wrote: > Thanks Bill, John and Matthias. Glad you guys joined this discussion. > I got a lot out of the discussion. > > I would like to update KIP-216 base on John's suggestion to r

[VOTE] KIP-558: Add Connect REST API endpoints to view the topics used by connectors in Kafka Connect

2020-01-17 Thread Konstantine Karantasis
Hi all, I'd like to open the vote on KIP-558 that had a constructive flurry of discussions in the past few days, in order to give this KIP the opportunity to be voted on by the current KIP deadline (Wed, Jan 22, 2020), if - of course - there's agreement upon its final form. KIP link here: https:/

Build failed in Jenkins: kafka-2.4-jdk8 #128

2020-01-17 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9449; Adds support for closing the producer's BufferPool. (#7967) -- [...truncated 2.74 MB...] org.apache.kafka.streams.test.OutputVerifierTest >

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Konstantine Karantasis
Hi all, I've updated KIP-558 with the following based on our previous discussion: * Added timestamp to the metadata (the record value). * The KIP now mentions a metric-based implementation in the Rejected Alternatives section. * The record key format is now using the single character ':' as a sep

Build failed in Jenkins: kafka-trunk-jdk11 #1095

2020-01-17 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9449; Adds support for closing the producer's BufferPool. (#7967) [jason] KAFKA-9329; KafkaController::replicasAreValid should return error [github] KAFKA-9338; Fetch session should

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9449; Adds support for closing the producer's BufferPool. (#7967) [jason] KAFKA-9329; KafkaController::replicasAreValid should return error [github] KAFKA-9338; Fetch session should c

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2020-01-17 Thread Ron Dagostino
Thanks, Rajini. Still not sure what the answer is, but I thought of a couple more issues with config inheritance that I wanted to raise. The first is a minor issue, but just to document it (and I will add it to the KIP as well), ZooKeeper does not support a key password that differs from the keyst

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Konstantine Karantasis
Thanks for the follow up Chris. Replies below: On Fri, Jan 17, 2020 at 3:07 PM Christopher Egerton wrote: > Thanks, Konstantine. Just a few more questions: > > > > 2. What is the motivation for the `topic.tracking.allow.reset` config? > Is > > > there any anticipated case where it would be usefu

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Christopher Egerton
Thanks, Konstantine. Just a few more questions: > > 2. What is the motivation for the `topic.tracking.allow.reset` config? Is > > there any anticipated case where it would be useful to have topic tracking > > enabled but with resets disabled? We could easily add this configuration > > later if a u

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [github] MINOR: Handle expandIsr in PartitionLockTest and ensure read threads not -- [...truncated 2.82 MB...] org.apache.kafka.streams.TopologyTestDriverT

[jira] [Created] (KAFKA-9451) Pass consumer group metadata to producer on commit

2020-01-17 Thread Matthias J. Sax (Jira)
Matthias J. Sax created KAFKA-9451: -- Summary: Pass consumer group metadata to producer on commit Key: KAFKA-9451 URL: https://issues.apache.org/jira/browse/KAFKA-9451 Project: Kafka Issue Ty

[jira] [Resolved] (KAFKA-9338) Incremental fetch sessions do not maintain or use leader epoch for fencing purposes

2020-01-17 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9338?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9338. Fix Version/s: 2.5.0 Resolution: Fixed Marking this just as 2.5 for now. If we don't

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2020-01-17 Thread Rajini Sivaram
Hi Ron, For Kafka, we moved from hostname verification disabled-by-default to enabled-by-default under https://cwiki.apache.org/confluence/display/KAFKA/KIP-294+-+Enable+TLS+hostname+verification+by+default. So we have tested empty String for disabling hostname verification and we know that it wor

[jira] [Resolved] (KAFKA-9329) KafkaController::replicasAreValid should return error

2020-01-17 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9329?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9329. Fix Version/s: 2.5.0 Resolution: Fixed > KafkaController::replicasAreValid should re

[jira] [Resolved] (KAFKA-9449) Producer's BufferPool may block the producer from closing.

2020-01-17 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9449?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9449. Fix Version/s: 2.4.1 Resolution: Fixed > Producer's BufferPool may block the produce

Re: [DISCUSS] KIP-546: Add quota-specific APIs to the Admin Client, redux

2020-01-17 Thread Brian Byrne
Thanks Colin, I've updated the KIP with the relevant changes. On Fri, Jan 17, 2020 at 10:17 AM Colin McCabe wrote: > I thought about this a little bit more, and maybe we can leave in the > enums rather than going with strings. But we need to have an "UNKNOWN" > value for all the enums, so that

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Konstantine Karantasis
Hey Chris! Thanks for the comments. Answers inline below: On Fri, Jan 17, 2020 at 11:47 AM Christopher Egerton wrote: > Hi Konstantine, > > Thanks for the KIP! There's been a lot of productive discussion so far so > I'll try to keep my remarks brief. > > 1. As far as resetting the active topics

Re: CompletableFuture?

2020-01-17 Thread radai
I'm currently doing my own completableFutures by using the callbacks. While i wont have the time to do this myself, I still think its a great idea and would prevent more people from having to do what I'm currently doing. On Wed, Jan 15, 2020 at 6:57 AM Vamsi Subahsh wrote: > > Hi, > > I'm interes

Re: [DISCUSS] KIP-489 Kafka Consumer Record Latency Metric

2020-01-17 Thread Sean Glover
Hi Habib, With regards to your earlier question about timezones, I've updated the KIP to remove the LatencyTime abstraction since it is no longer relevant. I added a note about epoch time as well. Thanks, Sean On Wed, Jan 15, 2020 at 8:28 AM Habib Nahas wrote: > Hi Sean, > > Thats great, look

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Fix typo in connect integration test class name (#7976) -- [...truncated 5.74 MB...] org.apache.kafka.streams.MockTimeTest > shouldSetStartTim

Re: [DISCUSS] KIP-489 Kafka Consumer Record Latency Metric

2020-01-17 Thread Sean Glover
Hi Gokul, Thank you for your detailed review. I've summarized the updates I've made to the KIP inline below. Please review the updated KIP when you have time. On Fri, Dec 20, 2019 at 6:56 AM Gokul Ramanan Subramanian < gokul24...@gmail.com> wrote: > Hi Sean. > > Thanks for writing this KIP. So

Re: [VOTE] KIP-555: An admin tools proposal to accomplish the deprecation of zookeeper access that is direct

2020-01-17 Thread Colin McCabe
Thanks, all. With binding +1 votes from Gwen Shapira, Manikumar Reddy, Mickael Maison, and M. Manna, and non-binding +1 votes from Ron Dagostino, Viktor Somogyi-Vass and David Jacot, the vote passes. best, Colin On Fri, Jan 17, 2020, at 10:59, David Jacot wrote: > +1 (non-binding) > > Thanks

Re: [DISCUSS] KIP-558: Track the set of actively used topics by connectors in Kafka Connect

2020-01-17 Thread Christopher Egerton
Hi Konstantine, Thanks for the KIP! There's been a lot of productive discussion so far so I'll try to keep my remarks brief. 1. As far as resetting the active topics for a connector goes, it's noted in the KIP that this can be done for a deleted connector. Can this also be done for connectors tha

Re: KIP-560 Discuss

2020-01-17 Thread Gwen Shapira
Seem like a very nice improvement to me. But I have to admit that I don't understand how this will how - how could you infer the input topics? On Thu, Jan 16, 2020 at 10:03 AM Sang wn Lee wrote: > > Hello, > > Starting this thread to discuss KIP-560: > wiki link : > https://cwiki.apache.org/confl

Build failed in Jenkins: kafka-trunk-jdk11 #1094

2020-01-17 Thread Apache Jenkins Server
See Changes: [github] MINOR: Handle expandIsr in PartitionLockTest and ensure read threads not -- Started by an SCM change Running as SYSTEM [EnvInject] - Loading node

[jira] [Created] (KAFKA-9450) Decouple inner state flushing from committing with EOS

2020-01-17 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-9450: -- Summary: Decouple inner state flushing from committing with EOS Key: KAFKA-9450 URL: https://issues.apache.org/jira/browse/KAFKA-9450 Project: Kafka

Re: [VOTE] KIP-555: An admin tools proposal to accomplish the deprecation of zookeeper access that is direct

2020-01-17 Thread David Jacot
+1 (non-binding) Thanks for the KIP! David Le ven. 17 janv. 2020 à 19:25, Colin McCabe a écrit : > Hi all, > > I'm going to close the vote later today. > > thanks, > Colin > > > On Wed, Jan 15, 2020, at 06:34, M. Manna wrote: > > +1 (Binding) - a long awaited KIP to have a simpler partition >

Re: [VOTE] KIP-555: An admin tools proposal to accomplish the deprecation of zookeeper access that is direct

2020-01-17 Thread Colin McCabe
Hi all, I'm going to close the vote later today. thanks, Colin On Wed, Jan 15, 2020, at 06:34, M. Manna wrote: > +1 (Binding) - a long awaited KIP to have a simpler partition reassignment > script (without ZK)> > > Kudos to you Colin :) > > On Wed, 15 Jan 2020 at 10:10, Viktor Somogyi-Vass >

Build failed in Jenkins: kafka-trunk-jdk11 #1093

2020-01-17 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8847; Deprecate and remove usage of supporting classes in [mimaison] KAFKA-9218: MirrorMaker 2 can fail to create topics (#7745) [jason] MINOR: Fix typo in connect integration test

Re: [DISCUSS] KIP-546: Add quota-specific APIs to the Admin Client, redux

2020-01-17 Thread Colin McCabe
Hi Brian, Thanks again for working on this. It's looking good. I thought about this a little bit more, and maybe we can leave in the enums rather than going with strings. But we need to have an "UNKNOWN" value for all the enums, so that if a value that the client doesn't understand is returne

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [github] KAFKA-8847; Deprecate and remove usage of supporting classes in [mimaison] KAFKA-9218: MirrorMaker 2 can fail to create topics (#7745) -- [...trunc

[jira] [Created] (KAFKA-9449) Producer's BufferPool may block the producer from closing.

2020-01-17 Thread Brian Byrne (Jira)
Brian Byrne created KAFKA-9449: -- Summary: Producer's BufferPool may block the producer from closing. Key: KAFKA-9449 URL: https://issues.apache.org/jira/browse/KAFKA-9449 Project: Kafka Issue Ty

Re: [VOTE] KIP-373: Allow users to create delegation tokens for other users

2020-01-17 Thread Rajini Sivaram
Hi Viktor, Thanks for the KIP. A few questions: 1) kafka-acls.sh has options like* --topic* that specifies a single topic. Is there a reason why we want to have *--users* instead of *--user *with a single user? 2) We use user principal rather than just the name everywhere else. Can we do the same

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [jason] MINOR: Remove unnecessary call to `super` in `MetricConfig` constructor [rajinisivaram] [MINOR]: Fix typo in Fetcher comment (#7934) [github] MINOR: Suppress DescribeConfigs Denied log duri

Build failed in Jenkins: kafka-trunk-jdk11 #1092

2020-01-17 Thread Apache Jenkins Server
See Changes: -- Started by an SCM change Running as SYSTEM [EnvInject] - Loading node environment variables. Building remotely on H29 (ubuntu) in workspace

Build failed in Jenkins: kafka-trunk-jdk11 #1091

2020-01-17 Thread Apache Jenkins Server
See Changes: -- Started by an SCM change Started by an SCM change Running as SYSTEM [EnvInject] - Loading node environment variables. Building remotely on H29 (ubuntu) in workspace

Jenkins build is back to normal : kafka-trunk-jdk11 #1090

2020-01-17 Thread Apache Jenkins Server
See

Re: [VOTE] KIP-373: Allow users to create delegation tokens for other users

2020-01-17 Thread Viktor Somogyi-Vass
Hi Jun & Richard, Jun, thanks for your feedback and vote. 100. Thanks, I'll correct that. 101. (@Richard) in this case the principal names will be something like "CN=writeuser,OU=Unknown,O=Unknown,L=Unknown,ST=Unknown,C=Unknown" unless principal mapping or builder is defined (refer to [1]). I th

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2020-01-17 Thread Ron Dagostino
Hi again, Rajini. I've updated the KIP, and while doing it I became concerned about using zookeeper.ssl.endpoint.identification.algorithm for enabling/disabling hostname verification. The KIP reflects what we decided, but upon reading it, I wonder if it is workable. Here's what it says for this

Re: [VOTE] KIP-560: Auto infer external topic partitions in stream reset tool

2020-01-17 Thread Sang wn Lee
Hello. Boyang Thanks for the guide! I agree It is a good idea to expand the input topic. So I want to add '--reset-all-external-topics' and extend '--input-topic'. Thank you for your feedback On 2020/01/17 05:39:03, Boyang Chen wrote: > Hey Sang, > > thanks so much for driving this effort.

[DISCUSS] : KIP-562: Allow fetching a key from a single partition rather than iterating over all the stores on an instance

2020-01-17 Thread Navinder Brar
Hi all, I have created a new KIP:  https://cwiki.apache.org/confluence/display/KAFKA/KIP-562%3A+Allow+fetching+a+key+from+a+single+partition+rather+than+iterating+over+all+the+stores+on+an+instance Please take a look if you get a chance. ~Navinder

[jira] [Resolved] (KAFKA-9218) MirrorMaker 2 can fail to create topics

2020-01-17 Thread Mickael Maison (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9218?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mickael Maison resolved KAFKA-9218. --- Fix Version/s: 2.5.0 Resolution: Fixed > MirrorMaker 2 can fail to create topics > ---

[jira] [Resolved] (KAFKA-8865) KIP-504: New Java Authorizer API

2020-01-17 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8865?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8865. --- Fix Version/s: 2.4.0 Resolution: Fixed > KIP-504: New Java Authorizer API > --

[jira] [Resolved] (KAFKA-8847) Deprecate and remove usage of supporting classes in kafka.security.auth

2020-01-17 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8847?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8847. --- Reviewer: Manikumar Resolution: Fixed > Deprecate and remove usage of supporting classe

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2020-01-17 Thread Ron Dagostino
Hi Rajini. I’ll submit a documentation PR to clarify the Kafka behavior of #1 and will adopt the same config for ZK. I agree now we should inherit for AclAuthorizer too — I was just stuck on the “no inheritance” idea more than I realized, and while the ZK quorums are different that doesn’t nec

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

2020-01-17 Thread Apache Jenkins Server
See Changes: -- Started by an SCM change Started by an SCM change Started by an SCM change Started by an SCM change Started by an SCM change Running as SYSTEM [EnvInject] - Loading node

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

2020-01-17 Thread Apache Jenkins Server
See Changes: [vvcephei] KAFKA-9431: Expose API in KafkaStreams to fetch all local offset lags [github] MINOR: fix flaky StreamsUpgradeTestIntegrationTest (#7974) -- [...

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-01-17 Thread Николай Ижиков
Thanks, Rajini. Will do it, shortly. > 17 янв. 2020 г., в 14:50, Rajini Sivaram написал(а): > > Hi Nikolay, > > 1) You can update KIP-553 to disable old protocols. This would mean: > 1a) SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS would be just TLSv1.2 > 1b) SslConfigs.DEFAULT_SSL_PROTOCOL wo

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-01-17 Thread Rajini Sivaram
Hi Nikolay, 1) You can update KIP-553 to disable old protocols. This would mean: 1a) SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS would be just TLSv1.2 1b) SslConfigs.DEFAULT_SSL_PROTOCOL would become TLSv1.2 2) When the testing for TLSv1.3 has been done, open a new KIP to enable TLSv1.3 by def

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-01-17 Thread Николай Ижиков
Hello, Rajini. Yes, we can! I have to write another KIP that goal will be keep only TLSv1.2 and TLSv1.3 in SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS Is it correct? > 17 янв. 2020 г., в 14:13, Rajini Sivaram написал(а): > > Hi Nikolay, > > Can we split this KIP into two: > 1) Remove insecure

Re: [DISCUSS] KIP-553: Enable TLSv1.3 by default and disable all protocols except [TLSV1.2, TLSV1.3]

2020-01-17 Thread Rajini Sivaram
Hi Nikolay, Can we split this KIP into two: 1) Remove insecure TLS protocols from the default values 2) Enable TLSv1.3 Since we are coming up to KIP freeze for 2.5.0 release, it will be good if we can get at least the first one into 2.5.0. It would be a much smaller change and won't get blocked b

Re: [DISCUSS] KIP-515: Enable ZK client to use the new TLS supported authentication

2020-01-17 Thread Rajini Sivaram
Hi Ron, Unresolved item #1: Yes, Kafka disables hostname verification if "ssl.endpoint.identification.algorithm" is an empty string. Unresolved item #2: Yes, those 9 plus hostname verification. If we do decide to inherit Kafka configs, wouldn't we inherit these 10 in AclAuthorizer as well? Unres

Build failed in Jenkins: kafka-trunk-jdk11 #1089

2020-01-17 Thread Apache Jenkins Server
See Changes: -- Started by an SCM change Running as SYSTEM [EnvInject] - Loading node environment variables. Building remotely on H29 (ubuntu) in workspace