You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Konstantine Karantasis <ko...@confluent.io> on 2019/12/12 19:59:55 UTC

[DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

I've taken a second look to KIP-158 after syncing with Randall Hauch, who
was the original author of the proposal, and I have updated the KIP in
place.

The main new features of this updated KIP-158 is the introduction of groups
of configs that can be composed and the ability to match topics to these
groups via the use of regex. The design builds on top of the existing
definition of config groups used in single message transformations (SMT)
and therefore I'm hoping that the approach fits well in Kafka Connect's
current configuration capabilities.

The new proposal aims to strike a good balance between requiring to
explicitly set the configs for each possible topic or having a
one-size-fits-all default set of properties for all the topics a connector
may create during runtime.


The updated KIP-158 can be found under the same page as the old one:

https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics

I've intentionally changed the title here in this thread to avoid confusion
with the threads that discussed KIP-158 previously.
Looking forward to your comments and hoping we can pick up this work from
the very good starting point that was reached in the previous discussions.


Best,
Konstantine

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Randall Hauch <rh...@gmail.com>.
Thanks for updating and improving this KIP, Konstantine! Overall I'm pretty
happy with the KIP, but I have a few fairly nit-picky questions/comments.

1) Will connectors see these configuration properties when Connect calls
Connector.start(...)? We've added a number of configs to connectors that
are handled by the framework, and it'd be good to follow the same
precedence and mention in this KIP what that behavior is.
2) Is it worth mentioning that the replication factor must be no larger
than the number of brokers in the cluster? I'm not sure it is, but it is a
common problem for folks that are less familiar with Kafka semantics. And
because the connector would fail only after trying to create
3) It may be worth mentioning *when* the connector might fail due to
problems with topic creation. It won't fail upon initial deployment, but
rather only if/when the connector writes to a topic that does not exist and
Connect attempts to create that topic.
4) Fourth, what do you think about adding an example that uses Connect's
client override feature by specifying `producer.override.` or
`admin.override.` properties. Using `producer.override.` properties is
sufficient, but `admin.override.` properties will take precedence for the
AdminClient use. (This might make more sense if "Configuration Examples"
appeared after the "Security" section.) Also, the last paragraph in the
"Compatibility" section mentions:

"If ACLs are used, the Kafka principal used in the Connect worker's
producer.* settings is assumed to have privilege..."


This should probably be corrected, since this is not quite the whole story.

5) The "Compatibility" section mentions:

This feature will do nothing if the broker does not support the Admin API
methods, which is equivalent to relying upon auto-topic creation.


Is this really what we want? Should the connector fail since earlier we
mention that we don't want to silently ignore properties?



Best regards,

Randall

On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> I've taken a second look to KIP-158 after syncing with Randall Hauch, who
> was the original author of the proposal, and I have updated the KIP in
> place.
>
> The main new features of this updated KIP-158 is the introduction of groups
> of configs that can be composed and the ability to match topics to these
> groups via the use of regex. The design builds on top of the existing
> definition of config groups used in single message transformations (SMT)
> and therefore I'm hoping that the approach fits well in Kafka Connect's
> current configuration capabilities.
>
> The new proposal aims to strike a good balance between requiring to
> explicitly set the configs for each possible topic or having a
> one-size-fits-all default set of properties for all the topics a connector
> may create during runtime.
>
>
> The updated KIP-158 can be found under the same page as the old one:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
>
> I've intentionally changed the title here in this thread to avoid confusion
> with the threads that discussed KIP-158 previously.
> Looking forward to your comments and hoping we can pick up this work from
> the very good starting point that was reached in the previous discussions.
>
>
> Best,
> Konstantine
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Konstantine Karantasis <ko...@confluent.io>.
Hi Tom,

thanks for your comments, they are good points and useful corrections.
Here's how things look now, after I updated the KIP:

10) I assumed proper escaping, but indeed it's better to be precise that
every pattern will be treated as a regex. I also added an example with a
topic name that contains dots.

11) Added an explicit mention regarding the precedence of the exclusion
list over the inclusion list.

12) Trying to remember the particularities of java regex, I thought I
spotted a difference in the use of the star wildcard. The use of these
wildcards is now corrected, plus there should be proper escaping for dots.

13) This was corrected elsewhere in the Compatibility section but not here.
Seems like the updated behavior will be that, if this proposed feature is
enabled, the tasks will fail during runtime. The previous behavior will be
used only if this feature is disabled.

Let me know what you think.

Best,
Konstantine

On Wed, Dec 18, 2019 at 7:21 AM Tom Bentley <tb...@redhat.com> wrote:

> Hi Konstantine,
>
> Thanks for the KIP. I can see this would be a useful addition to Kafka
> Connect.
>
> 10. The documentation of topic.creation.$alias.include says "A list of
> strings that represent either exact topic names or regular expressions that
> may match topic names." How do you know whether the string "foo.bar" is
> intended to be an exact match or a regex?
>
> 11. I think it's implicit that topic.creation.$alias.exclude overrides a
> topic that was matched by topic.creation.$alias.include? It would be good
> to say this explicitly early on.
>
> 12. In example 2 you have "topic.creation.inorder.include=status, orders*",
> which isn't necessarily wrong, but I suspect you meant "orders.*" (and even
> if not, it looks misleadingly like some kind of non-regex wildcard/glob).
> Similarly, other examples seem to be using non-regex wildcards when the KIP
> explicitly refers to Java regexes.
>
> 13. In the Security section you say: "If the worker's producer does not
> have the necessary privileges to DESCRIBE existing and CREATE missing
> topics but a source connector does specify the topic.creation.*
> configuration properties, the worker will log a warning and will default to
> the previous behavior of assuming the topics already exist or that the
> broker will auto-create them when needed." I think this means it's
> something which is checked when the connector is created/configured, rather
> than at some point during runtime when an attempt to create a topic fails
> due to authorization. It would be good to say so explicitly.
>
> Kind regards,
>
> Tom
>
>
>
> On Wed, Dec 18, 2019 at 12:47 AM Konstantine Karantasis <
> konstantine@confluent.io> wrote:
>
> > Thanks Randall and Ryanne for your comments.
> >
> > I'm replying to them below, in order of appearance:
> >
> > To Randall's comments:
> > 1) I assumed these properties would be visible to connectors, since by
> > definition these are connector properties. I added a mention. However I'm
> > not sure if you are also making a specific suggestion with this
> question. I
> > didn't find a similar mention in KIP-458, but 'override' directives also
> > appear in both the connector and the task properties. Given this
> precedent,
> > I think it makes sense to forward these properties to the connector as
> > well.
> >
> > 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> > definitely belongs to the Kafka Connect docs that will describe how to
> > operate Connect with this feature enabled.
> >
> > 3) Added a note to mention that a task might fail during runtime and that
> > early validation won't be in place for this feature.
> >
> > 4) Examples added and the sentence regarding ACLs and failure was
> adjusted
> > to reflect the new proposal.
> >
> > 5) Also addressed and the KIP now mentions that the task will fail if the
> > feature is enabled and the broker does not support the Admin API.
> >
> > To your point Ryanne, I'm also often in favor of reserving some room for
> > customizations that will be able to address specific user needs, but I
> > don't think we have a strong case for making this functionality pluggable
> > at the moment. Topics are not very transient entities in Kafka. And this
> > feature is focusing specifically on topic creation and does not suggest
> > altering configuration of existing topics, including topics that may be
> > created once by a connector that will use this new functionality.
> > Therefore, adapting to changes to the attainable replication factor
> during
> > runtime, without expressing this in the configuration of a connector
> seems
> > to involve more risks than benefits. Overall, a generic topic creation
> hook
> > shares similarities to exposing an admin client to the connector itself
> and
> > based on previous discussions, seems that this approach will result in
> > considerable extensions in both configuration and implementation without
> it
> > being fully justified at the moment.
> >
> > I suggest moving forward without pluggable classes for now, and if in the
> > future we wish to return to this topic for second iteration, then
> factoring
> > out the proposed functionality under the configuration of a module that
> > applies topic creation based on regular expressions should be easy to do
> in
> > a compatible way.
> >
> > Best,
> > Konstantine
> >
> >
> > On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com>
> > wrote:
> >
> > > Konstantine, thanks for the updates. I wonder if we should take your
> > > proposal one step further and make this pluggable. Your include/exclude
> > > regexes are great out-of-the-box features, but it may be valuable to
> > > plug-in more sophisticated logic to handle topic creation.
> > >
> > > Instead of enabling/disabling the feature as a whole, the default
> > > TopicCreator (or whatever) could be a nop. Then we include a
> > > RegexTopicCreator with your proposed behavior. This would be almost
> > > indistinguishable from your current KIP from a user's perspective, but
> > > would enable plug-in TopicCreators that do some of the things you have
> > > listed in the Rejected Alternatives, e.g. to automatically adjust the
> > > replication factor based on the number of nodes, etc.
> > >
> > > My team leverages Connect's plug-ins in other places to enable seamless
> > > integration with the rest of our platform. We would definitely use a
> > topic
> > > creation hook if one existed. In particular, we have a concept of
> "topic
> > > profiles" that we could use here.
> > >
> > > Ryanne
> > >
> > > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > > konstantine@confluent.io> wrote:
> > >
> > > > I've taken a second look to KIP-158 after syncing with Randall Hauch,
> > who
> > > > was the original author of the proposal, and I have updated the KIP
> in
> > > > place.
> > > >
> > > > The main new features of this updated KIP-158 is the introduction of
> > > groups
> > > > of configs that can be composed and the ability to match topics to
> > these
> > > > groups via the use of regex. The design builds on top of the existing
> > > > definition of config groups used in single message transformations
> > (SMT)
> > > > and therefore I'm hoping that the approach fits well in Kafka
> Connect's
> > > > current configuration capabilities.
> > > >
> > > > The new proposal aims to strike a good balance between requiring to
> > > > explicitly set the configs for each possible topic or having a
> > > > one-size-fits-all default set of properties for all the topics a
> > > connector
> > > > may create during runtime.
> > > >
> > > >
> > > > The updated KIP-158 can be found under the same page as the old one:
> > > >
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > > >
> > > > I've intentionally changed the title here in this thread to avoid
> > > confusion
> > > > with the threads that discussed KIP-158 previously.
> > > > Looking forward to your comments and hoping we can pick up this work
> > from
> > > > the very good starting point that was reached in the previous
> > > discussions.
> > > >
> > > >
> > > > Best,
> > > > Konstantine
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Tom Bentley <tb...@redhat.com>.
Hi Konstantine,

Thanks for the KIP. I can see this would be a useful addition to Kafka
Connect.

10. The documentation of topic.creation.$alias.include says "A list of
strings that represent either exact topic names or regular expressions that
may match topic names." How do you know whether the string "foo.bar" is
intended to be an exact match or a regex?

11. I think it's implicit that topic.creation.$alias.exclude overrides a
topic that was matched by topic.creation.$alias.include? It would be good
to say this explicitly early on.

12. In example 2 you have "topic.creation.inorder.include=status, orders*",
which isn't necessarily wrong, but I suspect you meant "orders.*" (and even
if not, it looks misleadingly like some kind of non-regex wildcard/glob).
Similarly, other examples seem to be using non-regex wildcards when the KIP
explicitly refers to Java regexes.

13. In the Security section you say: "If the worker's producer does not
have the necessary privileges to DESCRIBE existing and CREATE missing
topics but a source connector does specify the topic.creation.*
configuration properties, the worker will log a warning and will default to
the previous behavior of assuming the topics already exist or that the
broker will auto-create them when needed." I think this means it's
something which is checked when the connector is created/configured, rather
than at some point during runtime when an attempt to create a topic fails
due to authorization. It would be good to say so explicitly.

Kind regards,

Tom



On Wed, Dec 18, 2019 at 12:47 AM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> Thanks Randall and Ryanne for your comments.
>
> I'm replying to them below, in order of appearance:
>
> To Randall's comments:
> 1) I assumed these properties would be visible to connectors, since by
> definition these are connector properties. I added a mention. However I'm
> not sure if you are also making a specific suggestion with this question. I
> didn't find a similar mention in KIP-458, but 'override' directives also
> appear in both the connector and the task properties. Given this precedent,
> I think it makes sense to forward these properties to the connector as
> well.
>
> 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> definitely belongs to the Kafka Connect docs that will describe how to
> operate Connect with this feature enabled.
>
> 3) Added a note to mention that a task might fail during runtime and that
> early validation won't be in place for this feature.
>
> 4) Examples added and the sentence regarding ACLs and failure was adjusted
> to reflect the new proposal.
>
> 5) Also addressed and the KIP now mentions that the task will fail if the
> feature is enabled and the broker does not support the Admin API.
>
> To your point Ryanne, I'm also often in favor of reserving some room for
> customizations that will be able to address specific user needs, but I
> don't think we have a strong case for making this functionality pluggable
> at the moment. Topics are not very transient entities in Kafka. And this
> feature is focusing specifically on topic creation and does not suggest
> altering configuration of existing topics, including topics that may be
> created once by a connector that will use this new functionality.
> Therefore, adapting to changes to the attainable replication factor during
> runtime, without expressing this in the configuration of a connector seems
> to involve more risks than benefits. Overall, a generic topic creation hook
> shares similarities to exposing an admin client to the connector itself and
> based on previous discussions, seems that this approach will result in
> considerable extensions in both configuration and implementation without it
> being fully justified at the moment.
>
> I suggest moving forward without pluggable classes for now, and if in the
> future we wish to return to this topic for second iteration, then factoring
> out the proposed functionality under the configuration of a module that
> applies topic creation based on regular expressions should be easy to do in
> a compatible way.
>
> Best,
> Konstantine
>
>
> On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com>
> wrote:
>
> > Konstantine, thanks for the updates. I wonder if we should take your
> > proposal one step further and make this pluggable. Your include/exclude
> > regexes are great out-of-the-box features, but it may be valuable to
> > plug-in more sophisticated logic to handle topic creation.
> >
> > Instead of enabling/disabling the feature as a whole, the default
> > TopicCreator (or whatever) could be a nop. Then we include a
> > RegexTopicCreator with your proposed behavior. This would be almost
> > indistinguishable from your current KIP from a user's perspective, but
> > would enable plug-in TopicCreators that do some of the things you have
> > listed in the Rejected Alternatives, e.g. to automatically adjust the
> > replication factor based on the number of nodes, etc.
> >
> > My team leverages Connect's plug-ins in other places to enable seamless
> > integration with the rest of our platform. We would definitely use a
> topic
> > creation hook if one existed. In particular, we have a concept of "topic
> > profiles" that we could use here.
> >
> > Ryanne
> >
> > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > konstantine@confluent.io> wrote:
> >
> > > I've taken a second look to KIP-158 after syncing with Randall Hauch,
> who
> > > was the original author of the proposal, and I have updated the KIP in
> > > place.
> > >
> > > The main new features of this updated KIP-158 is the introduction of
> > groups
> > > of configs that can be composed and the ability to match topics to
> these
> > > groups via the use of regex. The design builds on top of the existing
> > > definition of config groups used in single message transformations
> (SMT)
> > > and therefore I'm hoping that the approach fits well in Kafka Connect's
> > > current configuration capabilities.
> > >
> > > The new proposal aims to strike a good balance between requiring to
> > > explicitly set the configs for each possible topic or having a
> > > one-size-fits-all default set of properties for all the topics a
> > connector
> > > may create during runtime.
> > >
> > >
> > > The updated KIP-158 can be found under the same page as the old one:
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > >
> > > I've intentionally changed the title here in this thread to avoid
> > confusion
> > > with the threads that discussed KIP-158 previously.
> > > Looking forward to your comments and hoping we can pick up this work
> from
> > > the very good starting point that was reached in the previous
> > discussions.
> > >
> > >
> > > Best,
> > > Konstantine
> > >
> >
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Randall Hauch <rh...@gmail.com>.
LGTM. Thanks!

On Fri, Jun 5, 2020 at 12:41 PM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> Thanks for bringing up KIP-464 Jose and apologies for taking that long to
> respond.
>
> It made sense to allow the users to use the broker defaults for the
> replication factor and the number of partitions when their source
> connectors create topics and the implementation has incorporated this
> ability.
> I've now updated KIP-158 to reflect this feature in the doc. Given that
> this is a minor and useful amendment I think we don't have to vote again on
> this change but please let me know if you think otherwise.
>
> Best,
> Konstantine
>
> On Mon, Feb 3, 2020 at 3:17 PM Jose Garcia Sancio <js...@confluent.io>
> wrote:
>
> > Thanks Konstantine. Looking forward to this feature.
> >
> > The KIP mentions:
> >
> > > For the *default* group this configuration is required. For any other
> > group defined in topic.creation.groups this config is optional and if
> it's
> > missing it gets the value the *default* group
> >
> > For the properties "topic.creation.$alias.replication.factor" and
> > "topic.creation.$alias.partitions". I think that we can and should make
> > this optional for all groups including the "default" group. Kafka's
> > CreateTopicRequest message allows these two fields to be optional. Here
> are
> > their descriptions respectively:
> >
> > > The number of replicas to create for each partition in the topic, or -1
> > if we are either specifying a manual partition assignment or using the
> > default repli
> > cation factor.
> > > The number of partitions to create in the topic, or -1 if we are either
> > specifying a manual partition assignment or using the default partitions.
> >
> > At the Java Client level this is model using Java's Optional type. I
> think
> > that we can make them both optional and resolve them to
> "Optional.empty()"
> > if neither the specific group or "default" is set.
> >
> > Thanks,
> > Jose
> >
> >
> > On Thu, Dec 19, 2019 at 8:27 PM Tom Bentley <tb...@redhat.com> wrote:
> >
> > > Thanks Konstantine, lgtm.
> > >
> > > On Thu, Dec 19, 2019 at 5:34 PM Ryanne Dolan <ry...@gmail.com>
> > > wrote:
> > >
> > > > Thanks for the reply Konstantine. Makes sense.
> > > >
> > > > Ryanne
> > > >
> > > > On Tue, Dec 17, 2019, 6:41 PM Konstantine Karantasis <
> > > > konstantine@confluent.io> wrote:
> > > >
> > > > > Thanks Randall and Ryanne for your comments.
> > > > >
> > > > > I'm replying to them below, in order of appearance:
> > > > >
> > > > > To Randall's comments:
> > > > > 1) I assumed these properties would be visible to connectors, since
> > by
> > > > > definition these are connector properties. I added a mention.
> However
> > > I'm
> > > > > not sure if you are also making a specific suggestion with this
> > > > question. I
> > > > > didn't find a similar mention in KIP-458, but 'override' directives
> > > also
> > > > > appear in both the connector and the task properties. Given this
> > > > precedent,
> > > > > I think it makes sense to forward these properties to the connector
> > as
> > > > > well.
> > > > >
> > > > > 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> > > > > definitely belongs to the Kafka Connect docs that will describe how
> > to
> > > > > operate Connect with this feature enabled.
> > > > >
> > > > > 3) Added a note to mention that a task might fail during runtime
> and
> > > that
> > > > > early validation won't be in place for this feature.
> > > > >
> > > > > 4) Examples added and the sentence regarding ACLs and failure was
> > > > adjusted
> > > > > to reflect the new proposal.
> > > > >
> > > > > 5) Also addressed and the KIP now mentions that the task will fail
> if
> > > the
> > > > > feature is enabled and the broker does not support the Admin API.
> > > > >
> > > > > To your point Ryanne, I'm also often in favor of reserving some
> room
> > > for
> > > > > customizations that will be able to address specific user needs,
> but
> > I
> > > > > don't think we have a strong case for making this functionality
> > > pluggable
> > > > > at the moment. Topics are not very transient entities in Kafka. And
> > > this
> > > > > feature is focusing specifically on topic creation and does not
> > suggest
> > > > > altering configuration of existing topics, including topics that
> may
> > be
> > > > > created once by a connector that will use this new functionality.
> > > > > Therefore, adapting to changes to the attainable replication factor
> > > > during
> > > > > runtime, without expressing this in the configuration of a
> connector
> > > > seems
> > > > > to involve more risks than benefits. Overall, a generic topic
> > creation
> > > > hook
> > > > > shares similarities to exposing an admin client to the connector
> > itself
> > > > and
> > > > > based on previous discussions, seems that this approach will result
> > in
> > > > > considerable extensions in both configuration and implementation
> > > without
> > > > it
> > > > > being fully justified at the moment.
> > > > >
> > > > > I suggest moving forward without pluggable classes for now, and if
> in
> > > the
> > > > > future we wish to return to this topic for second iteration, then
> > > > factoring
> > > > > out the proposed functionality under the configuration of a module
> > that
> > > > > applies topic creation based on regular expressions should be easy
> to
> > > do
> > > > in
> > > > > a compatible way.
> > > > >
> > > > > Best,
> > > > > Konstantine
> > > > >
> > > > >
> > > > > On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <
> ryannedolan@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Konstantine, thanks for the updates. I wonder if we should take
> > your
> > > > > > proposal one step further and make this pluggable. Your
> > > include/exclude
> > > > > > regexes are great out-of-the-box features, but it may be valuable
> > to
> > > > > > plug-in more sophisticated logic to handle topic creation.
> > > > > >
> > > > > > Instead of enabling/disabling the feature as a whole, the default
> > > > > > TopicCreator (or whatever) could be a nop. Then we include a
> > > > > > RegexTopicCreator with your proposed behavior. This would be
> almost
> > > > > > indistinguishable from your current KIP from a user's
> perspective,
> > > but
> > > > > > would enable plug-in TopicCreators that do some of the things you
> > > have
> > > > > > listed in the Rejected Alternatives, e.g. to automatically adjust
> > the
> > > > > > replication factor based on the number of nodes, etc.
> > > > > >
> > > > > > My team leverages Connect's plug-ins in other places to enable
> > > seamless
> > > > > > integration with the rest of our platform. We would definitely
> use
> > a
> > > > > topic
> > > > > > creation hook if one existed. In particular, we have a concept of
> > > > "topic
> > > > > > profiles" that we could use here.
> > > > > >
> > > > > > Ryanne
> > > > > >
> > > > > > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > > > > > konstantine@confluent.io> wrote:
> > > > > >
> > > > > > > I've taken a second look to KIP-158 after syncing with Randall
> > > Hauch,
> > > > > who
> > > > > > > was the original author of the proposal, and I have updated the
> > KIP
> > > > in
> > > > > > > place.
> > > > > > >
> > > > > > > The main new features of this updated KIP-158 is the
> introduction
> > > of
> > > > > > groups
> > > > > > > of configs that can be composed and the ability to match topics
> > to
> > > > > these
> > > > > > > groups via the use of regex. The design builds on top of the
> > > existing
> > > > > > > definition of config groups used in single message
> > transformations
> > > > > (SMT)
> > > > > > > and therefore I'm hoping that the approach fits well in Kafka
> > > > Connect's
> > > > > > > current configuration capabilities.
> > > > > > >
> > > > > > > The new proposal aims to strike a good balance between
> requiring
> > to
> > > > > > > explicitly set the configs for each possible topic or having a
> > > > > > > one-size-fits-all default set of properties for all the topics
> a
> > > > > > connector
> > > > > > > may create during runtime.
> > > > > > >
> > > > > > >
> > > > > > > The updated KIP-158 can be found under the same page as the old
> > > one:
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > > > > > >
> > > > > > > I've intentionally changed the title here in this thread to
> avoid
> > > > > > confusion
> > > > > > > with the threads that discussed KIP-158 previously.
> > > > > > > Looking forward to your comments and hoping we can pick up this
> > > work
> > > > > from
> > > > > > > the very good starting point that was reached in the previous
> > > > > > discussions.
> > > > > > >
> > > > > > >
> > > > > > > Best,
> > > > > > > Konstantine
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> >
> > --
> > -Jose
> >
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Konstantine Karantasis <ko...@confluent.io>.
Thanks for bringing up KIP-464 Jose and apologies for taking that long to
respond.

It made sense to allow the users to use the broker defaults for the
replication factor and the number of partitions when their source
connectors create topics and the implementation has incorporated this
ability.
I've now updated KIP-158 to reflect this feature in the doc. Given that
this is a minor and useful amendment I think we don't have to vote again on
this change but please let me know if you think otherwise.

Best,
Konstantine

On Mon, Feb 3, 2020 at 3:17 PM Jose Garcia Sancio <js...@confluent.io>
wrote:

> Thanks Konstantine. Looking forward to this feature.
>
> The KIP mentions:
>
> > For the *default* group this configuration is required. For any other
> group defined in topic.creation.groups this config is optional and if it's
> missing it gets the value the *default* group
>
> For the properties "topic.creation.$alias.replication.factor" and
> "topic.creation.$alias.partitions". I think that we can and should make
> this optional for all groups including the "default" group. Kafka's
> CreateTopicRequest message allows these two fields to be optional. Here are
> their descriptions respectively:
>
> > The number of replicas to create for each partition in the topic, or -1
> if we are either specifying a manual partition assignment or using the
> default repli
> cation factor.
> > The number of partitions to create in the topic, or -1 if we are either
> specifying a manual partition assignment or using the default partitions.
>
> At the Java Client level this is model using Java's Optional type. I think
> that we can make them both optional and resolve them to "Optional.empty()"
> if neither the specific group or "default" is set.
>
> Thanks,
> Jose
>
>
> On Thu, Dec 19, 2019 at 8:27 PM Tom Bentley <tb...@redhat.com> wrote:
>
> > Thanks Konstantine, lgtm.
> >
> > On Thu, Dec 19, 2019 at 5:34 PM Ryanne Dolan <ry...@gmail.com>
> > wrote:
> >
> > > Thanks for the reply Konstantine. Makes sense.
> > >
> > > Ryanne
> > >
> > > On Tue, Dec 17, 2019, 6:41 PM Konstantine Karantasis <
> > > konstantine@confluent.io> wrote:
> > >
> > > > Thanks Randall and Ryanne for your comments.
> > > >
> > > > I'm replying to them below, in order of appearance:
> > > >
> > > > To Randall's comments:
> > > > 1) I assumed these properties would be visible to connectors, since
> by
> > > > definition these are connector properties. I added a mention. However
> > I'm
> > > > not sure if you are also making a specific suggestion with this
> > > question. I
> > > > didn't find a similar mention in KIP-458, but 'override' directives
> > also
> > > > appear in both the connector and the task properties. Given this
> > > precedent,
> > > > I think it makes sense to forward these properties to the connector
> as
> > > > well.
> > > >
> > > > 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> > > > definitely belongs to the Kafka Connect docs that will describe how
> to
> > > > operate Connect with this feature enabled.
> > > >
> > > > 3) Added a note to mention that a task might fail during runtime and
> > that
> > > > early validation won't be in place for this feature.
> > > >
> > > > 4) Examples added and the sentence regarding ACLs and failure was
> > > adjusted
> > > > to reflect the new proposal.
> > > >
> > > > 5) Also addressed and the KIP now mentions that the task will fail if
> > the
> > > > feature is enabled and the broker does not support the Admin API.
> > > >
> > > > To your point Ryanne, I'm also often in favor of reserving some room
> > for
> > > > customizations that will be able to address specific user needs, but
> I
> > > > don't think we have a strong case for making this functionality
> > pluggable
> > > > at the moment. Topics are not very transient entities in Kafka. And
> > this
> > > > feature is focusing specifically on topic creation and does not
> suggest
> > > > altering configuration of existing topics, including topics that may
> be
> > > > created once by a connector that will use this new functionality.
> > > > Therefore, adapting to changes to the attainable replication factor
> > > during
> > > > runtime, without expressing this in the configuration of a connector
> > > seems
> > > > to involve more risks than benefits. Overall, a generic topic
> creation
> > > hook
> > > > shares similarities to exposing an admin client to the connector
> itself
> > > and
> > > > based on previous discussions, seems that this approach will result
> in
> > > > considerable extensions in both configuration and implementation
> > without
> > > it
> > > > being fully justified at the moment.
> > > >
> > > > I suggest moving forward without pluggable classes for now, and if in
> > the
> > > > future we wish to return to this topic for second iteration, then
> > > factoring
> > > > out the proposed functionality under the configuration of a module
> that
> > > > applies topic creation based on regular expressions should be easy to
> > do
> > > in
> > > > a compatible way.
> > > >
> > > > Best,
> > > > Konstantine
> > > >
> > > >
> > > > On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com>
> > > > wrote:
> > > >
> > > > > Konstantine, thanks for the updates. I wonder if we should take
> your
> > > > > proposal one step further and make this pluggable. Your
> > include/exclude
> > > > > regexes are great out-of-the-box features, but it may be valuable
> to
> > > > > plug-in more sophisticated logic to handle topic creation.
> > > > >
> > > > > Instead of enabling/disabling the feature as a whole, the default
> > > > > TopicCreator (or whatever) could be a nop. Then we include a
> > > > > RegexTopicCreator with your proposed behavior. This would be almost
> > > > > indistinguishable from your current KIP from a user's perspective,
> > but
> > > > > would enable plug-in TopicCreators that do some of the things you
> > have
> > > > > listed in the Rejected Alternatives, e.g. to automatically adjust
> the
> > > > > replication factor based on the number of nodes, etc.
> > > > >
> > > > > My team leverages Connect's plug-ins in other places to enable
> > seamless
> > > > > integration with the rest of our platform. We would definitely use
> a
> > > > topic
> > > > > creation hook if one existed. In particular, we have a concept of
> > > "topic
> > > > > profiles" that we could use here.
> > > > >
> > > > > Ryanne
> > > > >
> > > > > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > > > > konstantine@confluent.io> wrote:
> > > > >
> > > > > > I've taken a second look to KIP-158 after syncing with Randall
> > Hauch,
> > > > who
> > > > > > was the original author of the proposal, and I have updated the
> KIP
> > > in
> > > > > > place.
> > > > > >
> > > > > > The main new features of this updated KIP-158 is the introduction
> > of
> > > > > groups
> > > > > > of configs that can be composed and the ability to match topics
> to
> > > > these
> > > > > > groups via the use of regex. The design builds on top of the
> > existing
> > > > > > definition of config groups used in single message
> transformations
> > > > (SMT)
> > > > > > and therefore I'm hoping that the approach fits well in Kafka
> > > Connect's
> > > > > > current configuration capabilities.
> > > > > >
> > > > > > The new proposal aims to strike a good balance between requiring
> to
> > > > > > explicitly set the configs for each possible topic or having a
> > > > > > one-size-fits-all default set of properties for all the topics a
> > > > > connector
> > > > > > may create during runtime.
> > > > > >
> > > > > >
> > > > > > The updated KIP-158 can be found under the same page as the old
> > one:
> > > > > >
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > > > > >
> > > > > > I've intentionally changed the title here in this thread to avoid
> > > > > confusion
> > > > > > with the threads that discussed KIP-158 previously.
> > > > > > Looking forward to your comments and hoping we can pick up this
> > work
> > > > from
> > > > > > the very good starting point that was reached in the previous
> > > > > discussions.
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Konstantine
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
> -Jose
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Jose Garcia Sancio <js...@confluent.io>.
Thanks Konstantine. Looking forward to this feature.

The KIP mentions:

> For the *default* group this configuration is required. For any other
group defined in topic.creation.groups this config is optional and if it's
missing it gets the value the *default* group

For the properties "topic.creation.$alias.replication.factor" and
"topic.creation.$alias.partitions". I think that we can and should make
this optional for all groups including the "default" group. Kafka's
CreateTopicRequest message allows these two fields to be optional. Here are
their descriptions respectively:

> The number of replicas to create for each partition in the topic, or -1
if we are either specifying a manual partition assignment or using the
default repli
cation factor.
> The number of partitions to create in the topic, or -1 if we are either
specifying a manual partition assignment or using the default partitions.

At the Java Client level this is model using Java's Optional type. I think
that we can make them both optional and resolve them to "Optional.empty()"
if neither the specific group or "default" is set.

Thanks,
Jose


On Thu, Dec 19, 2019 at 8:27 PM Tom Bentley <tb...@redhat.com> wrote:

> Thanks Konstantine, lgtm.
>
> On Thu, Dec 19, 2019 at 5:34 PM Ryanne Dolan <ry...@gmail.com>
> wrote:
>
> > Thanks for the reply Konstantine. Makes sense.
> >
> > Ryanne
> >
> > On Tue, Dec 17, 2019, 6:41 PM Konstantine Karantasis <
> > konstantine@confluent.io> wrote:
> >
> > > Thanks Randall and Ryanne for your comments.
> > >
> > > I'm replying to them below, in order of appearance:
> > >
> > > To Randall's comments:
> > > 1) I assumed these properties would be visible to connectors, since by
> > > definition these are connector properties. I added a mention. However
> I'm
> > > not sure if you are also making a specific suggestion with this
> > question. I
> > > didn't find a similar mention in KIP-458, but 'override' directives
> also
> > > appear in both the connector and the task properties. Given this
> > precedent,
> > > I think it makes sense to forward these properties to the connector as
> > > well.
> > >
> > > 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> > > definitely belongs to the Kafka Connect docs that will describe how to
> > > operate Connect with this feature enabled.
> > >
> > > 3) Added a note to mention that a task might fail during runtime and
> that
> > > early validation won't be in place for this feature.
> > >
> > > 4) Examples added and the sentence regarding ACLs and failure was
> > adjusted
> > > to reflect the new proposal.
> > >
> > > 5) Also addressed and the KIP now mentions that the task will fail if
> the
> > > feature is enabled and the broker does not support the Admin API.
> > >
> > > To your point Ryanne, I'm also often in favor of reserving some room
> for
> > > customizations that will be able to address specific user needs, but I
> > > don't think we have a strong case for making this functionality
> pluggable
> > > at the moment. Topics are not very transient entities in Kafka. And
> this
> > > feature is focusing specifically on topic creation and does not suggest
> > > altering configuration of existing topics, including topics that may be
> > > created once by a connector that will use this new functionality.
> > > Therefore, adapting to changes to the attainable replication factor
> > during
> > > runtime, without expressing this in the configuration of a connector
> > seems
> > > to involve more risks than benefits. Overall, a generic topic creation
> > hook
> > > shares similarities to exposing an admin client to the connector itself
> > and
> > > based on previous discussions, seems that this approach will result in
> > > considerable extensions in both configuration and implementation
> without
> > it
> > > being fully justified at the moment.
> > >
> > > I suggest moving forward without pluggable classes for now, and if in
> the
> > > future we wish to return to this topic for second iteration, then
> > factoring
> > > out the proposed functionality under the configuration of a module that
> > > applies topic creation based on regular expressions should be easy to
> do
> > in
> > > a compatible way.
> > >
> > > Best,
> > > Konstantine
> > >
> > >
> > > On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com>
> > > wrote:
> > >
> > > > Konstantine, thanks for the updates. I wonder if we should take your
> > > > proposal one step further and make this pluggable. Your
> include/exclude
> > > > regexes are great out-of-the-box features, but it may be valuable to
> > > > plug-in more sophisticated logic to handle topic creation.
> > > >
> > > > Instead of enabling/disabling the feature as a whole, the default
> > > > TopicCreator (or whatever) could be a nop. Then we include a
> > > > RegexTopicCreator with your proposed behavior. This would be almost
> > > > indistinguishable from your current KIP from a user's perspective,
> but
> > > > would enable plug-in TopicCreators that do some of the things you
> have
> > > > listed in the Rejected Alternatives, e.g. to automatically adjust the
> > > > replication factor based on the number of nodes, etc.
> > > >
> > > > My team leverages Connect's plug-ins in other places to enable
> seamless
> > > > integration with the rest of our platform. We would definitely use a
> > > topic
> > > > creation hook if one existed. In particular, we have a concept of
> > "topic
> > > > profiles" that we could use here.
> > > >
> > > > Ryanne
> > > >
> > > > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > > > konstantine@confluent.io> wrote:
> > > >
> > > > > I've taken a second look to KIP-158 after syncing with Randall
> Hauch,
> > > who
> > > > > was the original author of the proposal, and I have updated the KIP
> > in
> > > > > place.
> > > > >
> > > > > The main new features of this updated KIP-158 is the introduction
> of
> > > > groups
> > > > > of configs that can be composed and the ability to match topics to
> > > these
> > > > > groups via the use of regex. The design builds on top of the
> existing
> > > > > definition of config groups used in single message transformations
> > > (SMT)
> > > > > and therefore I'm hoping that the approach fits well in Kafka
> > Connect's
> > > > > current configuration capabilities.
> > > > >
> > > > > The new proposal aims to strike a good balance between requiring to
> > > > > explicitly set the configs for each possible topic or having a
> > > > > one-size-fits-all default set of properties for all the topics a
> > > > connector
> > > > > may create during runtime.
> > > > >
> > > > >
> > > > > The updated KIP-158 can be found under the same page as the old
> one:
> > > > >
> > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > > > >
> > > > > I've intentionally changed the title here in this thread to avoid
> > > > confusion
> > > > > with the threads that discussed KIP-158 previously.
> > > > > Looking forward to your comments and hoping we can pick up this
> work
> > > from
> > > > > the very good starting point that was reached in the previous
> > > > discussions.
> > > > >
> > > > >
> > > > > Best,
> > > > > Konstantine
> > > > >
> > > >
> > >
> >
>


-- 
-Jose

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Tom Bentley <tb...@redhat.com>.
Thanks Konstantine, lgtm.

On Thu, Dec 19, 2019 at 5:34 PM Ryanne Dolan <ry...@gmail.com> wrote:

> Thanks for the reply Konstantine. Makes sense.
>
> Ryanne
>
> On Tue, Dec 17, 2019, 6:41 PM Konstantine Karantasis <
> konstantine@confluent.io> wrote:
>
> > Thanks Randall and Ryanne for your comments.
> >
> > I'm replying to them below, in order of appearance:
> >
> > To Randall's comments:
> > 1) I assumed these properties would be visible to connectors, since by
> > definition these are connector properties. I added a mention. However I'm
> > not sure if you are also making a specific suggestion with this
> question. I
> > didn't find a similar mention in KIP-458, but 'override' directives also
> > appear in both the connector and the task properties. Given this
> precedent,
> > I think it makes sense to forward these properties to the connector as
> > well.
> >
> > 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> > definitely belongs to the Kafka Connect docs that will describe how to
> > operate Connect with this feature enabled.
> >
> > 3) Added a note to mention that a task might fail during runtime and that
> > early validation won't be in place for this feature.
> >
> > 4) Examples added and the sentence regarding ACLs and failure was
> adjusted
> > to reflect the new proposal.
> >
> > 5) Also addressed and the KIP now mentions that the task will fail if the
> > feature is enabled and the broker does not support the Admin API.
> >
> > To your point Ryanne, I'm also often in favor of reserving some room for
> > customizations that will be able to address specific user needs, but I
> > don't think we have a strong case for making this functionality pluggable
> > at the moment. Topics are not very transient entities in Kafka. And this
> > feature is focusing specifically on topic creation and does not suggest
> > altering configuration of existing topics, including topics that may be
> > created once by a connector that will use this new functionality.
> > Therefore, adapting to changes to the attainable replication factor
> during
> > runtime, without expressing this in the configuration of a connector
> seems
> > to involve more risks than benefits. Overall, a generic topic creation
> hook
> > shares similarities to exposing an admin client to the connector itself
> and
> > based on previous discussions, seems that this approach will result in
> > considerable extensions in both configuration and implementation without
> it
> > being fully justified at the moment.
> >
> > I suggest moving forward without pluggable classes for now, and if in the
> > future we wish to return to this topic for second iteration, then
> factoring
> > out the proposed functionality under the configuration of a module that
> > applies topic creation based on regular expressions should be easy to do
> in
> > a compatible way.
> >
> > Best,
> > Konstantine
> >
> >
> > On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com>
> > wrote:
> >
> > > Konstantine, thanks for the updates. I wonder if we should take your
> > > proposal one step further and make this pluggable. Your include/exclude
> > > regexes are great out-of-the-box features, but it may be valuable to
> > > plug-in more sophisticated logic to handle topic creation.
> > >
> > > Instead of enabling/disabling the feature as a whole, the default
> > > TopicCreator (or whatever) could be a nop. Then we include a
> > > RegexTopicCreator with your proposed behavior. This would be almost
> > > indistinguishable from your current KIP from a user's perspective, but
> > > would enable plug-in TopicCreators that do some of the things you have
> > > listed in the Rejected Alternatives, e.g. to automatically adjust the
> > > replication factor based on the number of nodes, etc.
> > >
> > > My team leverages Connect's plug-ins in other places to enable seamless
> > > integration with the rest of our platform. We would definitely use a
> > topic
> > > creation hook if one existed. In particular, we have a concept of
> "topic
> > > profiles" that we could use here.
> > >
> > > Ryanne
> > >
> > > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > > konstantine@confluent.io> wrote:
> > >
> > > > I've taken a second look to KIP-158 after syncing with Randall Hauch,
> > who
> > > > was the original author of the proposal, and I have updated the KIP
> in
> > > > place.
> > > >
> > > > The main new features of this updated KIP-158 is the introduction of
> > > groups
> > > > of configs that can be composed and the ability to match topics to
> > these
> > > > groups via the use of regex. The design builds on top of the existing
> > > > definition of config groups used in single message transformations
> > (SMT)
> > > > and therefore I'm hoping that the approach fits well in Kafka
> Connect's
> > > > current configuration capabilities.
> > > >
> > > > The new proposal aims to strike a good balance between requiring to
> > > > explicitly set the configs for each possible topic or having a
> > > > one-size-fits-all default set of properties for all the topics a
> > > connector
> > > > may create during runtime.
> > > >
> > > >
> > > > The updated KIP-158 can be found under the same page as the old one:
> > > >
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > > >
> > > > I've intentionally changed the title here in this thread to avoid
> > > confusion
> > > > with the threads that discussed KIP-158 previously.
> > > > Looking forward to your comments and hoping we can pick up this work
> > from
> > > > the very good starting point that was reached in the previous
> > > discussions.
> > > >
> > > >
> > > > Best,
> > > > Konstantine
> > > >
> > >
> >
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Ryanne Dolan <ry...@gmail.com>.
Thanks for the reply Konstantine. Makes sense.

Ryanne

On Tue, Dec 17, 2019, 6:41 PM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> Thanks Randall and Ryanne for your comments.
>
> I'm replying to them below, in order of appearance:
>
> To Randall's comments:
> 1) I assumed these properties would be visible to connectors, since by
> definition these are connector properties. I added a mention. However I'm
> not sure if you are also making a specific suggestion with this question. I
> didn't find a similar mention in KIP-458, but 'override' directives also
> appear in both the connector and the task properties. Given this precedent,
> I think it makes sense to forward these properties to the connector as
> well.
>
> 2) Doesn't hurt to add a note in the KIP. Added in the table. This
> definitely belongs to the Kafka Connect docs that will describe how to
> operate Connect with this feature enabled.
>
> 3) Added a note to mention that a task might fail during runtime and that
> early validation won't be in place for this feature.
>
> 4) Examples added and the sentence regarding ACLs and failure was adjusted
> to reflect the new proposal.
>
> 5) Also addressed and the KIP now mentions that the task will fail if the
> feature is enabled and the broker does not support the Admin API.
>
> To your point Ryanne, I'm also often in favor of reserving some room for
> customizations that will be able to address specific user needs, but I
> don't think we have a strong case for making this functionality pluggable
> at the moment. Topics are not very transient entities in Kafka. And this
> feature is focusing specifically on topic creation and does not suggest
> altering configuration of existing topics, including topics that may be
> created once by a connector that will use this new functionality.
> Therefore, adapting to changes to the attainable replication factor during
> runtime, without expressing this in the configuration of a connector seems
> to involve more risks than benefits. Overall, a generic topic creation hook
> shares similarities to exposing an admin client to the connector itself and
> based on previous discussions, seems that this approach will result in
> considerable extensions in both configuration and implementation without it
> being fully justified at the moment.
>
> I suggest moving forward without pluggable classes for now, and if in the
> future we wish to return to this topic for second iteration, then factoring
> out the proposed functionality under the configuration of a module that
> applies topic creation based on regular expressions should be easy to do in
> a compatible way.
>
> Best,
> Konstantine
>
>
> On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com>
> wrote:
>
> > Konstantine, thanks for the updates. I wonder if we should take your
> > proposal one step further and make this pluggable. Your include/exclude
> > regexes are great out-of-the-box features, but it may be valuable to
> > plug-in more sophisticated logic to handle topic creation.
> >
> > Instead of enabling/disabling the feature as a whole, the default
> > TopicCreator (or whatever) could be a nop. Then we include a
> > RegexTopicCreator with your proposed behavior. This would be almost
> > indistinguishable from your current KIP from a user's perspective, but
> > would enable plug-in TopicCreators that do some of the things you have
> > listed in the Rejected Alternatives, e.g. to automatically adjust the
> > replication factor based on the number of nodes, etc.
> >
> > My team leverages Connect's plug-ins in other places to enable seamless
> > integration with the rest of our platform. We would definitely use a
> topic
> > creation hook if one existed. In particular, we have a concept of "topic
> > profiles" that we could use here.
> >
> > Ryanne
> >
> > On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> > konstantine@confluent.io> wrote:
> >
> > > I've taken a second look to KIP-158 after syncing with Randall Hauch,
> who
> > > was the original author of the proposal, and I have updated the KIP in
> > > place.
> > >
> > > The main new features of this updated KIP-158 is the introduction of
> > groups
> > > of configs that can be composed and the ability to match topics to
> these
> > > groups via the use of regex. The design builds on top of the existing
> > > definition of config groups used in single message transformations
> (SMT)
> > > and therefore I'm hoping that the approach fits well in Kafka Connect's
> > > current configuration capabilities.
> > >
> > > The new proposal aims to strike a good balance between requiring to
> > > explicitly set the configs for each possible topic or having a
> > > one-size-fits-all default set of properties for all the topics a
> > connector
> > > may create during runtime.
> > >
> > >
> > > The updated KIP-158 can be found under the same page as the old one:
> > >
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> > >
> > > I've intentionally changed the title here in this thread to avoid
> > confusion
> > > with the threads that discussed KIP-158 previously.
> > > Looking forward to your comments and hoping we can pick up this work
> from
> > > the very good starting point that was reached in the previous
> > discussions.
> > >
> > >
> > > Best,
> > > Konstantine
> > >
> >
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Konstantine Karantasis <ko...@confluent.io>.
Thanks Randall and Ryanne for your comments.

I'm replying to them below, in order of appearance:

To Randall's comments:
1) I assumed these properties would be visible to connectors, since by
definition these are connector properties. I added a mention. However I'm
not sure if you are also making a specific suggestion with this question. I
didn't find a similar mention in KIP-458, but 'override' directives also
appear in both the connector and the task properties. Given this precedent,
I think it makes sense to forward these properties to the connector as
well.

2) Doesn't hurt to add a note in the KIP. Added in the table. This
definitely belongs to the Kafka Connect docs that will describe how to
operate Connect with this feature enabled.

3) Added a note to mention that a task might fail during runtime and that
early validation won't be in place for this feature.

4) Examples added and the sentence regarding ACLs and failure was adjusted
to reflect the new proposal.

5) Also addressed and the KIP now mentions that the task will fail if the
feature is enabled and the broker does not support the Admin API.

To your point Ryanne, I'm also often in favor of reserving some room for
customizations that will be able to address specific user needs, but I
don't think we have a strong case for making this functionality pluggable
at the moment. Topics are not very transient entities in Kafka. And this
feature is focusing specifically on topic creation and does not suggest
altering configuration of existing topics, including topics that may be
created once by a connector that will use this new functionality.
Therefore, adapting to changes to the attainable replication factor during
runtime, without expressing this in the configuration of a connector seems
to involve more risks than benefits. Overall, a generic topic creation hook
shares similarities to exposing an admin client to the connector itself and
based on previous discussions, seems that this approach will result in
considerable extensions in both configuration and implementation without it
being fully justified at the moment.

I suggest moving forward without pluggable classes for now, and if in the
future we wish to return to this topic for second iteration, then factoring
out the proposed functionality under the configuration of a module that
applies topic creation based on regular expressions should be easy to do in
a compatible way.

Best,
Konstantine


On Thu, Dec 12, 2019 at 1:37 PM Ryanne Dolan <ry...@gmail.com> wrote:

> Konstantine, thanks for the updates. I wonder if we should take your
> proposal one step further and make this pluggable. Your include/exclude
> regexes are great out-of-the-box features, but it may be valuable to
> plug-in more sophisticated logic to handle topic creation.
>
> Instead of enabling/disabling the feature as a whole, the default
> TopicCreator (or whatever) could be a nop. Then we include a
> RegexTopicCreator with your proposed behavior. This would be almost
> indistinguishable from your current KIP from a user's perspective, but
> would enable plug-in TopicCreators that do some of the things you have
> listed in the Rejected Alternatives, e.g. to automatically adjust the
> replication factor based on the number of nodes, etc.
>
> My team leverages Connect's plug-ins in other places to enable seamless
> integration with the rest of our platform. We would definitely use a topic
> creation hook if one existed. In particular, we have a concept of "topic
> profiles" that we could use here.
>
> Ryanne
>
> On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
> konstantine@confluent.io> wrote:
>
> > I've taken a second look to KIP-158 after syncing with Randall Hauch, who
> > was the original author of the proposal, and I have updated the KIP in
> > place.
> >
> > The main new features of this updated KIP-158 is the introduction of
> groups
> > of configs that can be composed and the ability to match topics to these
> > groups via the use of regex. The design builds on top of the existing
> > definition of config groups used in single message transformations (SMT)
> > and therefore I'm hoping that the approach fits well in Kafka Connect's
> > current configuration capabilities.
> >
> > The new proposal aims to strike a good balance between requiring to
> > explicitly set the configs for each possible topic or having a
> > one-size-fits-all default set of properties for all the topics a
> connector
> > may create during runtime.
> >
> >
> > The updated KIP-158 can be found under the same page as the old one:
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
> >
> > I've intentionally changed the title here in this thread to avoid
> confusion
> > with the threads that discussed KIP-158 previously.
> > Looking forward to your comments and hoping we can pick up this work from
> > the very good starting point that was reached in the previous
> discussions.
> >
> >
> > Best,
> > Konstantine
> >
>

Re: [DISCUSS] KIP-158 UPDATED: Enable source connectors to create new topics with specific configs in Kafka Connect during runtime

Posted by Ryanne Dolan <ry...@gmail.com>.
Konstantine, thanks for the updates. I wonder if we should take your
proposal one step further and make this pluggable. Your include/exclude
regexes are great out-of-the-box features, but it may be valuable to
plug-in more sophisticated logic to handle topic creation.

Instead of enabling/disabling the feature as a whole, the default
TopicCreator (or whatever) could be a nop. Then we include a
RegexTopicCreator with your proposed behavior. This would be almost
indistinguishable from your current KIP from a user's perspective, but
would enable plug-in TopicCreators that do some of the things you have
listed in the Rejected Alternatives, e.g. to automatically adjust the
replication factor based on the number of nodes, etc.

My team leverages Connect's plug-ins in other places to enable seamless
integration with the rest of our platform. We would definitely use a topic
creation hook if one existed. In particular, we have a concept of "topic
profiles" that we could use here.

Ryanne

On Thu, Dec 12, 2019 at 2:00 PM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> I've taken a second look to KIP-158 after syncing with Randall Hauch, who
> was the original author of the proposal, and I have updated the KIP in
> place.
>
> The main new features of this updated KIP-158 is the introduction of groups
> of configs that can be composed and the ability to match topics to these
> groups via the use of regex. The design builds on top of the existing
> definition of config groups used in single message transformations (SMT)
> and therefore I'm hoping that the approach fits well in Kafka Connect's
> current configuration capabilities.
>
> The new proposal aims to strike a good balance between requiring to
> explicitly set the configs for each possible topic or having a
> one-size-fits-all default set of properties for all the topics a connector
> may create during runtime.
>
>
> The updated KIP-158 can be found under the same page as the old one:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-158%3A+Kafka+Connect+should+allow+source+connectors+to+set+topic-specific+settings+for+new+topics
>
> I've intentionally changed the title here in this thread to avoid confusion
> with the threads that discussed KIP-158 previously.
> Looking forward to your comments and hoping we can pick up this work from
> the very good starting point that was reached in the previous discussions.
>
>
> Best,
> Konstantine
>