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 <tbent...@redhat.com> wrote: > Thanks Konstantine, lgtm. > > On Thu, Dec 19, 2019 at 5:34 PM Ryanne Dolan <ryannedo...@gmail.com> > wrote: > > > Thanks for the reply Konstantine. Makes sense. > > > > Ryanne > > > > On Tue, Dec 17, 2019, 6:41 PM Konstantine Karantasis < > > konstant...@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 <ryannedo...@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 < > > > > konstant...@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