Hi Sandeep,
Thanks for the KIP, I have few comments below.

>>“To take advantage of these custom algorithms, we want to support java 
>>security provider parameter in security config. This param can be used by 
>>kafka brokers or kafka clients(when connecting to the kafka brokers). The 
>>security providers can also be used for configuring security algorithms in 
>>SASL based communication.”

You may want to mention use case like
spiffe.provider.SpiffeProvider[1] in streaming applications like
Flink, Spark or Storm etc.

>>"We add new config parameter in KafkaConfig named “security.provider.class”. 
>>The value of “security.provider” is expected to be a string representing the 
>>provider’s full classname. This provider class will be added to the JVM 
>>properties through Security.addProvider api. Security class can be used to 
>>programmatically add the provider classes to the JVM."

It is good to have this property as a list of providers instead of a
single property. This will allow configuring multiple providers if it
is needed in the future without introducing hacky solutions like
security.provider.class.name.x, where x is a sequence number. You can
change the property name to “security.provider.class.names” and its
value is a list of fully qualified provider class names separated by
‘,'.
For example:
security.provider.class.names=spiffe.provider.SpiffeProvider,com.foo.MyProvider

Typo in existing properties section:
“ssl.provider” instead of “ssl.providers”.

Thanks,
Satish.

1. https://github.com/spiffe/java-spiffe


On Mon, Jul 15, 2019 at 11:41 AM Sandeep Mopuri <mpr...@gmail.com> wrote:
>
> Hello all,
>
> I'd like to start a discussion thread for KIP-492.
> This KIP plans on introducing a new security config parameter for a custom
> security providers. Please take a look and let me know what do you think.
>
> More information can be found here:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-492%3A+Add+java+security+providers+in+Kafka+Security+config
> --
> Thanks,
> Sai Sandeep

Reply via email to