Re: [VOTE] KIP-881: Rack-aware Partition Assignment for Kafka Consumers

2022-11-15 Thread Maulin Vasavada
+1 (non-binding). Makes sense, Rajini. This would be a great addition. On Tue, Nov 15, 2022 at 10:55 AM Rajini Sivaram wrote: > Hi all, > > I would like to start vote on KIP-881 to support rack-aware partition > assignment for Kafka consumers: > >

Re: [DISCUSS] KIP-651 - Support PEM format for SSL certificates and

2022-03-22 Thread Maulin Vasavada
just a clarification on the part "... we are not left with any choice but to use the file based config .." - That is "if" we want material rotations to be picked up automatically. On Tue, Mar 22, 2022 at 1:57 PM Maulin Vasavada wrote: > Hi Dejan and Rajini > &

Re: [DISCUSS] KIP-651 - Support PEM format for SSL certificates and

2022-03-22 Thread Maulin Vasavada
Hi Dejan and Rajini I tend to agree with both the arguments when I read them in isolation :) However, what I feel is - we have to also take an operator's view of this setup. If we have encrypted keys/certs already it may be easier to fit with file based material but if we don't have material

Re: [VOTE] KIP-651 - Support PEM format for SSL certificates and private key

2020-08-06 Thread Maulin Vasavada
+1 (non-binding) On Thu, Aug 6, 2020 at 11:36 AM Ron Dagostino wrote: > +1 (non-binding) > > Ron > > > On Aug 6, 2020, at 1:42 PM, Rajini Sivaram > wrote: > > > > Hi all, > > > > I would like to start vote on KIP-651 to support SSL key stores and trust > > stores in PEM format: > > > > - > >

Re: [DISCUSS] KIP-651 - Support PEM format for SSL certificates and private key

2020-08-06 Thread Maulin Vasavada
Looks great Rajini. Indeed PEM is a standard needed to be supported in a first class way on Kafka. On Wed, Aug 5, 2020 at 9:55 PM Manikumar wrote: > Thanks for the KIP. LGTM. > > On Wed, Aug 5, 2020 at 3:43 AM Ron Dagostino wrote: > > > Yes, Rajinit, it looks good -- an excellently-written

Re: Preliminary blog post about the Apache 2.6.0 release

2020-08-05 Thread Maulin Vasavada
t that easy. > > I chose not to include KIP-519 in the blog simply because it requires > installing and integration into the broker components that are not included > in the official distribution. I hope that helps explain my thought process. > > Best regards, > > Randall > > On W

Re: Preliminary blog post about the Apache 2.6.0 release

2020-08-05 Thread Maulin Vasavada
Hi Randall One question: Do we mention all KIPs/NewFeatures in the blog that are listed in the release notes document - https://home.apache.org/~rhauch/kafka-2.6.0-rc2/RELEASE_NOTES.html ? I see that [KAFKA-8890 ] - KIP-519: Make SSL

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-06-10 Thread Maulin Vasavada
types. The interface wasn't public before anyways. > > > > > > > > Given that, my small request would be to rephrase in the > compatibility > > > > section to say something like: > > > > 'Existing MessageFormatters implementations will require no changes > >

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-06-01 Thread Maulin Vasavada
se (although KIP is accepted). We plan to include > it > > in a next release. > > > > > > Cheers, > > Kowshik > > > > > > On Fri, May 29, 2020 at 11:43 AM Maulin Vasavada < > > maulin.vasav...@gmail.com> > > wrote: > > > &

Re: [DISCUSS] Apache Kafka 2.6.0 release

2020-05-29 Thread Maulin Vasavada
Hi Randall Hauch Can we add KIP-519 to 2.6? It was merged to Trunk already in April - https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952. Thanks Maulin On Fri, May 29, 2020 at 11:01 AM Randall Hauch wrote: > Here's an update on the AK 2.6.0 release. > > Code freeze was

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2020-05-20 Thread Maulin Vasavada
Thank you for the KIP. I sincerely hope we get enough votes on this KIP. I was thinking of similar changes while working on DR capabilities and offsets are Achilles Heels and this KIP addresses it. On Mon, May 18, 2020 at 6:10 PM Maulin Vasavada wrote: > +1 (non-binding) > > On Mo

Re: [VOTE] KIP-545 support automated consumer offset sync across clusters in MM 2.0

2020-05-18 Thread Maulin Vasavada
+1 (non-binding) On Mon, May 18, 2020 at 9:41 AM Ryanne Dolan wrote: > Bump. Looks like we've got 6 non-binding votes and 1 binding. > > On Thu, Feb 20, 2020 at 11:25 AM Ning Zhang > wrote: > > > Hello committers, > > > > I am the author of the KIP-545 and if we still miss votes from the > >

Re: [VOTE] KIP-597: MirrorMaker2 internal topics Formatters

2020-05-18 Thread Maulin Vasavada
+1 (non-binding) On Mon, May 18, 2020 at 8:49 AM Mickael Maison wrote: > Bumping this thread as KIP freeze is approaching. > > It's a pretty small change and I have a PR ready: > https://github.com/apache/kafka/pull/8604 > > Thanks > > On Mon, May 4, 2020 at 5:26 PM Ryanne Dolan wrote: > > > >

Re: [DISCUSS] KIP-317 - Add end-to-end data encryption functionality to Apache Kafka

2020-05-09 Thread Maulin Vasavada
Hi Sonke Thanks for bringing this for discussion. There are lot of considerations even if we assume we have end-to-end encryption done. Example depending upon company's setup there could be restrictions on how/which encryption keys are shared. Environment could have multiple security and network

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

2020-04-01 Thread Maulin Vasavada
HI all The summary of the vote: *Total 6 +ve* with *+4 Binding*, +2 non-Binding. I'll move the KIP to Accepted state. Thanks Maulin On Tue, Mar 31, 2020 at 10:32 PM Maulin Vasavada wrote: > Thanks Rajini and Jun. I'll update the shouldBeRebuilt() docs on what > happens to existi

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

2020-03-31 Thread Maulin Vasavada
ses. If we want > > to move the new interface into an SSL-specific package, we should perhaps > > create a new public package rather than use an existing internal one? > > > > On Tue, Mar 31, 2020 at 7:56 AM Manikumar > > wrote: > > > > > +1 (binding).

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

2020-03-30 Thread Maulin Vasavada
Hi all, So far we got 3 Binding votes. I am planning to keep the voting phase open until Tuesday 10 PM Pacific Time which will be more than 72 hours from the first binding vote on Thursday 12:36 PM Pacific Time. Thanks Maulin On Mon, Mar 30, 2020 at 10:32 PM Maulin Vasavada wrote: > Hi

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

2020-03-30 Thread Maulin Vasavada
Hi all, I updated the Javadoc in the KIP details and the actual SslEngineFactory interface for shouldBeRebuilt(). For the first comment, probably I'll try to address it tomorrow. Thanks Maulin On Mon, Mar 30, 2020 at 7:44 PM Maulin Vasavada wrote: > Thanks Jun Rao for your vote and comme

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

2020-03-30 Thread Maulin Vasavada
curity.ssl instead > of org.apache.kafka.common.security.auth? > 2. Could you document when shouldBeRebuilt() will be called? > > Jun > > On Mon, Mar 30, 2020 at 5:07 PM Maulin Vasavada > > wrote: > > > ^^^ bump ^^^ The vote is open for 2-3 days and gotten 1 Binding vote so > > far, can you pl

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

2020-03-30 Thread Maulin Vasavada
;Rajini Sivaram" wrote: > > +1 (binding) > Thanks for the KIP, Maulin! > > Regards, > > Rajini > > On Thu, Mar 26, 2020 at 4:14 PM Maulin Vasavada < > maulin.vasav...@gmail.com> > wrote: > > > FYI - we have updated the K

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

2020-03-26 Thread Maulin Vasavada
FYI - we have updated the KIP documentation also with appropriate code samples for interfaces and few important changes. Thanks Maulin On Wed, Mar 25, 2020 at 10:21 AM Maulin Vasavada wrote: > bump > > On Wed, Mar 25, 2020 at 10:20 AM Maulin Vasavada < > maulin.vasav...@g

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

2020-03-25 Thread Maulin Vasavada
bump On Wed, Mar 25, 2020 at 10:20 AM Maulin Vasavada wrote: > Hi all > > After much await on the approach conclusion we have a PR > https://github.com/apache/kafka/pull/8338. > > Can you please provide your vote so that we can more this forward? > > Thanks > Maul

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

2020-03-25 Thread Maulin Vasavada
Hi all After much await on the approach conclusion we have a PR https://github.com/apache/kafka/pull/8338. Can you please provide your vote so that we can more this forward? Thanks Maulin On Sun, Jan 26, 2020 at 11:03 PM Maulin Vasavada wrote: > Hi all > > After a good discussion o

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

2020-03-24 Thread Maulin Vasavada
ulin On Mon, Mar 23, 2020 at 6:57 PM Maulin Vasavada wrote: > Hi Rajini > > When I configure the Default value for ssl.engine.factory.class as Type > Class, it is resulting into lot of test cases failures since in many places > - tests and actual classes/scala code it is converting the c

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

2020-03-23 Thread Maulin Vasavada
is *not* optional - it needs a value by default. Will keep you posted. I am thinking of reverting the config type to String and then load it as String and do Class loading in SslFactory. Thanks Maulin On Mon, Mar 23, 2020 at 1:38 AM Maulin Vasavada wrote: > still working on the pull request. hopefu

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

2020-03-23 Thread Maulin Vasavada
still working on the pull request. hopefully will be done soon. On Wed, Mar 11, 2020 at 11:48 AM Maulin Vasavada wrote: > Thanks Rajini. Sounds good. I'll make changes and update this thread. > > On Wed, Mar 11, 2020 at 6:41 AM Rajini Sivaram > wrote: > >> Hi Maulin,

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

2020-03-11 Thread Maulin Vasavada
; > We pass all configs to other plugins, so we can do the same here. That > would be safer than assuming that all custom SSL-related configs start with > `ssl.`. You can use principal builder as an example for what we do today. > > Regards, > > Rajini > > On Thu, Mar 5, 2

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

2020-03-04 Thread Maulin Vasavada
Hi Rajini I made changes suggested by you on https://github.com/maulin-vasavada/kafka/pull/4. Please check. In particular I had challenge in 'SslFactory#configure()' method the first time to know which configs I have to add without having actual SslEngineFactory impl. I think it is best to just

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

2020-02-05 Thread Maulin Vasavada
Hi Rajini Will accommodate your comments. Celement, while SSLContext factories are common, in this particular case, we need SSLEngine object because Kafka is using SSLEngine (as mentioned much before in this email thread, the SSLContext acts as factory for getting SSLEngine, SSLSocketFactory or

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

2020-01-26 Thread Maulin Vasavada
Hi all After a good discussion on the KIP at https://www.mail-archive.com/dev@kafka.apache.org/msg101011.html I think we are ready to start voting. KIP: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952 The KIP proposes - Making SSLEngine creation pluggable to support

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

2020-01-26 Thread Maulin Vasavada
Hi all I will start the voting thread on this now. Thanks Maulin On Thu, Jan 23, 2020 at 12:51 AM Maulin Vasavada wrote: > 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! > &g

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 chan

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: https

Re: [ANNOUNCE] New committer: Mickael Maison

2019-11-07 Thread Maulin Vasavada
Congratulations Mickael! On Thu, Nov 7, 2019 at 8:27 PM Manikumar wrote: > Congrats Mickeal! > > On Fri, Nov 8, 2019 at 9:05 AM Dong Lin wrote: > > > Congratulations Mickael! > > > > On Thu, Nov 7, 2019 at 1:38 PM Jun Rao wrote: > > > > > Hi, Everyone, > > > > > > The PMC of Apache Kafka is

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

2019-10-23 Thread Maulin Vasavada
bump! Clement/Rajini? Any responses based on the latest posts? On Wed, Oct 16, 2019 at 10:58 PM Maulin Vasavada wrote: > bump! > > On Sun, Oct 13, 2019 at 11:16 PM Maulin Vasavada < > maulin.vasav...@gmail.com> wrote: > >> Hi Clement >> >> 1) existing va

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

2019-10-16 Thread Maulin Vasavada
bump! On Sun, Oct 13, 2019 at 11:16 PM Maulin Vasavada wrote: > Hi Clement > > 1) existing validation code will remain in SslFactory > 2) the createEngine() method in SslEngineBuilder will move to SslFactory > and the client/server mode setting will go there (I documented this

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

2019-10-14 Thread Maulin Vasavada
ontext > previously created elsewhere in the application. Can your new proposal > handle this use case? > > -----Original Message- > From: Maulin Vasavada [mailto:maulin.vasav...@gmail.com] > Sent: Friday, October 11, 2019 2:13 AM > To: dev@kafka.apache.org > Subject: Re: [DISC

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

2019-10-11 Thread Maulin Vasavada
Kafka's SslEngineBuilder is doing right now. On Thu, Oct 10, 2019 at 11:06 PM Maulin Vasavada wrote: > You meant JSSE not JCE right? We are not talking about cryptographic > providers we are talking about ssl providers hence JSSE. > > I do understand how JSSE Providers work and als

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

2019-10-11 Thread Maulin Vasavada
va/javase/tech/getcodesigningcertificate-361306.html > JCE will refuse to load your provider if it is not signed. Getting the > certificate is a pain and it takes time. You also have to worry about the > certificate expiration date. There are JVMs that don't require signed JCE > provider

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

2019-10-04 Thread Maulin Vasavada
random config and help creates SSLEngine, SocketFactories via the TLS algorithm's provider specified by Security Provider configuration. Thanks Maulin On Fri, Oct 4, 2019 at 10:28 AM Maulin Vasavada wrote: > Hi Clement > > The explanation about using Java Security Providers i

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

2019-10-04 Thread Maulin Vasavada
SSLEngine > wrap/unwrap methods which are the heart of it. That urges me to think more > that actually we need SSLContext to be pluggable. > > Either way, point of discussions about reconfigurability and questions > Clement asked remains similar BUT I think we might have to first really > resolve

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

2019-10-04 Thread Maulin Vasavada
9 at 10:56 PM Maulin Vasavada wrote: > Ack. I should be able to get back to this on Friday. > > On Mon, Sep 23, 2019 at 10:35 AM Pellerin, Clement < > clement_pelle...@ibi.com> wrote: > >> When I worked on KIP-383 I was told the way to pass extra arguments to an

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

2019-09-25 Thread Maulin Vasavada
Ack. I should be able to get back to this on Friday. On Mon, Sep 23, 2019 at 10:35 AM Pellerin, Clement wrote: > When I worked on KIP-383 I was told the way to pass extra arguments to an > instance is to add extra arguments to configure. I would now suggest we do > like the KeySerializer. If

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

2019-09-23 Thread Maulin Vasavada
> > The other custom reconfigurable extension points don't seem to have a > problem with that. You may have a point though, so I need to look at the > reconfiguration code you mention. > > Is your latest prototype available so I can study it? > > -Original Message- &g

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

2019-09-20 Thread Maulin Vasavada
e. > We will encounter issues when we try to prototype it that way. > We will solve the issues or backtrack to another solution. > We must make an attempt, if only to explain why it fails in the Rejected > Alternatives section of the KIP. > > -----Original Message- >

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

2019-09-20 Thread Maulin Vasavada
in certain way. For that, I should not be penalized by Kafka to know all about Java security Providers and how to really create SSLContext object etc given Java already provides a way to feed in KeyStore object regardless of how I load it. On Thu, Sep 19, 2019 at 10:57 PM Maulin Vasavada wrote: &g

Re: [VOTE] KIP-517: Add consumer metrics to observe user poll behavior

2019-09-20 Thread Maulin Vasavada
+1 (non-binding). Thanks for the KIP. On Thu, Sep 19, 2019 at 10:38 PM Manikumar wrote: > +1 (binding), Thanks for the KIP. > > On Fri, Sep 20, 2019 at 12:41 AM Harsha Chintalapani > wrote: > > > +1 (binding). Thanks for the KIP. > > > > -Harsha > > > > On Wed, Sep 18, 2019 at 9:07 AM Mickael

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

2019-09-19 Thread Maulin Vasavada
this proposal address all the reasons you had not to do exactly what > other extension points are doing? > > -Original Message- > From: Maulin Vasavada [mailto:maulin.vasav...@gmail.com] > Sent: Thursday, September 19, 2019 10:21 PM > To: dev@kafka.apache.org > Subject: Re: [D

Re: [ DISCUSS ] KIP-512:Adding headers to RecordMetaData

2019-09-19 Thread Maulin Vasavada
Hi all, Can this KIP get to voting? I don't see any major concerns so far. It is already adding ProducerRecord in Producer Interceptor which was one of the main ask. Thanks Maulin On Mon, Sep 16, 2019 at 4:09 PM Renuka M wrote: > Hi All, > > The motivation behind this KIP is have info about

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

2019-09-19 Thread Maulin Vasavada
you please provide your input on my detailed post along with this recent point I am making? Thanks Maulin On Thu, Sep 19, 2019 at 5:04 PM Maulin Vasavada wrote: > Hi Clement, > > Thanks for pointing to AbstractConfig. Now I understand what you were > saying. I'll respond by tonig

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

2019-09-19 Thread Maulin Vasavada
nce the (Re)Configurable interface dictates the signature of the > configure() method, that forces the addition of a new init(...) method to > pass the other constructor arguments. > > Do we agree on that before we move on to other issues? > > -Original Message- > From: Maul

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

2019-09-18 Thread Maulin Vasavada
gt; need to fuss about its API. > > -Original Message- > From: Maulin Vasavada [mailto:maulin.vasav...@gmail.com] > Sent: Tuesday, September 17, 2019 2:28 AM > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-519: Make SSL context/engine configuration > extensible

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

2019-09-17 Thread Maulin Vasavada
e, it would all just work. We can easily find a different > > way > >of continuing to reconfigure SslFactory without config changes if we > > needed to since it is not a pluggable class. > > > > C) *Triggering reconfiguration:* > > > >- We conti

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

2019-09-14 Thread Maulin Vasavada
that MetricReporter interface defines init() and extends Configurable. Please clarify if I am missing something. Thanks Maulin On Fri, Sep 13, 2019 at 12:15 PM Maulin Vasavada wrote: > Hi Clement/Rajini > > I've gone through the code to understand how reconfigruation work &g

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

2019-09-13 Thread Maulin Vasavada
on/kafka/server/DynamicBrokerReconfigurationTest.scala#L792 > > > > > > > > > > > > On Thu, Sep 12, 2019 at 3:24 PM Pellerin, Clement < > > > clement_pelle...@ibi.com> wrote: > > > > > >> For the push notification, Rajini prefe

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

2019-09-12 Thread Maulin Vasavada
ective the next time it > > is called. SslFactory would not cache the SslEngine and always ask > > SslEngineFactory for it. You don't even need an inner thread if > > SslEngineFactory checks for a change when it is called. SslEngineFactory > > would no longer be immutable, so maybe

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

2019-09-11 Thread Maulin Vasavada
Hi all, Since the "custom config" seems the main topic of interest let us talk about it. 1. I want to confirm that I interpret the definition of 'custom config of SslEngineFactory' the same way Clement is suggesting - "a config that does not exist in Kafka but is specified by a custom

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

2019-09-10 Thread Maulin Vasavada
Hi Clement/Rajini/Colin Please review our latest updates on the KIP and let me know your thoughts. Clement, please let me know if my understanding about the "custom configs" is correct based on what I wrote in the KIP. Thanks Maulin On Mon, Sep 9, 2019 at 3:28 PM Maulin Vasavada wr

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

2019-09-09 Thread Maulin Vasavada
complex validation code and make it > mandatory. > This is also the place to argue against KIP-383. Hint: because it does not > handle reconfiguration in the presence of custom configs. > > When I wrote KIP-383, I felt I needed a prototype before I could solidify > the proposal. >

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

2019-09-09 Thread Maulin Vasavada
on that, we will update the KIP-486 appropriately. Thanks Maulin On Sun, Sep 8, 2019 at 11:08 PM Maulin Vasavada wrote: > Thank you Clement and Colin for shading light on the process for me. I > will probably start working on a new kip for pluggable SslEngine. > > Side note: The pull reques

[jira] [Created] (KAFKA-8890) KIP-519: Make SSL context/engine configuration extensible

2019-09-09 Thread Maulin Vasavada (Jira)
Maulin Vasavada created KAFKA-8890: -- Summary: KIP-519: Make SSL context/engine configuration extensible Key: KAFKA-8890 URL: https://issues.apache.org/jira/browse/KAFKA-8890 Project: Kafka

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

2019-09-09 Thread Maulin Vasavada
Hi all Based on longer discussion on another KIP-486 we are opening this KIP-519 ( https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=128650952). Please help us review this and provide your suggestions. Thanks Maulin

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

2019-09-09 Thread Maulin Vasavada
it in a KIP and that KIP needs to pass a vote. Any votes before the > KIP > > > vote starts is meaningless. > > > > > > As for the ownership and authorship of the KIPs, I don't plan to work > > > on this, so KIP-383 is better left the way it is. I would p

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

2019-09-06 Thread Maulin Vasavada
Hi all Any input on my last question about the process? Also, I have updated the code based on the feedback so far at: https://github.com/maulin-vasavada/kafka/pull/2/files. Still have to figure out how to plug keys/certs loading while using DefaultSslEngineFactory's implementation (still need

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

2019-09-05 Thread Maulin Vasavada
Pluggable key/trust store on top of making SslEngineBuilder pluggable we will need changes suggested by KIP-486 with some differences to the original proposal. It would great if someone can help us clarify the next steps. Thanks Maulin On Wed, Sep 4, 2019 at 1:54 PM Maulin Vasavada wrote: > Do

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

2019-09-04 Thread Maulin Vasavada
Do you guys think it would be easier if you can provide comments on GitHub and we can continue there and summarize the conclusion here? We should not lose addressing any comments. On Wed, Sep 4, 2019 at 12:34 PM Pellerin, Clement wrote: > The proposed interface does not look like the Builder

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

2019-09-04 Thread Maulin Vasavada
SslEngineBuilder Interface somehow take those? On Wed, Sep 4, 2019 at 10:58 AM Colin McCabe wrote: > On Tue, Sep 3, 2019, at 22:56, Maulin Vasavada wrote: > > Hi all > > > > Please check > > > https://github.com/maulin-vasavada/kafka/commit/44f86395b1

Re: [VOTE] KIP-512:Adding headers to RecordMetaData

2019-09-04 Thread Maulin Vasavada
+1 (non-binding) On Tue, Sep 3, 2019 at 3:38 PM Renuka M wrote: > Hi All, > > After good discussion for KIP-512 > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-512%3AAdding+headers+to+RecordMetaData > , > am starting thread for voting. > > Thanks > Renuka M >

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

2019-09-03 Thread Maulin Vasavada
Hi all Please check https://github.com/maulin-vasavada/kafka/commit/44f86395b1ba3fe4bd87de89029d72da77995ff8 This is just the first cut obviously. There are few call outs I would like to make, 1. So far I kept the old SslEngineBuilder hence I had to name the interface with "I" (that

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

2019-08-30 Thread Maulin Vasavada
lEngineBuilder > factory. > > -----Original Message- > From: Maulin Vasavada [mailto:maulin.vasav...@gmail.com] > Sent: Friday, August 30, 2019 3:07 PM > To: dev@kafka.apache.org > Subject: Re: [DISCUSS] KIP-486 Support for pluggable KeyStore and > TrustStore > >

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

2019-08-30 Thread Maulin Vasavada
that, does this mean, we resurrect KIP-383 and update it with latest context and go from there? We are willing to take up that work for making it configurable. Thanks Maulin On Fri, Aug 30, 2019 at 10:34 AM Maulin Vasavada wrote: > Why don't we make SSLEngineBuilder code delegate the whole Key/Tr

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

2019-08-30 Thread Maulin Vasavada
gt; custom > > class needs to implement. A large part of that is about loading > > keystore/truststore. I agree it has slightly more code than KIP-486 > > proposes, but since it lets you customize creation of SSLEngine, it > > would > > address every possible scenari

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

2019-08-29 Thread Maulin Vasavada
ey objects. Will probably be able to update it later today or tomorrow. Thanks Maulin On Thu, Aug 29, 2019 at 2:30 PM Maulin Vasavada wrote: > On that, I actually looked at KIP-383 before briefly. However, that > sounded like lot of changes suggested. > > One "key&quo

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

2019-08-29 Thread Maulin Vasavada
ly. Using a pluggable SslEngineBuilder > will also solve several other use cases at the same time. KIP-383 hasn't > been voted through yet, but perhaps you could take a look and we could > revive that instead if it solves your use case as well? > > Regards, > > Rajini > >

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

2019-08-29 Thread Maulin Vasavada
stores instead of returning certs, private > keys etc. . If others in the community are ok with such interface we can > move forward. > > Thanks, > Harsha > > > On Wed, Aug 28, 2019 at 1:51 PM, Maulin Vasavada < > maulin.vasav...@gmail.com> > wrote: > > &

[VOTE] KIP-486: Support custom way to load KeyStore and TrustStore

2019-08-29 Thread Maulin Vasavada
Hi all After a good discussion on the KIP at https://www.mail-archive.com/dev@kafka.apache.org/msg99126.html I think we are ready to start voting. KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-486%3A+Support+custom+way+to+load+KeyStore+and+TrustStore The KIP proposes - supporting

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

2019-08-29 Thread Maulin Vasavada
d > certs into KMS and sometimes it’s not feasible to copy them to file-system > instead directly use them from the memory. So, KIP-486 provides a custom > way to load keys/certs without implementing security-provider.Thanks,Rajan* > > On Wed, Aug 28, 2019 at 2:18 PM Maulin Vasavada >

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

2019-08-28 Thread Maulin Vasavada
ybody else would like to see on the functionality however. Thanks Maulin On Thu, Aug 22, 2019 at 9:12 PM Maulin Vasavada wrote: > Hi Harsha > > Any response on my question? I feel this KIP is worth accommodating. Your > help is much appreciated. > > Thanks > Maulin >

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

2019-08-22 Thread Maulin Vasavada
Hi Harsha Any response on my question? I feel this KIP is worth accommodating. Your help is much appreciated. Thanks Maulin On Tue, Aug 20, 2019 at 11:52 PM Maulin Vasavada wrote: > Hi Harsha > > I've examined the SPIFFE provider more and have one question - > > If SPIFFE did

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

2019-08-21 Thread Maulin Vasavada
tory > .getInstance(TrustManagerFactory.getDefaultAlgorithm());and use > tmf.chekServerTrusted() > or use > https://docs.oracle.com/javase/7/docs/api/javax/net/ssl/TrustManagerFactory.html#getInstance(java.lang.String)if > you want a specific provider. > > -Harsha > >

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

2019-08-20 Thread Maulin Vasavada
omTrustManagerFactory"); > } > } > > > The PR for this is in review and can be found here: > https://github.com/apache/kafka/pull/7090 > This PR includes the fixed insertProviderAt function call. > > On Tue, Aug 20, 2019 at 9:56 AM Harsha Chintalapani > wrote: > >

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

2019-08-19 Thread Maulin Vasavada
Instead of keep diverging in different directions, it would be helpful if you guys take my detailed posts with 1st to 4th points I mentioned and start referring/commenting on each of those if you agree with them or not. On Mon, Aug 19, 2019 at 10:45 PM Maulin Vasavada wrote: > Hi Colin >

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

2019-08-19 Thread Maulin Vasavada
e discussion of > Security.insertProviderAt() that you included. SslEngineBuilder doesn't > use that API to get the security provider. Instead, it calls > "SSLContext.getInstance(protocol, provider)", where provider is the name of > the provider. > > best, > Colin

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

2019-08-17 Thread Maulin Vasavada
a local file and doesn't require writing a Provider necessarily. If we do NOT have a custom algorithm, it makes less sense to write a Provider. Thanks Maulin On Thu, Aug 15, 2019 at 11:45 PM Maulin Vasavada wrote: > Hi Harsha/Colin > > I did the sample with a custom Provider for

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

2019-08-16 Thread Maulin Vasavada
ious experiments I did. I would encourage you to do similar exercise by writing a Provider for "Standard Algorithm" for TrustStoreManager (like our needs) and see what you find since only writing samples can bring out the complexity/challenges we face. Thanks Maulin On Wed, Aug 14, 2019 at 1

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

2019-08-15 Thread Maulin Vasavada
Just to update - still working on it. Get to work only on and off on it :( On Thu, Aug 8, 2019 at 4:05 PM Maulin Vasavada wrote: > Hi Harsha > > Let me try to write samples and will let you know. > > Thanks > Maulin > > On Thu, Aug 8, 2019 at 4:00 PM Harsha C

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

2019-08-08 Thread Maulin Vasavada
t > implementation anyway. > Spire agent is an example, its nothing but a GRPC server listening on a > unix domain socket . Above code is making a RPC call to the local daemon to > get the certificate and keys. The mechanics are pretty much same as what > you are asking for. &g

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

2019-08-08 Thread Maulin Vasavada
ith strict InfoSec requirements but we still do not need Custom SSL Provider for Kafka. Thanks Maulin On Thu, Aug 8, 2019 at 3:47 PM Maulin Vasavada wrote: > Imagine a scenario like - We know we have a custom KMS and as a Kafka > owner we want to comply to using that KMS source to

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

2019-08-08 Thread Maulin Vasavada
. Going the Provider way requires to know lot more than we should, isn't it? Not that we would have concern/shy-away knowing those details - but if we don't have to - why should we? On Thu, Aug 8, 2019 at 3:23 PM Maulin Vasavada wrote: > Hi Harsha > > We don't have spire (or similar) agen

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

2019-08-08 Thread Maulin Vasavada
08, 2019 at 2:04 PM, Maulin Vasavada < > maulin.vasav...@gmail.com> > wrote: > > > Hi Harsha > > > > The reason we rejected the SslProvider route is that - we only needed a > > custom way to load keys/certs. Not touch any policy that existing > Providers > &

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

2019-08-08 Thread Maulin Vasavada
cwiki.apache.org/confluence/display/KAFKA/ > > KIP-492%3A+Add+java+security+providers+in+Kafka+Security+config also > > addresses easy registration of such providers. > > > > Thanks, > > Harsha > > > > > > On Wed, Aug 07, 2019 at 11:31 PM, Maulin Vasa

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

2019-08-08 Thread Maulin Vasavada
Bump! Can somebody please review this? On Tue, Jul 16, 2019 at 1:51 PM Maulin Vasavada wrote: > Bump! Can somebody please review this? >

Re: [DISCUSS] KIP-317: Transparent Data Encryption

2019-08-08 Thread Maulin Vasavada
Hi Sönke Liebau Thanks for the great detailed documentation. However, I feel by leaving the KMS outside of Kafka might simplify the whole thing to a great extent. If the broker is not going to touch the

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

2019-07-16 Thread Maulin Vasavada
Bump! Can somebody please review this?

[jira] [Created] (KAFKA-8621) KIP-486: Support for pluggable KeyStore and TrustStore

2019-07-02 Thread MAULIN VASAVADA (JIRA)
MAULIN VASAVADA created KAFKA-8621: -- Summary: KIP-486: Support for pluggable KeyStore and TrustStore Key: KAFKA-8621 URL: https://issues.apache.org/jira/browse/KAFKA-8621 Project: Kafka

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

2019-07-02 Thread maulin . vasavada
Hi all, Starting this thread to discuss about KIP 486 - https://cwiki.apache.org/confluence/display/KAFKA/KIP-486%3A+Support+for+pluggable+KeyStore+and+TrustStore Thanks Maulin

Request for Permission to Create KIP

2019-06-27 Thread Maulin Vasavada
Hi Can you please give me permission to Create KIP? My username: maulin.vasavada Thank you. Maulin