While implementing the KIP it became evident that for the validate call
from the Connect Rest interface it was required to pass each config
individually to the policy implementation to be able to determine which
overrides are allowed. This happened since the interface relied on using
Exception(PolicyViolationException) as the mechanism to indicate an error
in the configuration. This defeats the purpose of the interface which
allows all overridden configurations to be passed together. To overcome
this limitation, I have updated the KIP (specifically the interface) to
return a list of ConfigValue (org.apache.kafka.common.config.ConfigValue )
instead of throwing an exception. This pattern is already followed in the
validate() of the Connector class. This change doesn't alter the semantics
or behavior of the out-of-the-box policies or how policies are configured
or enforced. Let me know if you have any questions or comments on the
change.

Apart from the above interface change, I have also made another minor
change which was missed out in the list of Configs allowed by the
`Principal` policy. I have included `security.protocol` and
`sasl.mechanism` which were missed out.

On Fri, May 10, 2019 at 9:23 AM Magesh Nandakumar <mage...@confluent.io>
wrote:

> Thanks a lot, Colin.  This KIP has now passed voting with 3 binding votes
> ( Randall, Rajini & Colin) and 1 non-binding vote (Chris).
> Thanks a lot, everyone for the feedback & discussion on this KIP.
>
> On Fri, May 10, 2019 at 9:12 AM Colin McCabe <cmcc...@apache.org> wrote:
>
>> +1 (binding).  Thanks, Magesh.
>>
>> cheers,
>> Colin
>>
>> On Thu, May 9, 2019, at 18:31, Randall Hauch wrote:
>> > I'm still +1 and like the simplification.
>> >
>> > Randall
>> >
>> > On Thu, May 9, 2019 at 5:54 PM Magesh Nandakumar <mage...@confluent.io>
>> > wrote:
>> >
>> > > I have updated the KIP to remove the `Ignore` policy and also the
>> > > useOverrides()
>> > > method in the interface.
>> > > Thanks a lot for your thoughts, Colin. I believe this certainly
>> simplifies
>> > > the KIP.
>> > >
>> > > On Thu, May 9, 2019 at 3:44 PM Magesh Nandakumar <
>> mage...@confluent.io>
>> > > wrote:
>> > >
>> > > > Unless anyone has objections, I'm going to update the KIP to remove
>> the
>> > > > `Ignore` policy and make `None` as the default. I will also remove
>> the `
>> > > > default boolean useOverrides()` in the interface which was
>> introduced for
>> > > > the purpose of backward compatibility.
>> > > >
>> > > > On Thu, May 9, 2019 at 3:27 PM Randall Hauch <rha...@gmail.com>
>> wrote:
>> > > >
>> > > >> I have also seen users include in connector configs the
>> `producer.*` and
>> > > >> `consumer.*` properties that should go into the worker configs. But
>> > > those
>> > > >> don't match, and the likelihood that someone is already using
>> > > >> `producer.override.*` or `consumer.override.*` properties in their
>> > > >> connector configs does seem pretty tiny.
>> > > >>
>> > > >> I'd be fine with removing the `Ignore` for backward compatibility.
>> Still
>> > > >> +1
>> > > >> either way.
>> > > >>
>> > > >> On Thu, May 9, 2019 at 5:23 PM Magesh Nandakumar <
>> mage...@confluent.io>
>> > > >> wrote:
>> > > >>
>> > > >> > To add more details regarding the backward compatibility; I have
>> > > >> generally
>> > > >> > seen users trying to set "producer.request.timeout.ms
>> > > >> > <http://producer.override.request.timeout.ms/>" in their
>> connector
>> > > >> config
>> > > >> > under the assumption that it will get used and would never come
>> back
>> > > to
>> > > >> > remove it. The initial intent of the KIP was to use the same
>> prefix
>> > > but
>> > > >> > since that potentially collided with MM2 configs, we agreed to
>> use a
>> > > >> > different prefix "producer.override". With this context, I think
>> the
>> > > >> > likelihood of someone using this is very small and should
>> generally
>> > > not
>> > > >> be
>> > > >> > a problem.
>> > > >> >
>> > > >> > On Thu, May 9, 2019 at 3:15 PM Magesh Nandakumar <
>> > > mage...@confluent.io>
>> > > >> > wrote:
>> > > >> >
>> > > >> > > Colin,
>> > > >> > >
>> > > >> > > Thanks a lot for the feedback.  As you said, the possibilities
>> of
>> > > >> someone
>> > > >> > > having "producer.override.request.timeout.ms" in their
>> connector
>> > > >> config
>> > > >> > > in AK 2.2 or lower is very slim. But the key thing is if in
>> case,
>> > > >> someone
>> > > >> > > has it AK2.2 doesn't do anything with it and it silently
>> ignores the
>> > > >> > > configuration. If others think that it's not really a problem,
>> then
>> > > >> I'm
>> > > >> > > fine with removing the complicated compatibility issue.
>> > > >> > >
>> > > >> > > I have explicitly called out the behavior when the exception is
>> > > >> thrown.
>> > > >> > >
>> > > >> > > Let me know what you think.
>> > > >> > >
>> > > >> > > Thanks,
>> > > >> > > Magesh
>> > > >> > >
>> > > >> > > On Thu, May 9, 2019 at 2:45 PM Colin McCabe <
>> cmcc...@apache.org>
>> > > >> wrote:
>> > > >> > >
>> > > >> > >> Hi Magesh,
>> > > >> > >>
>> > > >> > >> Thanks for the KIP.  It looks good overall.
>> > > >> > >>
>> > > >> > >> >    default boolean useOverrides() {
>> > > >> > >> >        return true;
>> > > >> > >> >    }
>> > > >> > >>
>> > > >> > >> Is this method really needed?  As I understand, nobody should
>> have
>> > > >> any
>> > > >> > >> connector client config overrides set right now, since they
>> don't
>> > > do
>> > > >> > >> anything right now.
>> > > >> > >>
>> > > >> > >> For example, you wouldn't expect a Kafka 2.2 installation to
>> have "
>> > > >> > >> producer.override.request.timeout.ms" set, since that
>> doesn't do
>> > > >> > >> anything in Kafka 2.2.  So is the option to ignore it in
>> Kafka 2.3
>> > > >> > really
>> > > >> > >> necessary?
>> > > >> > >>
>> > > >> > >> Can you add some details about what happens if a
>> > > >> > >> PolicyValidationException is thrown?  I'm assuming that we
>> fail to
>> > > >> > create
>> > > >> > >> the new Connector, I'm not sure if that's completely spelled
>> out
>> > > >> > (unless I
>> > > >> > >> missed it).
>> > > >> > >>
>> > > >> > >> best,
>> > > >> > >> Colin
>> > > >> > >>
>> > > >> > >>
>> > > >> > >> On Thu, May 9, 2019, at 08:05, Rajini Sivaram wrote:
>> > > >> > >> > Hi Magesh,
>> > > >> > >> >
>> > > >> > >> > Thanks for the KIP, +1 (binding)
>> > > >> > >> >
>> > > >> > >> > Regards,
>> > > >> > >> >
>> > > >> > >> > Rajini
>> > > >> > >> >
>> > > >> > >> >
>> > > >> > >> > On Thu, May 9, 2019 at 3:55 PM Randall Hauch <
>> rha...@gmail.com>
>> > > >> > wrote:
>> > > >> > >> >
>> > > >> > >> > > Nice work, Magesh.
>> > > >> > >> > >
>> > > >> > >> > > +1 (binding)
>> > > >> > >> > >
>> > > >> > >> > > Randall
>> > > >> > >> > >
>> > > >> > >> > > On Wed, May 8, 2019 at 7:22 PM Magesh Nandakumar <
>> > > >> > >> mage...@confluent.io>
>> > > >> > >> > > wrote:
>> > > >> > >> > >
>> > > >> > >> > > > Thanks a lot Chris. So far, the KIP has one non-binding
>> vote
>> > > >> and
>> > > >> > I'm
>> > > >> > >> > > still
>> > > >> > >> > > > looking forward to the KIP to be voted by Friday's
>> deadline.
>> > > >> > >> > > >
>> > > >> > >> > > > On Tue, May 7, 2019 at 10:00 AM Chris Egerton <
>> > > >> > chr...@confluent.io>
>> > > >> > >> > > wrote:
>> > > >> > >> > > >
>> > > >> > >> > > > > Hi Magesh,
>> > > >> > >> > > > >
>> > > >> > >> > > > > This looks great! Very excited to see these changes
>> finally
>> > > >> > >> coming to
>> > > >> > >> > > > > Connect.
>> > > >> > >> > > > > +1 (non-binding)
>> > > >> > >> > > > >
>> > > >> > >> > > > > Cheers,
>> > > >> > >> > > > >
>> > > >> > >> > > > > Chris
>> > > >> > >> > > > >
>> > > >> > >> > > > > On Tue, May 7, 2019 at 9:51 AM Magesh Nandakumar <
>> > > >> > >> mage...@confluent.io
>> > > >> > >> > > >
>> > > >> > >> > > > > wrote:
>> > > >> > >> > > > >
>> > > >> > >> > > > > > Hi All,
>> > > >> > >> > > > > >
>> > > >> > >> > > > > > I would like to start a vote on
>> > > >> > >> > > > > >
>> > > >> > >> > > > > >
>> > > >> > >> > > > >
>> > > >> > >> > > >
>> > > >> > >> > >
>> > > >> > >>
>> > > >> >
>> > > >>
>> > >
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-458%3A+Connector+Client+Config+Override+Policy
>> > > >> > >> > > > > >
>> > > >> > >> > > > > > The discussion thread can be found here
>> > > >> > >> > > > > > <
>> > > >> > >>
>> https://www.mail-archive.com/dev@kafka.apache.org/msg97124.html>.
>> > > >> > >> > > > > >
>> > > >> > >> > > > > > Thanks,
>> > > >> > >> > > > > > Magesh
>> > > >> > >> > > > > >
>> > > >> > >> > > > >
>> > > >> > >> > > >
>> > > >> > >> > >
>> > > >> > >> >
>> > > >> > >>
>> > > >> > >
>> > > >> >
>> > > >>
>> > > >
>> > >
>> >
>>
>

Reply via email to