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

2020-01-22 Thread Navinder Brar
Oh sorry, my bad. Will wait for another 12 hours. On Thursday, 23 January, 2020, 12:09:57 pm IST, Matthias J. Sax wrote: Navinder, a KIP vote must be open for 72h and cannot be closed earlier. -Matthias On 1/22/20 10:27 PM, Navinder Brar wrote: > Thanks, everyone for voting.

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

2020-01-22 Thread Matthias J. Sax
Navinder, a KIP vote must be open for 72h and cannot be closed earlier. -Matthias On 1/22/20 10:27 PM, Navinder Brar wrote: > Thanks, everyone for voting. KIP-562 has been accepted with binding votes > from John, Matthias, and Guozhang. > > On Thursday, 23 January, 2020, 09:40:07 am IST,

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

2020-01-22 Thread Navinder Brar
Thanks, everyone for voting. KIP-562 has been accepted with binding votes from John, Matthias, and Guozhang. On Thursday, 23 January, 2020, 09:40:07 am IST, Guozhang Wang wrote: +1 (binding) from me as well. On Wed, Jan 22, 2020 at 5:59 PM Matthias J. Sax wrote: > I have a few

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

2020-01-22 Thread Vito Jeng
Thanks to everyone for voting. The KIP-216 has been accepted, with 3 binding votes from Bill, John and Matthias and 1 non-binding votes. --- Vito On Thu, Jan 23, 2020 at 1:46 PM Matthias J. Sax wrote: > +1 (binding) > > -Matthias > > On 1/17/20 9:35 PM, John Roesler wrote: > > Thanks for

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

2020-01-22 Thread Vito Jeng
Got it, thanks Matthias. --- Vito On Thu, Jan 23, 2020 at 1:31 PM Matthias J. Sax wrote: > Thanks Vito. > > I am also ok with either name. Just a personal slight preference, but > not a important. > > > -Matthias > > On 1/21/20 6:52 PM, Vito Jeng wrote: > > Thanks Matthias. > > > > The KIP is

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

2020-01-22 Thread Konstantine Karantasis
I agree Matthias. I also see the metadata stored in the value of the topic status record being extended in the near future to include additional useful information. I'm happy to announce that we met our first deadline and that this KIP has been approved with: 3 binding +1 votes from Randall

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

2020-01-22 Thread Guozhang Wang
+1 (binding) from me as well. On Wed, Jan 22, 2020 at 5:59 PM Matthias J. Sax wrote: > I have a few minor comments (compare the DISCUSS thread), but overall > the KIP looks good. > > +1 (binding) > > > -Matthias > > On 1/22/20 10:09 AM, John Roesler wrote: > > Thanks for updating the KIP,

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

2020-01-22 Thread Navinder Brar
+1 on changing to storeName() and includeStateStores(). We can add this to grammar wiki as Matthias suggested. I have edited the KIP to remove "Deprecating" in favor of "Changing" and I agree we can deprecate store(final String storeName, final  QueryableStoreType queryableStoreType).  Thanks

[jira] [Created] (KAFKA-9468) config.storage.topic partition count issue is hard to debug

2020-01-22 Thread Evelyn Bayes (Jira)
Evelyn Bayes created KAFKA-9468: --- Summary: config.storage.topic partition count issue is hard to debug Key: KAFKA-9468 URL: https://issues.apache.org/jira/browse/KAFKA-9468 Project: Kafka

[jira] [Created] (KAFKA-9467) Multiple wallclock punctuators may be scheduled after a rebalance

2020-01-22 Thread Sophie Blee-Goldman (Jira)
Sophie Blee-Goldman created KAFKA-9467: -- Summary: Multiple wallclock punctuators may be scheduled after a rebalance Key: KAFKA-9467 URL: https://issues.apache.org/jira/browse/KAFKA-9467 Project:

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

2020-01-22 Thread Jason Gustafson
Thanks for the proposal. Looks good overall. I wanted to suggest a possible name change. I was considering something like `idle.metadata.expiration.ms` or maybe `metadata.max.idle.ms`. Thoughts? -Jason On Tue, Jan 21, 2020 at 11:38 AM Guozhang Wang wrote: > Got it. > > I was proposing that we

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

2020-01-22 Thread Apache Jenkins Server
See

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

2020-01-22 Thread Matthias J. Sax
I have a few minor comments (compare the DISCUSS thread), but overall the KIP looks good. +1 (binding) -Matthias On 1/22/20 10:09 AM, John Roesler wrote: > Thanks for updating the KIP, Navinder. > > I'm +1 (binding) on the current proposal > > Thanks, > -John > > On Tue, Jan 21, 2020, at

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

2020-01-22 Thread Matthias J. Sax
Thanks for the clarifications about the getters. I agree that it makes sense to move to the new pattern incrementally. Might be useful to create a Jira (or multiple?) to track this. It's an straight forward change. A nit about the KIP: it should only list the signature but not the full code of

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

2020-01-22 Thread Matthias J. Sax
Thanks Vito. I am also ok with either name. Just a personal slight preference, but not a important. -Matthias On 1/21/20 6:52 PM, Vito Jeng wrote: > Thanks Matthias. > > The KIP is about InvalidStateStoreException. > I pick `StateStoreNotAvailableException` because it may be more intuitive >

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

2020-01-22 Thread Matthias J. Sax
+1 (binding) -Matthias On 1/17/20 9:35 PM, John Roesler wrote: > 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 >>

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

2020-01-22 Thread Matthias J. Sax
Thanks for the KIP. I am not sure how useful the timestamp and taskId information will be in practice, but I don't have any concern with regard to overhead/performance. Hence, as you think it might be useful, I trust your judgement. For the timestamp though, I would like to emphasize that I

[jira] [Resolved] (KAFKA-9082) Move ConfigCommand to use KafkaAdminClient APIs

2020-01-22 Thread Brian Byrne (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9082?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brian Byrne resolved KAFKA-9082. Resolution: Duplicate The outstanding work to be completed is now identical to KAFKA-7740. Marking

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

2020-01-22 Thread Apache Jenkins Server
See Changes: [jason] KAFKA-9418; Add new sendOffsetsToTransaction API to KafkaProducer -- [...truncated 2.83 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

[jira] [Created] (KAFKA-9466) Add documentation for new stream EOS change

2020-01-22 Thread Boyang Chen (Jira)
Boyang Chen created KAFKA-9466: -- Summary: Add documentation for new stream EOS change Key: KAFKA-9466 URL: https://issues.apache.org/jira/browse/KAFKA-9466 Project: Kafka Issue Type: Sub-task

Re: [DISCUSS] KIP-513: Distinguish between Key and Value serdes in scala wrapper library for kafka streams

2020-01-22 Thread Михаил Ерёменко
Hi, John! Sorry for the late reply. I am not really familiar with this mail list discussions, so I have not seen your mails. Regarding your question: > I guess what I'm struggling with is why you actually want to have different key and serdes for the same type I think good example will

[jira] [Created] (KAFKA-9465) Enclose consumer call with catching InvalidOffsetException

2020-01-22 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9465: - Summary: Enclose consumer call with catching InvalidOffsetException Key: KAFKA-9465 URL: https://issues.apache.org/jira/browse/KAFKA-9465 Project: Kafka Issue Type:

[jira] [Resolved] (KAFKA-9418) Add new sendOffsets API to include consumer group metadata

2020-01-22 Thread Jason Gustafson (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-9418?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson resolved KAFKA-9418. Fix Version/s: 2.5.0 Resolution: Fixed > Add new sendOffsets API to include

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

2020-01-22 Thread Ted Yu (Jira)
Ted Yu created KAFKA-9464: - Summary: Close the producer in completeShutdown Key: KAFKA-9464 URL: https://issues.apache.org/jira/browse/KAFKA-9464 Project: Kafka Issue Type: Bug Reporter:

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

2020-01-22 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-7273 Clarification on mutability of headers passed to -- [...truncated 5.72 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

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

2020-01-22 Thread Brian Byrne
Hi Jason, I agree on (1). It was Colin's original suggestion, too, but he had changed his mind in preference for enums. Strings are the more generic way for now, so hopefully Colin can share his thinking when he's back. The QuotaFilter usage was an error, this has been corrected. For (2), the

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

2020-01-22 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-7273 Clarification on mutability of headers passed to -- [...truncated 2.84 MB...] org.apache.kafka.streams.TopologyTestDriverTest >

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

2020-01-22 Thread John Roesler
Thanks for updating the KIP, Navinder. I'm +1 (binding) on the current proposal Thanks, -John On Tue, Jan 21, 2020, at 12:50, Navinder Brar wrote: > Thanks, Guozhang. I agree it makes total sense. I will make the > edits.~Navinder  > > On Tuesday, 21 January, 2020, 11:00:32 pm IST,

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

2020-01-22 Thread John Roesler
Thanks Navinder! I've also updated the motivation. Thanks, -John On Wed, Jan 22, 2020, at 11:12, Navinder Brar wrote: > I went through the grammar wiki page and since it is already agreed in > principle I will change from constructor to below method and add the > getters back. > public static

Re: [DISCUSS] KIP-553: Disable all SSL protocols except TLSV1.2 by default.

2020-01-22 Thread Николай Ижиков
Hello, Rajini. PR - https://github.com/apache/kafka/pull/7998 Please, review. > 22 янв. 2020 г., в 14:28, Николай Ижиков написал(а): > > Yes, I will do it the next few hours. > >> 22 янв. 2020 г., в 14:24, Rajini Sivaram >> написал(а): >> >> Hi Nikolay, >> >> Do you have time to submit a

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

2020-01-22 Thread Navinder Brar
I went through the grammar wiki page and since it is already agreed in principle I will change from constructor to below method and add the getters back. public static StoreQueryParams fromNameAndType(   final String storeName,   final QueryableStoreType  queryableStoreType ) Thanks, Navinder

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

2020-01-22 Thread Eno Thereska
This is awesome! +1 (non binding) Eno On Tue, Jan 21, 2020 at 10:00 PM Gwen Shapira wrote: > > Thank you for the KIP. Awesomely cloud-native improvement :) > > +1 (binding) > > > On Tue, Jan 21, 2020, 9:35 AM David Jacot wrote: > > > Hi all, > > > > I would like to start a vote on KIP-559: Make

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

2020-01-22 Thread John Roesler
22) I'm specifically proposing to establish a new convention. The existing convention is fundamentally broken and has been costly both for users and maintainers. That is the purpose of the grammar I proposed. The plan is to implement new APIs following the grammar and gradually to port old APIs

Re: [EXTERNAL] Re: Enable both SASL & SSL authentication...

2020-01-22 Thread Ron Dagostino
Hi Senthil. Yes, you should read KIP-368: Allow SASL Connections to Periodically Re-Authenticate (https://cwiki.apache.org/confluence/display/KAFKA/KIP-368%3A+Allow+SASL+Connections+to+Periodically+Re-Authenticate). This KIP was added in AK 2.2 and addresses your question about re-authentication.

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

2020-01-22 Thread Navinder Brar
10) Sure John, please go ahead. 21) I have no strong opinion on constructor vs static factory. If everyone's okay with it, I can make the change. 22) I looked at classes suggested by Matthias and I see there are no getters there. We are ok with breaking the convention? Thanks,Navinder Pal

RE: [EXTERNAL] Re: Enable both SASL & SSL authentication...

2020-01-22 Thread Senthilnathan Muthusamy
Hi Ron, Thanks for the details and this answers my question (i.e. we can have 2 listeners - 1 with SASL_SSL and another with SSL to achieve this). Another question related to oAuth token revoke scenario. Say once broker authenticated the presented oAuth token and if is valid for 24 hours.

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

2020-01-22 Thread John Roesler
Hi all, 10) For the motivation, I have some thoughts for why this KIP is absolutely essential as designed. If it's ok with you, Navinder, I'd just edit the motivation section of the wiki? If you're unhappy with my wording, you're of course welcome to revert or revise it; it just seems more

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

2020-01-22 Thread Ron Dagostino
Hi everyone. While finishing the PR for this KIP I realized that the inheritance of TLS ZooKeeper configs that happens in the *authorizer* does not reflect he spirit of our discussion. In particular, based on our inheritance discussion in the DISCUSS thread, the inheritance of authorizer configs

Re: Enable both SASL & SSL authentication...

2020-01-22 Thread Ron Dagostino
<<< some of our clients uses oAuth and some uses cert based auth Hi Senthil. Brokers support different clients using different types of authentication, so there is no problem here. The way it works is via the broker's listener -- each one listens on a separate port and is either a SSL listener

Re: [VOTE] KIP-553: Disable all SSL protocols except TLSV1.2 by default.

2020-01-22 Thread M. Manna
+1 (binding). A simple, and yet powerful enforcement of TLS version. Thanks for this KIP :) On Tue, 21 Jan 2020 at 20:39, Mickael Maison wrote: > +1 (binding) > Thanks > > On Tue, Jan 21, 2020 at 7:58 PM Ron Dagostino wrote: > > > > +1 (non-binding) > > > > Ron > > > > On Tue, Jan 21, 2020 at

Re: [DISCUSS] KIP-553: Disable all SSL protocols except TLSV1.2 by default.

2020-01-22 Thread Николай Ижиков
Yes, I will do it the next few hours. > 22 янв. 2020 г., в 14:24, Rajini Sivaram написал(а): > > Hi Nikolay, > > Do you have time to submit a PR for this before 2.5.0 feature freeze on Jan > 29th? > > On Tue, Jan 21, 2020 at 1:09 PM Ron Dagostino wrote: > >> Sure, go for it. >> >>> On Jan

Re: [DISCUSS] KIP-553: Disable all SSL protocols except TLSV1.2 by default.

2020-01-22 Thread Rajini Sivaram
Hi Nikolay, Do you have time to submit a PR for this before 2.5.0 feature freeze on Jan 29th? On Tue, Jan 21, 2020 at 1:09 PM Ron Dagostino wrote: > Sure, go for it. > > > On Jan 21, 2020, at 8:05 AM, Николай Ижиков wrote: > > > > Hello, Ron. > > > > Let’s start vote right now. > > What do

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

2020-01-22 Thread Navinder Brar
Thanks Matthias for the feedback. 10) As Guozhang suggested above, we thought of adding storeName and queryableStoreType as well in the StoreQueryParams, which is another motivation for this KIP as it overloads KafkaStreams#store(). I have updated the motivation in the KIP as well. 20) I

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

2020-01-22 Thread Apache Jenkins Server
See Changes: [rhauch] KAFKA-9143: Log task reconfiguration error only when it happened (#7648) -- [...truncated 5.76 MB...]

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

2020-01-22 Thread Maulin Vasavada
Hi all, Finally I squeezed time and I've a suggested code changes shown at https://github.com/maulin-vasavada/kafka/pull/4/files for discussing this further. I'll update the KIP document soon. Meanwhile, can you please take a look and continue the discussion? One challenge is at: