Re: KIP-560 Discuss

2020-01-23 Thread Boyang Chen
Thanks Sophie for the explanation! I read Sang's PR and basically he did exactly what you proposed (check it here in case I'm wrong). I think Sophie's response answers Gwen's question already, while in the meantime for a KIP itself we are not

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-23 Thread Satish Duggana
+1 (non-binding) On Fri, Jan 24, 2020 at 11:10 AM Harsha Chintalapani wrote: > > +1 ( binding). Much needed! > -Harsha > > > On Thu, Jan 23, 2020 at 7:17 PM, Guozhang Wang wrote: > > > +1 (binding) > > > > On Thu, Jan 23, 2020 at 1:55 PM Guozhang Wang wrote: > > > > Yeah that makes sense, it

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

2020-01-23 Thread Navinder Brar
Thanks, everyone for voting and discussions. The KIP is passed with 3 binding votes from John, Matthias, and Guozhang and 1 non-binding vote from Bruno. On Friday, 24 January, 2020, 08:50:21 am IST, Bruno Cadonna wrote: Hi Navinder, +1 (non-binding) Best, Bruno On Thu, Jan 23,

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-23 Thread Harsha Chintalapani
+1 ( binding). Much needed! -Harsha On Thu, Jan 23, 2020 at 7:17 PM, Guozhang Wang wrote: > +1 (binding) > > On Thu, Jan 23, 2020 at 1:55 PM Guozhang Wang wrote: > > Yeah that makes sense, it is a good-to-have if we can push through this in > 2.5 but if we do not have bandwidth that's fine

Re: KIP-560 Discuss

2020-01-23 Thread Sophie Blee-Goldman
Hi all, I think what Gwen is trying to ask (correct me if I'm wrong) is how we can infer which topics are associated with Streams from the admin client's topic list. I agree that this doesn't seem possible, since as she pointed out the topics list (or even description) lacks the specific

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

2020-01-23 Thread Bruno Cadonna
Hi Navinder, +1 (non-binding) Best, Bruno On Thu, Jan 23, 2020 at 9:19 AM John Roesler wrote: > > Thanks, Navinder. It's just to give everyone a chance to object if they > wanted to. > -John > > On Thu, Jan 23, 2020, at 00:44, Navinder Brar wrote: > > Oh sorry, my bad. Will wait for another

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-23 Thread Guozhang Wang
+1 (binding) On Thu, Jan 23, 2020 at 1:55 PM Guozhang Wang wrote: > Yeah that makes sense, it is a good-to-have if we can push through this in > 2.5 but if we do not have bandwidth that's fine too :) > > Guozhang > > On Thu, Jan 23, 2020 at 1:40 PM David Jacot wrote: > >> Hi Guozhang, >> >>

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

2020-01-23 Thread Navinder Brar
Hi John, Thanks for the responses. I will make the below changes as I had suggested earlier, and then close the vote in a few hours. includeStaleStores -> staleStores withIncludeStaleStores() > enableStaleStores() includeStaleStores() -> staleStoresEnabled() Thanks, Navinder Sent from Yahoo

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

2020-01-23 Thread John Roesler
Hi Bruno, Thanks for your question; it's a very reasonable response to what I said before. I didn't mean "field" as in an instance variable, just as in a specific property or attribute. It's hard to talk about because all the words for this abstract concept are also words that people commonly

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

2020-01-23 Thread Bruno Cadonna
Hi John, One question: Why must the field name be involved in the naming? It somehow contradicts encapsulation. Field name `includeStaleStores` (or `staleStoresEnabled`) sounds perfectly fine to me. IMO, we should decouple the parameter name from the actual field name. Bruno On Thu, Jan 23,

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

2020-01-23 Thread John Roesler
Hi all, Thanks for the discussion! The basic idea I used in the original draft of the grammar was to avoid "fancy code" and just write "normal java". That's why I favored java bean spec over Kafka code traditions. According to the bean spec, setters always start with "set" and getters always

[jira] [Resolved] (KAFKA-8317) ClassCastException using KTable.suppress()

2020-01-23 Thread Matthias J. Sax (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8317?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Matthias J. Sax resolved KAFKA-8317. Resolution: Duplicate Closing this ticket in favor of KAFKA-9259 that duplicates it. >

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-23 Thread Guozhang Wang
Yeah that makes sense, it is a good-to-have if we can push through this in 2.5 but if we do not have bandwidth that's fine too :) Guozhang On Thu, Jan 23, 2020 at 1:40 PM David Jacot wrote: > Hi Guozhang, > > Thank you for your input. > > 1) You're right. I've put it there due to the version

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-23 Thread David Jacot
Hi Guozhang, Thank you for your input. 1) You're right. I've put it there due to the version bump only. I'll make it clearer. 2) I'd rather prefer to keep the scope as it is because 1) that field is not related to the problem that we are solving here and 2) I am not sure that I will have the

Re: [DISCUSS] KIP-566: Add rebalance callbacks to ConsumerInterceptor

2020-01-23 Thread M. Manna
Hey Thomas, On Thu, 23 Jan 2020 at 21:17, Thomas Becker wrote: > Hi folks, > I'd like to open the discussion for KIP-566: Add rebalance callbacks to > ConsumerInterceptor. We've been looking to implement some custom metrics > via ConsumerInterceptor, and not knowing when partition ownership

[DISCUSS] KIP-566: Add rebalance callbacks to ConsumerInterceptor

2020-01-23 Thread Thomas Becker
Hi folks, I'd like to open the discussion for KIP-566: Add rebalance callbacks to ConsumerInterceptor. We've been looking to implement some custom metrics via ConsumerInterceptor, and not knowing when partition ownership changes is a significant impediment. I'd appreciate your thoughts.

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

2020-01-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7737; Use single path in producer for initializing the producerId -- [...truncated 5.76 MB...]

Re: [VOTE] KIP-526: Reduce Producer Metadata Lookups for Large Number of Topics

2020-01-23 Thread Jason Gustafson
Sounds good. +1 from me. On Thu, Jan 23, 2020 at 9:00 AM Brian Byrne wrote: > Thanks Jason, > > I'm in favor of the latter: metadata.max.idle.ms. I agree that describing > it as a "period" is inaccurate. With metadata.max.idle.ms, it also aligns > with metadata.max.age.ms for determining

[jira] [Created] (KAFKA-9470) Hanging test case `testBlockOnRequestCompletionFromStateChangeHandler`

2020-01-23 Thread Jason Gustafson (Jira)
Jason Gustafson created KAFKA-9470: -- Summary: Hanging test case `testBlockOnRequestCompletionFromStateChangeHandler` Key: KAFKA-9470 URL: https://issues.apache.org/jira/browse/KAFKA-9470 Project:

[jira] [Created] (KAFKA-9472) Reducing number of tasks for connector causes deleted tasks to show as UNASSIGNED

2020-01-23 Thread Chris Egerton (Jira)
Chris Egerton created KAFKA-9472: Summary: Reducing number of tasks for connector causes deleted tasks to show as UNASSIGNED Key: KAFKA-9472 URL: https://issues.apache.org/jira/browse/KAFKA-9472

[jira] [Created] (KAFKA-9471) Return empty collection for PENDING_SHUTDOWN

2020-01-23 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9471: - Summary: Return empty collection for PENDING_SHUTDOWN Key: KAFKA-9471 URL: https://issues.apache.org/jira/browse/KAFKA-9471 Project: Kafka Issue Type: Improvement

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

2020-01-23 Thread Bruno Cadonna
Hi Navinder, Thank you for the KIP! It looks good to me. Here my comments: 1) I agree with John and Matthias that you should remove the implementation of the methods in the KIP. Just the method signatures suffice and make the reading easier. 2) According to the grammar

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

2020-01-23 Thread Apache Jenkins Server
See Changes: [github] KAFKA-7737; Use single path in producer for initializing the producerId -- [...truncated 2.83 MB...]

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

2020-01-23 Thread Navinder Brar
Thanks Bruno, for the comments. 1) Fixed. 2) I would be okay to call the variable staleStores. Since anyways we are not using constructor, so the only way the variable is exposed outside is the getter and the optional builder method. With this variable name, we can name the builder method as

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

2020-01-23 Thread Brian Byrne
Thanks Rajini, 1) Good catch, fixed. 2) You're right. We'd need to extend ClientQuotaCallback#quotaLimit or add an alternate function. For the sake of an initial implementation, I'm going to remove '--show-overridden', and a subsequent KIP will have to propose an extents to ClientQuotaCallback

Re: [VOTE] KIP-559: Make the Kafka Protocol Friendlier with L7 Proxies

2020-01-23 Thread Guozhang Wang
Hello David, Thanks for the KIP! I have read through the proposal and had one minor and one meta comment. But overall it looks good to me! 1) The JoinGroupRequest format does not have any new fields proposed, so we could either clarify that it is listed here but without modifications (only

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

2020-01-23 Thread John Roesler
Thanks, Navinder. It's just to give everyone a chance to object if they wanted to. -John On Thu, Jan 23, 2020, at 00:44, Navinder Brar wrote: > Oh sorry, my bad. Will wait for another 12 hours. > > On Thursday, 23 January, 2020, 12:09:57 pm IST, Matthias J. Sax > wrote: > >

Re: [VOTE] KIP-526: Reduce Producer Metadata Lookups for Large Number of Topics

2020-01-23 Thread Brian Byrne
Thanks Jason, I'm in favor of the latter: metadata.max.idle.ms. I agree that describing it as a "period" is inaccurate. With metadata.max.idle.ms, it also aligns with metadata.max.age.ms for determining refresh period (which is an actual period). I've updated the docs. Thanks, Brian On Wed,

Re: Create KIP Permission

2020-01-23 Thread Bill Bejeck
Thomas, you're all set now. -Bill On Thu, Jan 23, 2020 at 10:17 AM Thomas Becker wrote: > I'd like permission to create a KIP please. My confluence account is > twbecker. > > > > > This email and any attachments may contain confidential and privileged >

[jira] [Resolved] (KAFKA-7737) Consolidate InitProducerId API

2020-01-23 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-7737?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-7737. Fix Version/s: 2.5.0 Resolution: Fixed > Consolidate InitProducerId API >

Create KIP Permission

2020-01-23 Thread Thomas Becker
I'd like permission to create a KIP please. My confluence account is twbecker. This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or

Re: [VOTE] KIP-515: Hardened TLS Configs to ZooKeeper

2020-01-23 Thread Ron Dagostino
Hi everyone. I discovered something minor while addressing the AclAuthorizer config inheritance issue that I need to document. The minimum 3 days for voting is up and we could successfully conclude the vote with 3 +1 binding votes and a +1 non-binding vote, but I'll leave the vote open another

[jira] [Created] (KAFKA-9469) Add zookeeper.ssl.context.supplier.class config if/when adopting ZooKeeper 3.6

2020-01-23 Thread Ron Dagostino (Jira)
Ron Dagostino created KAFKA-9469: Summary: Add zookeeper.ssl.context.supplier.class config if/when adopting ZooKeeper 3.6 Key: KAFKA-9469 URL: https://issues.apache.org/jira/browse/KAFKA-9469

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

2020-01-23 Thread Rajini Sivaram
Hi Brian, Thanks for the KIP. Looks good, hope we finally get this in! A few comments: 1) All the Admin interface methods seem to be using method names starting with upper-case letter, should be lower-case to be follow conventions. 2) Effective quotas returns not only the actual effective

[jira] [Resolved] (KAFKA-9464) Close the producer in completeShutdown

2020-01-23 Thread Ted Yu (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9464?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ted Yu resolved KAFKA-9464. --- Resolution: Not A Problem > Close the producer in completeShutdown > --

Re: [DISCUSS] KIP-519: Make SSL context/engine configuration extensible

2020-01-23 Thread Maulin Vasavada
Hi all, I have updated the KIP document with the current state of conclusions. Please review it and see if we are ready to move to Voting! Thanks Maulin On Wed, Jan 22, 2020 at 12:42 AM Maulin Vasavada wrote: > Hi all, > > Finally I squeezed time and I've a suggested code changes shown at >

Re: [DISCUSS] KIP-561: Regex Expressions Support for ConsumerGroupCommand

2020-01-23 Thread Alexander Dunayevsky
Hello guys, Let's discuss KIP-561 Regex Support for ConsumerGroupCommand: https://cwiki.apache.org/confluence/display/KAFKA/KIP-561%3A+Regex+Support+for+ConsumerGroupCommand Functionality already implemented and waiting to be reviewed. Best Regards, Alex Dunayevsky On Thu, 16 Jan 2020, 14:25