Re: [DISCUSS] KIP-524: Allow users to choose config source when describing configs

2019-09-16 Thread Rajini Sivaram
Thanks for the KIP Jason! Good idea. On Sun, Sep 15, 2019 at 5:41 AM Gwen Shapira wrote: > Ah, great idea. > > On Fri, Sep 13, 2019 at 8:47 AM Jason Gustafson > wrote: > > > > Hi All, > > > > I have a minor KIP to improve the config tool: > > > https://cwiki.apache.org/confluence/display/KAFKA/

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

2019-09-16 Thread Rajini Sivaram
ation does not use the standard > >> property for the keystore location. For that particular use case, it > would > >> probably be better to reuse the standard keystore location config and > >> change its semantics to a URL. Regardless, my point is the custom >

[DISCUSS] KIP-525 - Return topic metadata and configs in CreateTopics response

2019-09-13 Thread Rajini Sivaram
Hi all, I would like to start discussion on KIP-525 to return topic configs in CreateTopics response: - https://cwiki.apache.org/confluence/display/KAFKA/KIP-525+-+Return+topic+metadata+and+configs+in+CreateTopics+response When validateOnly=false, this will be the actual configs of the c

[jira] [Created] (KAFKA-8907) Return topic configs in CreateTopics response

2019-09-13 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8907: - Summary: Return topic configs in CreateTopics response Key: KAFKA-8907 URL: https://issues.apache.org/jira/browse/KAFKA-8907 Project: Kafka Issue Type

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

2019-09-12 Thread Rajini Sivaram
nfigure should reach the > SslEngineFactory instance. Am I missing something? > > -Original Message- > From: Rajini Sivaram [mailto:rajinisiva...@gmail.com] > Sent: Thursday, September 12, 2019 12:01 PM > To: dev > Subject: Re: [DISCUSS] KIP-519: Make SSL context/engine con

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

2019-09-12 Thread Rajini Sivaram
changed. On Thu, Sep 12, 2019 at 4:43 PM Rajini Sivaram wrote: > Hi Clement, > > Kafka does special handling for keystore/truststore file changes when an > AlterConfig request is processed, but that is not easy to extend to custom > configs. I was thinking we could just add a c

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

2019-09-12 Thread Rajini Sivaram
config.values() to > > config.originals(), this will affect the contract for the constructor of > > the SslEngineFactory. We might need to add custom configs support to > > KIP-519 or delay KIP-519 until the change to config.originals(). > > > > > > -Original Message--

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

2019-09-11 Thread Rajini Sivaram
Kafka already has the notion of custom configs. And we support reconfigurable custom configs for some interfaces e.g. MetricsReporter. We also recently added custom reconfigurable configs for Authorizer under KIP-504. The issue with custom configs for SSL is described in https://issues.apache.org/

[jira] [Resolved] (KAFKA-8886) Make Authorizer create/delete methods asynchronous

2019-09-11 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8886?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8886. --- Reviewer: Manikumar Resolution: Fixed > Make Authorizer create/delete meth

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-09 Thread Rajini Sivaram
Hi Jun, Thanks for the response. If we use the existing purgatory implementation, we should get additional purgatory metrics for ACL updates with the new purgatory name as tag, consistent with what we have for other delayed operations. I will add these to the KIP. We also have request metrics whic

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-09 Thread Rajini Sivaram
in a short > window. Could we just use CPU throttling to prevent too much resource being > used? There are a few other similar types of requests such as create/alter > configs, create/alter topic, etc. Do we plan to add an extra processing > stage for each of them too in the fu

[jira] [Created] (KAFKA-8887) Use purgatory for CreateAcls and DeleteAcls if implementation is async

2019-09-08 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8887: - Summary: Use purgatory for CreateAcls and DeleteAcls if implementation is async Key: KAFKA-8887 URL: https://issues.apache.org/jira/browse/KAFKA-8887 Project

[jira] [Created] (KAFKA-8886) Make Authorizer create/delete methods asynchronous

2019-09-08 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8886: - Summary: Make Authorizer create/delete methods asynchronous Key: KAFKA-8886 URL: https://issues.apache.org/jira/browse/KAFKA-8886 Project: Kafka Issue

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-08 Thread Rajini Sivaram
> > On Sun, Sep 8, 2019, 9:21 AM Rajini Sivaram > wrote: > > > Hi all, > > > > Thanks everyone for the very useful discussion. I have updated the KIP to > > make *createAcls()* and *deleteAcls()* asynchronous. Also added a section > > under `*Rejected Alterna

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-08 Thread Rajini Sivaram
> I'm on the move, but what Rajini said seems reasonable. I don't > think > > > using SSDs solves the issue. They can still hang for seconds when > > they > > > fail. Also, many people may not have local SSDs available (remote > > SSDs l

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-06 Thread Rajini Sivaram
till hang for seconds when > they > > fail. Also, many people may not have local SSDs available (remote > SSDs like > > EBS hang for tens of seconds when there are issues). > > > > We are currently vulnerable to all of these in the normal read/write &g

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-06 Thread Rajini Sivaram
ng the > > pluggable interface. > > > > Ismael > > > > On Wed, Sep 4, 2019 at 5:29 AM Jun Rao wrote: > > > > > Hi, Rajini, > > > > > > Thanks for the KIP. I was concerned about #4 too. If we change the > > handling >

[jira] [Resolved] (KAFKA-8866) Make Authorizer create/delete exceptions Optional

2019-09-05 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8866?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8866. --- Reviewer: Ismael Juma Resolution: Fixed > Make Authorizer create/delete excepti

[jira] [Resolved] (KAFKA-8857) Config describe should not return isReadOnly=false based on synonyms

2019-09-05 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8857?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8857. --- Reviewer: Manikumar Resolution: Fixed > Config describe should not return isReadO

[jira] [Created] (KAFKA-8866) Make Authorizer create/delete exceptions Optional

2019-09-04 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8866: - Summary: Make Authorizer create/delete exceptions Optional Key: KAFKA-8866 URL: https://issues.apache.org/jira/browse/KAFKA-8866 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-8760) KIP-504: Add new Java Authorizer API

2019-09-04 Thread Rajini Sivaram (Jira)
[ https://issues.apache.org/jira/browse/KAFKA-8760?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8760. --- Resolution: Fixed > KIP-504: Add new Java Authorizer

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

2019-09-04 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8865: - Summary: KIP-504: New Java Authorizer API Key: KAFKA-8865 URL: https://issues.apache.org/jira/browse/KAFKA-8865 Project: Kafka Issue Type: Improvement

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-04 Thread Rajini Sivaram
a of the implementation changes for custom authorizers that want to continue to be synchronous. - KafkaApis.scala continues to use the API synchronously for now. It can be updated later. Thank you, Rajini On Wed, Sep 4, 2019 at 9:38 AM Rajini Sivaram wrote: > Hi Don, > &g

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-04 Thread Rajini Sivaram
econd requests. Not sure whether making it async will add > any unnecessary overhead. > 4. How much complication would this add on the calling side? And is it > worth it? > > Thanks > > Bosco > > > On 9/3/19, 8:50 AM, "Rajini Sivaram" wrote: > > Hi all,

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-09-03 Thread Rajini Sivaram
rds > > Bosco > > > On 8/16/19, 11:30 AM, "Rajini Sivaram" wrote: > > Hi Don, > > That should be fine. I guess Ranger loads policies from the database > synchronously when the authorizer is configured, similar to > SimpleAclAuthorize

[jira] [Created] (KAFKA-8857) Config describe should not return isReadOnly=false based on synonyms

2019-09-02 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8857: - Summary: Config describe should not return isReadOnly=false based on synonyms Key: KAFKA-8857 URL: https://issues.apache.org/jira/browse/KAFKA-8857 Project: Kafka

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-09-02 Thread Rajini Sivaram
y/trust store loader interfaces will be "file based" > loading > > vs > > > if somebody wants to customize any of it they can. > > > > > > Would that make sense? > > > > > > On Fri, Aug 30, 2019 at 10:03 AM Colin McCabe > > wrote: &g

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-30 Thread Rajini Sivaram
vetoed > >> because doing this almost invariably meant the replacement lost all the > >> complex validation code in the default SslFactory. > >> > >> My current idea is to extract the validation code into another public > API > >> that SslFactory woul

Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and TrustStore

2019-08-29 Thread Rajini Sivaram
Hi Maulin, In SSL scenarios, I imagine security providers introduced by KIP-492 are likely to be most useful when you want to use third party providers. The biggest advantage of the config from that KIP is that you don't need to write much code to integrate existing security providers into Kafka b

[jira] [Created] (KAFKA-8848) Update system test to use new authorizer

2019-08-29 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8848: - Summary: Update system test to use new authorizer Key: KAFKA-8848 URL: https://issues.apache.org/jira/browse/KAFKA-8848 Project: Kafka Issue Type: Sub

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

2019-08-29 Thread Rajini Sivaram (Jira)
Rajini Sivaram created KAFKA-8847: - Summary: Deprecate and remove usage of supporting classes in kafka.security.auth Key: KAFKA-8847 URL: https://issues.apache.org/jira/browse/KAFKA-8847 Project

Re: [VOTE] KIP-504 - Add new Java Authorizer Interface

2019-08-27 Thread Rajini Sivaram
> > +1 (binding). > > > > > > > > > > Thanks for the KIP. LGTM. > > > > > > > > > > Regards, > > > > > Manikumar > > > > > > > > > > On Sat, Aug 17, 2019 at 4:41 AM Colin McCabe &

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-16 Thread Rajini Sivaram
up the > listener. This enables different authorization metadata stores for > different listeners." > > Since Ranger uses its own database for storing policies, do you anticipate > any issues? > > Thanks > > Bosco > > > On 8/16/19, 6:49 AM, "Rajini Sivaram&q

[VOTE] KIP-504 - Add new Java Authorizer Interface

2019-08-16 Thread Rajini Sivaram
Hi all, I would like to start the vote for KIP-504: https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface This KIP replaces the Scala Authorizer API with a new Java API similar to other pluggable APIs in the broker and addresses known limitations in the exi

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-16 Thread Rajini Sivaram
deprecated. A new authorizer implementation kafka.security.authorizer.AclAuthorizer will be added for the new API (this will not be in the public package). Please let me know if you have any concerns. Regards, Rajini On Fri, Aug 16, 2019 at 8:48 AM Rajini Sivaram wrote: > Thanks Colin. >

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-16 Thread Rajini Sivaram
t > > > logs. > > > > > `Authorizer#start` is now provided a generic `BrokerInfo` interface > > > that > > > > > gives cluster id, broker id and endpoint information. The generic > > > interface > > > > > can potentially be used in other broker

[jira] [Resolved] (KAFKA-8592) Broker Dynamic Configuration fails to resolve variables as per KIP-421

2019-08-16 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8592?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8592. --- Resolution: Fixed > Broker Dynamic Configuration fails to resolve variables as per KIP-

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-15 Thread Rajini Sivaram
THORIZE = true, false. FILTER = true, false. LIST_AUTHORIZED = > false, false. Would there be anything lost versus having the enum? > > best, > Colin > > > On Wed, Aug 14, 2019, at 06:29, Mickael Maison wrote: > > Hi Rajini, > > > > Thanks for the KIP! >

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-13 Thread Rajini Sivaram
ATORY_AUTHORIZE`. > > Regards, > David > > On Mon, Aug 12, 2019 at 2:54 PM Rajini Sivaram > wrote: > > > Hi David, > > > > Thanks for reviewing the KIP! Since questions about `authorization mode` > > and `count` have come up multip

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-12 Thread Rajini Sivaram
the different values RequestType and Mode. > > > > Thanks > > > > Bosco > > > > > > On 8/9/19, 5:08 AM, "Rajini Sivaram" wrote: > > > > Hi Don, > > > > Thanks for the suggestions. A few responses below: > >

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-09 Thread Rajini Sivaram
zeCount=10`. Or ignore count altogether. Authorizer doesn't know about partitions, so it is more about how many times the resource was accessed in a compound request. On Fri, Aug 9, 2019 at 1:08 PM Rajini Sivaram wrote: > Hi Don, > > Thanks for the suggestions. A few responses below:

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-09 Thread Rajini Sivaram
le to pass this through. > #5 - Assuming if the object instance is going out of scope, we should be > fine. Not a super important ask. Ranger is already catching KILL signal for > clean up. > > Thanks again. These are good enhancements. Appreciate your efforts here. > > Bosco

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-08 Thread Rajini Sivaram
more? How is it used? > 5. Do you feel having "stop" along with "start" be helpful? E.g. In Ranger > we try to optimize the Audit writing by caching the logs for a fixed > interval. But when the Broker terminates, we do a forced flush. Having an > explicit "sto

Re: [DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-07 Thread Rajini Sivaram
it would be good to have a common module outside of > clients > > that only applies to server side interfaces & changes. It looks like we > are > > increasingly in favor of using Java interface for pluggable modules on > the > > broker side. > > > > Thanks, &

[DISCUSS] KIP-504 - Add new Java Authorizer Interface

2019-08-06 Thread Rajini Sivaram
Hi all, I have created a KIP to replace the Scala Authorizer API with a new Java API: - https://cwiki.apache.org/confluence/display/KAFKA/KIP-504+-+Add+new+Java+Authorizer+Interface This is replacement for KIP-50 which was accepted but never merged. Apart from moving to a Java API consist

[jira] [Created] (KAFKA-8760) KIP-504: Add new Java Authorizer API

2019-08-06 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8760: - Summary: KIP-504: Add new Java Authorizer API Key: KAFKA-8760 URL: https://issues.apache.org/jira/browse/KAFKA-8760 Project: Kafka Issue Type: New

Re: Dynamic configuration of interbroker SSL certificates

2019-08-02 Thread Rajini Sivaram
st after a restart is what really sticks out to > me. Is there any state stored outside of Kafka that could be getting in the > way? Does Zookeeper store anything other than the filename of the keystore? > > Cheers, > Michael > > > > > On 1 Aug 2019, at 7:10 pm, Rajini Sivaram

Re: Dynamic configuration of interbroker SSL certificates

2019-08-01 Thread Rajini Sivaram
Hi Michael, Thank you for reporting this. It is possible that we missed out some client-side update in KAFKA-8336. Would it be possible for you to provide logs from multiple brokers around the time of a broker restart when handshake failures started occurring so that we can try to work out which c

[jira] [Resolved] (KAFKA-8688) Upgrade system tests fail due to data loss with older message format

2019-07-31 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8688?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8688. --- Resolution: Fixed Reviewer: Ismael Juma Fix Version/s: 2.4.0 > Upgr

Re: [VOTE] KIP-492 Add java security providers in Kafka Security config

2019-07-30 Thread Rajini Sivaram
+1 (binding) Thanks for the KIP, Sandeep! Regards, Rajini On Tue, Jul 30, 2019 at 4:40 AM Satish Duggana wrote: > +1 (non-binding) > > Thanks, > Satish. > > On Tue, Jul 30, 2019 at 5:18 AM Harsha Chintalapani > wrote: > > > > Thanks for the KIP Sandeep. > > > > +1 (binding) > > > > Thanks, >

Re: Fwd: [DISCUSS] KIP-492 Add java security providers in Kafka Security config

2019-07-26 Thread Rajini Sivaram
SSL only. Given for SASL users can configure a > provider for LoginModule. > Unless there is a usecase where we benefit from having a provider config > for SASL. > > > > On Thu, Jul 25, 2019, at 5:25 AM, Rajini Sivaram wrote: > > Hi Sandeep/Harsha, > > > &

Re: Fwd: [DISCUSS] KIP-492 Add java security providers in Kafka Security config

2019-07-25 Thread Rajini Sivaram
let us know if these addresses your > concerns. > > Thanks, > Harsha > > On Mon, Jul 22, 2019, at 9:36 AM, Sandeep Mopuri wrote: > > Hi Rajini, > > Thanks for raising the above questions. Please find the > > replies below > > > > On Wed, Ju

[jira] [Created] (KAFKA-8688) Upgrade system tests fail due to data loss with older message format

2019-07-19 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8688: - Summary: Upgrade system tests fail due to data loss with older message format Key: KAFKA-8688 URL: https://issues.apache.org/jira/browse/KAFKA-8688 Project: Kafka

Re: [DISCUSS] KIP-492 Add java security providers in Kafka Security config

2019-07-17 Thread Rajini Sivaram
Hi Sandeep, Thanks for the KIP. A few questions below: 1. Is the main use case for this KIP adding security providers for SSL? If so, wouldn't a more generic solution like KIP-383 work for this? 2. Presumably this config would also apply to clients. If so, have we thought through the

[jira] [Resolved] (KAFKA-8662) Produce fails if a previous produce was to an unauthorized topic

2019-07-16 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8662?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8662. --- Resolution: Fixed Reviewer: Jason Gustafson > Produce fails if a previous produce

[jira] [Created] (KAFKA-8662) Produce fails if a previous produce was to an unauthorized topic

2019-07-13 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8662: - Summary: Produce fails if a previous produce was to an unauthorized topic Key: KAFKA-8662 URL: https://issues.apache.org/jira/browse/KAFKA-8662 Project: Kafka

Re: [VOTE] KIP-476: Add Java AdminClient interface

2019-07-11 Thread Rajini Sivaram
+1 (binding) Thanks for the KIP, Andy! Regards, Rajini On Thu, Jul 11, 2019 at 1:18 PM Gwen Shapira wrote: > +1 (binding) > > Thank you for the improvement. > > On Thu, Jul 11, 2019, 3:53 AM Andy Coates wrote: > > > Hi All, > > > > So voting currently stands on: > > > > Binding: > > +1 Matt

Re: [ANNOUNCE] Apache Kafka 2.2.1

2019-06-04 Thread Rajini Sivaram
York Times, Uber, Yelp, and Zalando, among others. > > > > > > A big thank you for the following 30 contributors to this release! > > > > > > Anna Povzner, Arabelle Hou, A. Sophie Blee-Goldman, Bill Bejeck, Bob > > > Barrett, Chris Egerton, Colin Patrick McCabe, Cyrus Va

[jira] [Resolved] (KAFKA-7565) NPE in KafkaConsumer

2019-05-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7565?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7565. --- Resolution: Duplicate Fix Version/s: 2.3.0 > NPE in KafkaConsu

[jira] [Resolved] (KAFKA-8052) Intermittent INVALID_FETCH_SESSION_EPOCH error on FETCH request

2019-05-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8052?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8052. --- Resolution: Fixed Reviewer: Jason Gustafson Fix Version/s: 2.3.0

[jira] [Resolved] (KAFKA-8381) SSL factory for inter-broker listener is broken

2019-05-20 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8381. --- Resolution: Fixed Reviewer: Manikumar > SSL factory for inter-broker listener

[jira] [Created] (KAFKA-8381) SSL factory for inter-broker listener is broken

2019-05-17 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8381: - Summary: SSL factory for inter-broker listener is broken Key: KAFKA-8381 URL: https://issues.apache.org/jira/browse/KAFKA-8381 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-8379) Flaky test KafkaAdminClientTest.testUnreachableBootstrapServer

2019-05-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8379?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8379. --- Resolution: Fixed Reviewer: Ismael Juma > Flaky t

[jira] [Created] (KAFKA-8379) Flaky test KafkaAdminClientTest.testUnreachableBootstrapServer

2019-05-17 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8379: - Summary: Flaky test KafkaAdminClientTest.testUnreachableBootstrapServer Key: KAFKA-8379 URL: https://issues.apache.org/jira/browse/KAFKA-8379 Project: Kafka

[jira] [Resolved] (KAFKA-8336) Enable dynamic update of client-side SSL factory in brokers

2019-05-15 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8336?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8336. --- Resolution: Fixed Reviewer: Manikumar > Enable dynamic update of client-side

Re: [VOTE] KIP-465: Add Consolidated Connector Endpoint to Connect REST API

2019-05-09 Thread Rajini Sivaram
Hi Dan, Thanks for the KIP, +1 (binding) Regards, Rajini On Mon, May 6, 2019 at 8:08 PM Konstantine Karantasis < konstant...@confluent.io> wrote: > Useful and concise KIP > > +1 (non-binding) > > Konstantine > > On Mon, May 6, 2019 at 10:43 AM Randall Hauch wrote: > > > Thanks, Dan. As menti

Re: [VOTE] KIP-458: Connector Client Config Override Policy

2019-05-09 Thread Rajini Sivaram
Hi Magesh, Thanks for the KIP, +1 (binding) Regards, Rajini On Thu, May 9, 2019 at 3:55 PM Randall Hauch wrote: > Nice work, Magesh. > > +1 (binding) > > Randall > > On Wed, May 8, 2019 at 7:22 PM Magesh Nandakumar > wrote: > > > Thanks a lot Chris. So far, the KIP has one non-binding vote

Re: [VOTE] KIP-454: Expansion of the ConnectClusterState interface

2019-05-08 Thread Rajini Sivaram
Hi Chris, Thanks for the KIP, +1 (binding) Regards, Rajini On Wed, May 8, 2019 at 7:28 PM Manikumar wrote: > Hi Chris, > > +1 (binding). Thanks for the KIP. > > Thanks, > Manikumar > > On Tue, May 7, 2019 at 2:55 AM Konstantine Karantasis < > konstant...@confluent.io> wrote: > > > Nice and us

Re: [DISCUSS] KIP-454: Expansion of the ConnectClusterState interface

2019-05-08 Thread Rajini Sivaram
/config, so it seems reasonable to enable REST extensions > to view them as well. > > I hope this addresses your concerns; I'm happy to continue the discussion > if any follow-up is necessary. > > Thanks for your thoughts! > > Cheers, > > Chris > > On Wed,

Re: [DISCUSS] KIP-454: Expansion of the ConnectClusterState interface

2019-05-08 Thread Rajini Sivaram
Hi Chris, Thanks for the KIP, looks good. I have just one question. Can ` ConnectClusterState#connectorConfig()` return any sensitive configs like passwords? Thanks, Rajini On Wed, May 8, 2019 at 1:30 AM Chris Egerton wrote: > Hi all, > > Now that KAFKA-8304 (https://issues.apache.org/jira/b

[jira] [Created] (KAFKA-8336) Enable dynamic update of client-side SSL factory in brokers

2019-05-08 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8336: - Summary: Enable dynamic update of client-side SSL factory in brokers Key: KAFKA-8336 URL: https://issues.apache.org/jira/browse/KAFKA-8336 Project: Kafka

[jira] [Resolved] (KAFKA-8241) Dynamic update of keystore fails on listener without truststore

2019-04-17 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8241. --- Resolution: Fixed Reviewer: Manikumar Fix Version/s: 2.1.2 > Dynamic upd

Re: [DISCUSSION] KIP-421: Automatically resolve external configurations.

2019-04-17 Thread Rajini Sivaram
the password in it. I have > modified it. We basically wanted to show that you cam pass any additional > parameters required by the config provider> > > > 4. Yes all the public config classes (ProducerConfig, ConsumerConfig, > ConnectorConfig etc.) will> > > > > >

[jira] [Created] (KAFKA-8241) Dynamic update of keystore fails on listener without truststore

2019-04-16 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8241: - Summary: Dynamic update of keystore fails on listener without truststore Key: KAFKA-8241 URL: https://issues.apache.org/jira/browse/KAFKA-8241 Project: Kafka

[jira] [Resolved] (KAFKA-8232) Flaky test kafka.admin.TopicCommandWithAdminClientTest.testTopicDeletion

2019-04-15 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8232?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8232. --- Resolution: Fixed Reviewer: Manikumar Fix Version/s: 2.2.1 > Flaky t

[jira] [Created] (KAFKA-8232) Flaky test kafka.admin.TopicCommandWithAdminClientTest.testTopicDeletion

2019-04-14 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8232: - Summary: Flaky test kafka.admin.TopicCommandWithAdminClientTest.testTopicDeletion Key: KAFKA-8232 URL: https://issues.apache.org/jira/browse/KAFKA-8232 Project

Re: [VOTE] KIP-339: Create a new IncrementalAlterConfigs API

2019-04-11 Thread Rajini Sivaram
xample of this-- it often wouldn't make sense to change just one at > once. How about an input like Map> and a > result like: Map? > > best, > Colin > > On Mon, Apr 8, 2019, at 04:48, Rajini Sivaram wrote: > > Hi Colin, > > > > I am not sure the API

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-04-10 Thread Rajini Sivaram
do that. I'll try to get it done in the next few days. > > Although there's been quite a lot of interest this, the vote thread never > got any binding +1, so it's been stuck in limbo for a long time. It would > be great to get this moving again. > > Kind regards, &g

Re: [DISCUSS] KIP-201: Rationalising Policy interfaces

2019-04-09 Thread Rajini Sivaram
Hi Tom, Are you planning to extend this KIP to also include dynamic broker config update (currently covered under AlterConfigPolicy)? May be worth sending another note to make progress on this KIP since it has been around a while and reading through the threads, it looks like there has been a lot

Re: [DISCUSSION] KIP-422: Add support for user/client configuration in the Kafka Admin Client

2019-04-08 Thread Rajini Sivaram
Hi Jun, Yaodong, 21. The proposed approach sounds very hacky. User principals can contain arbitrary characters. So we can't simply split `user1/clients/clientA` into tokens using '/' as delimiter. Internally, we sanitize names before storing in ZK, but the names provided by the user are actual pr

Re: [VOTE] KIP-339: Create a new IncrementalAlterConfigs API

2019-04-08 Thread Rajini Sivaram
Hi Colin, I am not sure the API proposed in the KIP fits with the type of updates we support. The old API with Map fits better and we need to find a way to do something similar while still retaining the old one. Each request should specify a collection of updates for each ConfigResource with resu

[jira] [Resolved] (KAFKA-8190) Keystore update without file change doesn't update SSLContext

2019-04-05 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8190. --- Resolution: Fixed Reviewer: Manikumar Fix Version/s: 2.3.0 > Keystore upd

[jira] [Created] (KAFKA-8190) Keystore update without file change doesn't update SSLContext

2019-04-04 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8190: - Summary: Keystore update without file change doesn't update SSLContext Key: KAFKA-8190 URL: https://issues.apache.org/jira/browse/KAFKA-8190 Project:

[jira] [Resolved] (KAFKA-8123) Flaky Test RequestQuotaTest#testResponseThrottleTimeWhenBothProduceAndRequestQuotasViolated

2019-03-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8123. --- Resolution: Duplicate Fix Version/s: 2.2.1 > Flaky Test > RequestQuo

[jira] [Resolved] (KAFKA-7989) Flaky Test RequestQuotaTest#testResponseThrottleTimeWhenBothFetchAndRequestQuotasViolated

2019-03-21 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7989?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7989. --- Resolution: Fixed Reviewer: Rajini Sivaram > Flaky Test > RequestQuo

Re: [VOTE] 2.2.0 RC2

2019-03-20 Thread Rajini Sivaram
+1 (binding) Verified quick start using kafka_2.11 and kafka_2.12 binaries. Built from source and ran tests. Checked release notes, javadoc and documentation. Thanks for running the release, Matthias! Regards, Rajini On Wed, Mar 20, 2019 at 7:30 PM Bill Bejeck wrote: > +1 (non-binding) > >

Re: [DISCUSS] KIP-398: Support reading trust store from classpath

2019-03-20 Thread Rajini Sivaram
their infra and support File as the default > > > mechanism so that we can support existing users. > > > > > > > > I agree that this is a fairly small change that doesn’t aim to > support > > > all possible mechanisms that one might conceive of. I beli

[jira] [Resolved] (KAFKA-8121) ZooKeeperClient session expiry thread may leave clients behind after close

2019-03-18 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8121?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8121. --- Resolution: Fixed Reviewer: Ismael Juma Fix Version/s: 2.2.1

[jira] [Created] (KAFKA-8121) ZooKeeperClient session expiry thread may leave clients behind after close

2019-03-18 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8121: - Summary: ZooKeeperClient session expiry thread may leave clients behind after close Key: KAFKA-8121 URL: https://issues.apache.org/jira/browse/KAFKA-8121 Project

[jira] [Resolved] (KAFKA-8118) Ensure that tests close ZooKeeper clients since they can impact subsequent tests

2019-03-18 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8118?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8118. --- Resolution: Fixed Reviewer: Ismael Juma > Ensure that tests close ZooKeeper clie

[jira] [Created] (KAFKA-8119) KafkaConfig listener accessors may fail during dynamic update

2019-03-17 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8119: - Summary: KafkaConfig listener accessors may fail during dynamic update Key: KAFKA-8119 URL: https://issues.apache.org/jira/browse/KAFKA-8119 Project: Kafka

[jira] [Created] (KAFKA-8118) Ensure that tests close ZooKeeper clients since they can impact subsequent tests

2019-03-17 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8118: - Summary: Ensure that tests close ZooKeeper clients since they can impact subsequent tests Key: KAFKA-8118 URL: https://issues.apache.org/jira/browse/KAFKA-8118

[jira] [Resolved] (KAFKA-8114) Flaky Test DelegationTokenEndToEndAuthorizationTest#testNoGroupAcl

2019-03-16 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8114. --- Resolution: Fixed Assignee: Manikumar Reviewer: Rajini Sivaram Fix

[jira] [Resolved] (KAFKA-8111) KafkaProducer can't produce data

2019-03-16 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8111?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8111. --- Resolution: Fixed Reviewer: Manikumar Fix Version/s: 2.3.0 > KafkaProdu

[jira] [Created] (KAFKA-8112) Add system test to detect compatibility issues when requests are updated

2019-03-15 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8112: - Summary: Add system test to detect compatibility issues when requests are updated Key: KAFKA-8112 URL: https://issues.apache.org/jira/browse/KAFKA-8112 Project

Re: [DISCUSSION] KIP-421: Support resolving externalized secrets in AbstractConfig

2019-03-14 Thread Rajini Sivaram
g changes. In order to do that, you probably need to > use the KIP-297 interface directly. Which means that maybe we should go > back to the drawing board here, unfortunately. :(> > > > > best,> > > Colin> > > > > > > > > > With regards to on

[jira] [Resolved] (KAFKA-7976) Flaky Test DynamicBrokerReconfigurationTest#testUncleanLeaderElectionEnable

2019-03-12 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-7976?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-7976. --- Resolution: Fixed > Flaky Test DynamicBrokerReconfigurationT

[jira] [Resolved] (KAFKA-8091) Flaky test DynamicBrokerReconfigurationTest#testAddRemoveSaslListener

2019-03-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8091?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8091. --- Resolution: Fixed Reviewer: Manikumar > Flaky test DynamicBrokerReconfigurationT

[jira] [Created] (KAFKA-8091) Flaky test DynamicBrokerReconfigurationTest#testAddRemoveSaslListener

2019-03-11 Thread Rajini Sivaram (JIRA)
Rajini Sivaram created KAFKA-8091: - Summary: Flaky test DynamicBrokerReconfigurationTest#testAddRemoveSaslListener Key: KAFKA-8091 URL: https://issues.apache.org/jira/browse/KAFKA-8091 Project

[jira] [Resolved] (KAFKA-8072) Transient failure in SslSelectorTest.testCloseOldestConnectionWithMultipleStagedReceives

2019-03-11 Thread Rajini Sivaram (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-8072?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Rajini Sivaram resolved KAFKA-8072. --- Resolution: Fixed Fix Version/s: (was: 2.2.1) (was: 2.3.0

<    1   2   3   4   5   6   7   8   9   10   >