Randall,

Thanks a lot for your feedback.

You bring up an interesting point regarding the overrides being available
to the connectors. Today everything that is specified in the config while
creating is available for the connector. But this is a specific case and we
could do either of the following


   - don't pass any configs with these prefixes to the ConnectorConfig
   instance that's passed in the startConnector
   - allow policies as to whether the configurations with the prefixes
   should be made available to the connector or not. Should this also define a
   list of configurations?

I personally prefer not passing the configs to Connector since that's
simple, straight forward and don't see a reason for the connector to access
those.

For the second point,  None - doesn't allow overrides and the default
policy is null. We preserve backward compatibility when no policy is
configured. Let me know if that's not clear in the KIP.

Thanks,
Magesh

On Mon, Apr 29, 2019 at 4:07 PM Randall Hauch <rha...@gmail.com> wrote:

> Per the proposal, a connector configuration can define one or more
> properties that begin with any of the three prefixes: "producer.override.",
> "consumer.override.", and "admin.override.". The proposal states:
>
> Since the users can specify any of these policies, the connectors itself
> should not rely on these configurations to be available. The overrides are
> to be used purely from an operational perspective.
>
>
> Does this mean that any such properties are visible to connectors, or will
> they be hidden to connectors? Currently no connectors have access to such
> client properties, and users are unlike to just put them into a connector
> configuration unnecessarily. A connector implementation could have defined
> such properties as normal connector-specific properties, in which case they
> are required, but is that likely given the log prefixes? One concern that I
> have is that this might allow connector implementations start attempting to
> circumvent the Connect API if these properties are included.
>
> Second, does the None policy allow but ignore these additional properties
> (e.g., "validate(...)" is simply a no-op)? Or does the None policy fail if
> any client overrides are specified? The former seems more in line with the
> current behavior, whereas the "disallows" policy seems useful but not
> exactly backward compatible. Should we also offer a "Disallow" policy? In
> fact, should the policies be named "Ignore" (default), "Disallow",
> "Prinicipal", and "All"?
>
> Otherwise, I like the idea of this. There have been several requests over
> the past year or two for adding subsets of this functionality. Might be
> good to find and list all of the related KAFKA issues.
>
> Randall
>
> On Fri, Apr 26, 2019 at 4:04 PM Chris Egerton <chr...@confluent.io> wrote:
>
> > Hi Magesh,
> >
> > Changes look good to me! Excited to see this happen, hope the KIP passes
> :)
> >
> > Cheers,
> >
> > Chris
> >
> > On Fri, Apr 26, 2019 at 1:44 PM Magesh Nandakumar <mage...@confluent.io>
> > wrote:
> >
> > > Hi Chris,
> > >
> > > I have updated the KIP to reflect the changes that we discussed for the
> > > prefix. Thanks for all your inputs.
> > >
> > > Thanks,
> > > Magesh
> > >
> > > On Thu, Apr 25, 2019 at 2:18 PM Chris Egerton <chr...@confluent.io>
> > wrote:
> > >
> > > > Hi Magesh,
> > > >
> > > > Agreed that we should avoid `dlq.admin`. I also don't have a strong
> > > opinion
> > > > between `connector.` and `.override`, but I have a slight inclination
> > > > toward `.override` since `connector.` feels a little redundant given
> > that
> > > > the whole configuration is for the connector and the use of
> "override"
> > > may
> > > > shed a little light on how the properties for these clients are
> > computed
> > > > and help make the learning curve a little gentler on new devs and
> > users.
> > > >
> > > > Regardless, I think the larger issue of conflicts with existing
> > > properties
> > > > (both in MM2 and potentially other connectors) has been
> satisfactorily
> > > > addressed, so I'm happy.
> > > >
> > > > Cheers,
> > > >
> > > > Chris
> > > >
> > > > On Wed, Apr 24, 2019 at 11:14 AM Magesh Nandakumar <
> > mage...@confluent.io
> > > >
> > > > wrote:
> > > >
> > > > > HI Chrise,
> > > > >
> > > > > You are right about the "admin." prefix creating conflicts. Here
> are
> > > few
> > > > > options that I can think of
> > > > >
> > > > > 1. Use `dlq.admin` since admin client is used only for DLQ. But
> this
> > > > might
> > > > > not really be the case in the future. So, we should possibly drop
> > this
> > > > idea
> > > > > :)
> > > > > 2.  Use `connector.producer`, `connector.consumer` and
> > > `connector.admin`
> > > > -
> > > > > provides better context that its connector specific property
> > > > > 3.  Use `producer.override`, '`consumer.override` and
> > `admin.override`
> > > -
> > > > > provides better clarity that these are overrides.
> > > > >
> > > > > I don't have a strong opinion in choosing between #2 and #3. Let me
> > > > > know what you think.
> > > > >
> > > > > Thanks
> > > > > Magesh
> > > > >
> > > > > On Wed, Apr 24, 2019 at 10:25 AM Chris Egerton <
> chr...@confluent.io>
> > > > > wrote:
> > > > >
> > > > > > Hi Magesh,
> > > > > >
> > > > > > Next round :)
> > > > > >
> > > > > > 1. It looks like MM2 will also support "admin." properties that
> > > affect
> > > > > > AdminClients it creates and uses, which IIUC is the same prefix
> > name
> > > to
> > > > > be
> > > > > > used for managing the DLQ for sink connectors in this KIP.
> Doesn't
> > > that
> > > > > > still leave room for conflict? I'm imagining a scenario like
> this:
> > a
> > > > > > Connect worker is configured to use the
> > > > > > PrincipalConnectorClientConfigPolicy, someone tries to start an
> > > > instance
> > > > > of
> > > > > > an MM2 sink with "admin." properties beyond just
> > > > > "admin.sasl.jaas.config",
> > > > > > and gets rejected because those properties are then interpreted
> by
> > > the
> > > > > > worker as overrides for the AdminClient it uses to manage the
> DLQ.
> > > > > > 2. (LGTM)
> > > > > > 3. I'm convinced by this, as long as nobody else identifies a
> > common
> > > > use
> > > > > > case that would involve a similar client config policy
> > implementation
> > > > > that
> > > > > > would be limited to a small set of whitelisted configs. For now
> > > keeping
> > > > > the
> > > > > > PrincipalConnectorClientConfigPolicy sounds fine to me.
> > > > > >
> > > > > > Cheers,
> > > > > >
> > > > > > Chris
> > > > > >
> > > > > > On Tue, Apr 23, 2019 at 10:30 PM Magesh Nandakumar <
> > > > mage...@confluent.io
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > I also have a draft implementation of the KIP
> > > > > > > https://github.com/apache/kafka/pull/6624. I would still need
> to
> > > > > include
> > > > > > > more tests and docs but I thought it would be useful to have
> this
> > > for
> > > > > the
> > > > > > > KIP discussion. Looking forward to all of your valuable
> feedback.
> > > > > > >
> > > > > > > Thanks
> > > > > > > Magesh
> > > > > > >
> > > > > > > On Tue, Apr 23, 2019 at 10:27 PM Magesh Nandakumar <
> > > > > mage...@confluent.io
> > > > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Chrise,
> > > > > > > >
> > > > > > > > Thanks a lot for your feedback. I will address them in order
> of
> > > > your
> > > > > > > > questions/comments.
> > > > > > > >
> > > > > > > > 1. Thanks for bringing this to my attention about KIP-382. I
> > had
> > > a
> > > > > > closer
> > > > > > > > look at the KIP and IIUC, the KIP allows `consumer.` prefix
> for
> > > > > > > SourceConnector
> > > > > > > > and producer. prefix for SinkConnector since those are
> > additional
> > > > > > > > connector properties to help resolve the Kafka cluster other
> > than
> > > > the
> > > > > > one
> > > > > > > > Connect framework knows about. Whereas, the proposal in
> KIP-458
> > > > > applies
> > > > > > > > producer policies for SinkConnectors and consumer policies
> > > > > > > > SourceConnectors.  So, from what I understand this new policy
> > > > should
> > > > > > work
> > > > > > > > without any issues even for Mirror Maker 2.0.
> > > > > > > > 2. I have updated the KIP to use a default value of null and
> > use
> > > > that
> > > > > > to
> > > > > > > > determine if we need to ignore overrides.
> > > > > > > > 3. I would still prefer to keep the special
> > > > > > > PrincipalConnectorClientConfigPolicy
> > > > > > > > since that is one of the most common use cases one would
> choose
> > > to
> > > > > use
> > > > > > > this
> > > > > > > > feature. If we make it a general case, that would involve
> users
> > > > > > requiring
> > > > > > > > to add additional configuration and they might require well
> > more
> > > > than
> > > > > > > just
> > > > > > > > the list of configs but might also want some restriction on
> > > values.
> > > > > If
> > > > > > > the
> > > > > > > > concern is about users wanting principal and also other
> > configs,
> > > it
> > > > > > would
> > > > > > > > still be possible by means of a custom implementation. As
> is, I
> > > > would
> > > > > > > > prefer to keep the proposal to be the same for this. Let me
> > know
> > > > your
> > > > > > > > thoughts.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Magesh
> > > > > > > >
> > > > > > > >
> > > > > > > > On Mon, Apr 22, 2019 at 3:44 PM Chris Egerton <
> > > chr...@confluent.io
> > > > >
> > > > > > > wrote:
> > > > > > > >
> > > > > > > >> Hi Magesh,
> > > > > > > >>
> > > > > > > >> This is an exciting KIP! I have a few questions/comments but
> > > > > overall I
> > > > > > > >> like
> > > > > > > >> the direction it's headed in and hope to see it included in
> > the
> > > > > > Connect
> > > > > > > >> framework soon.
> > > > > > > >>
> > > > > > > >> 1. With the proposed "consumer.", "producer.", and "admin."
> > > > > prefixes,
> > > > > > > how
> > > > > > > >> will this interact with connectors such as the upcoming
> Mirror
> > > > Maker
> > > > > > 2.0
> > > > > > > >> (KIP-382) that already support properties with those
> prefixes?
> > > > Would
> > > > > > it
> > > > > > > be
> > > > > > > >> possible for a user to configure MM2 with those properties
> > > without
> > > > > > them
> > > > > > > >> being interpreted as Connect client overrides, without
> > isolating
> > > > MM2
> > > > > > > onto
> > > > > > > >> its own cluster and using the
> > IgnoreConnectorClientConfigPolicy
> > > > > > policy?
> > > > > > > >> 2. Is the IgnoreConnectorClientConfigPolicy class necessary?
> > The
> > > > > > default
> > > > > > > >> for the connector.client.config.policy property could simply
> > be
> > > > null
> > > > > > > >> instead of a new policy that, as far as I can tell, isn't an
> > > > actual
> > > > > > > policy
> > > > > > > >> in that its validate(...) method is never invoked and
> instead
> > > > > > > represents a
> > > > > > > >> special case to the Connect framework that says "Drop all
> > > > overrides
> > > > > > and
> > > > > > > >> never use me".
> > > > > > > >> 3. The PrincipalConnectorClientConfigPolicy seems like a
> > > specific
> > > > > > > instance
> > > > > > > >> of a more general use case: allow exactly a small set of
> > > overrides
> > > > > and
> > > > > > > no
> > > > > > > >> others. Why not generalize here and create a policy that
> > > accepts a
> > > > > > list
> > > > > > > of
> > > > > > > >> allowed overrides during configuration?
> > > > > > > >>
> > > > > > > >> Thanks again for the KIP.
> > > > > > > >>
> > > > > > > >> Cheers,
> > > > > > > >>
> > > > > > > >> Chris
> > > > > > > >>
> > > > > > > >> On Fri, Apr 19, 2019 at 2:53 PM Magesh Nandakumar <
> > > > > > mage...@confluent.io
> > > > > > > >
> > > > > > > >> wrote:
> > > > > > > >>
> > > > > > > >> > Hi all,
> > > > > > > >> >
> > > > > > > >> > I've posted "KIP-458: Connector Client Config Override
> > > Policy",
> > > > > > which
> > > > > > > >> > allows users to override the connector client
> configurations
> > > > based
> > > > > > on
> > > > > > > a
> > > > > > > >> > policy defined by the administrator.
> > > > > > > >> >
> > > > > > > >> > The KIP can be found at
> > > > > > > >> >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-458%3A+Connector+Client+Config+Override+Policy
> > > > > > > >> > .
> > > > > > > >> >
> > > > > > > >> > Looking forward for the discussion on the KIP and all of
> > your
> > > > > > > thoughts &
> > > > > > > >> > feedback on this enhancement to Connect.
> > > > > > > >> >
> > > > > > > >> > Thanks,
> > > > > > > >> > Magesh
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to