Hi,

Thanks again for all of the feedback. Based upon this feedback, I do agree
that we should indeed just solve the simple problem for the vast majority
of use cases that only require a few simple properties. I believe I was the
only person advocating for a more general and flexible solution, but that
flexibility is simply not worth the much greater complexity.

So, I've dramatically simplified the KIP as follows:

   1. The KIP no longer proposes changes to the Java API. Source connector
   implementations would only be able to use this API if and only if they are
   willing to constrain the connectors to be deployed to AK 2.2 (or whichever
   is the first version that includes this feature) or later. I would be
   surprised if very many source connector developers would take advantage of
   this feature. Plus, removing this from the proposal eliminated a lot of
   complexity.
   2. A new `topic.creation.enable` Connect worker configuration property
   allows a Connect cluster operator to control whether or not connectors can
   use this feature. It defaults to not enabling the feature, which makes this
   fully backward compatible.
   3. The new properties specified in the source connector configurations
   are no longer rule based and are straightforward: the default replication
   factor, number of partitions, and other topic-specific settings for any and
   all new topics created by Connect. This also eliminates a lot of complexity
   of the design and should make using this feature much easier.

I would love for this to get into AK 2.2, which means voting needs to start
in the next few days. Short notice, but hopefully this smaller and simpler
proposal is something we can all agree to. Let's start simple and learn
from our users whether or not they need more flexibility and control.

Please respond with your thoughts. Thanks!

Best regards,

Randall

On Tue, Nov 27, 2018 at 7:36 PM Ryanne Dolan <ryannedo...@gmail.com> wrote:

> Randall, have you considered something like:
>
> - introduce TopicCreationPolicy interface, with methods like
> partitionsForTopic(topic).
> - provide a DefaultTopicCreationPolicy implementation that implements the
> current behavior.
> - provide a SimpleTopicCreationPolicy that honors topic.creation.partitions
> property, etc.
> - perhaps also provide a RegexTopicCreationPolicy.
> - users can provide their own TopicCreationPolicy implementations when
> necessary.
> - support topic.creation.policy.class property in worker configs, with
> default = org.apache.kafka.connect.DefaultTopicCreationPolicy.
>
> This way, the only new configuration property is
> topic.creation.policy.class. The default behavior doesn't change from what
> we have today. If a user wants to change from the default, they can opt-in
> to one of the other policies or implement their own.
>
> Ryanne
>
> On Tue, Nov 27, 2018 at 6:31 PM Randall Hauch <rha...@gmail.com> wrote:
>
> > Thanks for the feedback. Some thoughts inline.
> >
> > On Tue, Nov 27, 2018 at 5:47 PM Ewen Cheslack-Postava <e...@confluent.io
> >
> > wrote:
> >
> > > re: AdminClient vs this proposal, one consideration is that AdminClient
> > > exposes a lot more surface area and probably a bunch of stuff we
> actually
> > > don't want Connectors to be able to do, such as deleting topics. You
> can
> > > always lock down by ACLs, but what the framework enables directly vs
> > > requiring the user to opt in via connector-specific config is an
> > important
> > > distinction.
> > >
> > > I'm not a fan of how complex the config is (same deal with
> > > transformations), and agree with Ryanne that any case requiring
> multiple
> > > rules is probably an outlier. A cleaner option for the common case
> might
> > be
> > > worth it. One option that's still aligned with the current state of the
> > KIP
> > > would be to change the default for topic.creation to a fixed default
> > value
> > > (e.g. 'default'), if that's the case turn the
> > topic.creation.default.regex
> > > default to .*, and then 99% use case would just be specifying the # of
> > > partitions with a single config and relying on cluster defaults for the
> > > rest. (I would suggest the same thing for transformations if we added a
> > > simple scripting transformation such that most use cases wouldn't need
> to
> > > compose multiple transformations.)
> > >
> >
> > I agree that any case requiring multiple rules is probably an outlier,
> and
> > I've been trying to think about how to start simple with a single case
> but
> > leave room if we really do need multiple rules in the future. I like
> Ewen's
> > suggestion a lot. IIUC, it would change the following common case:
> >
> > topic.creation=default
> > topic.creation.default.regex=.*
> > topic.creation.default.partitions=5
> >
> > into the following:
> >
> > topic.creation.default.partitions=5
> >
> > where the replication defaults to 3 and all others default to the
> brokers'
> > default topic settings. This is significantly simpler, yet it still
> allows
> > us to handle multiple rules if they're needed.
> >
> > Another common case is to use compacted topics, so that might required
> > adding:
> >
> > topic.creation.default.cleanup.policy=compact
> >
> > Also, I like the idea of defaulting the regex to '.*', but I wonder if
> it'd
> > be easier to explain if that default value applied to the *last* rule in
> > the list of rules, rather than to apply only to the rule named "default"
> > when that's the only named rule. WDYT?
> >
> >
> > >
> > > Along related lines, is there actually a need for TopicSettings class?
> We
> > > already have NewTopic in the AdminClient APIs. Does that not suffice?
> > >
> >
> > There are three reasons I added TopicSettings and didn't simply use
> > NewTopic. First, NewTopic is immutable, which makes it impractical for
> > Connect to pass to a connector and to allow the connector to change it.
> > Second, TopicSettings is essentially a builder with easy to use and
> > chainable methods, whereas NewTopic relies upon Map<String, String>,
> String
> > constants (in another class), and essentially untyped values. Third,
> > NewTopic is not an interface, and I think it's better to expose an
> > interface in the connector API when we don't want/expect connectors to
> > instantiate the instance and to instead only use what we provide them.
> >
> > Now, another option is to move TopicSettings into the
> > `org.apache.kafka.clients.admin` package and turn it into
> `NewTopicBuilder`
> > instead. Then it'd be useful outside of Connect, but is it strange that
> > it's not in the Connect API packages?
> >
> > Randall
> >
> >
> > >
> > > -Ewen
> > >
> > > On Mon, Sep 24, 2018 at 11:56 AM Andrew Otto <o...@wikimedia.org>
> wrote:
> > >
> > > > FWIW, I’d find this feature useful.
> > > >
> > > > On Mon, Sep 24, 2018 at 2:42 PM Randall Hauch <rha...@gmail.com>
> > wrote:
> > > >
> > > > > Ryanne,
> > > > >
> > > > > If your connector is already using the AdminClient, then you as the
> > > > > developer have a choice of switching to the new Connect-based
> > > > functionality
> > > > > or keeping the existing use of the AdminClient. If the connector
> uses
> > > > both
> > > > > mechanisms (which I wouldn't recommend, simply because of the
> > > complexity
> > > > of
> > > > > it for a user), then the topic will be created by the first
> mechanism
> > > to
> > > > > actually attempt and successfully create the topic(s) in the Kafka
> > > > cluster
> > > > > that the Connect worker uses. As mentioned in the KIP, "This
> feature
> > > ...
> > > > > does not change the topic-specific settings on any existing
> topics."
> > > IOW,
> > > > > if the topic already exists, it can't be created again and
> therefore
> > > the
> > > > > `topic.creation.*` properties will not apply for that existing
> topic.
> > > > >
> > > > > > Do these settings apply to internal topics created by the
> framework
> > > on
> > > > > > bahalf of a connector, e.g. via KafkaConfigBackingStore?
> > > > >
> > > > > No, they don't, and I'm happy to add a clarification to the KIP if
> > you
> > > > feel
> > > > > it is necessary.
> > > > >
> > > > > > I'd have the same questions if e.g. transformations could be
> > ignored
> > > or
> > > > > > overridden by connectors or if there were multiple places to
> > specify
> > > > what
> > > > > > serde to use.
> > > > >
> > > > > There are multiple places that converters can be defined: the
> worker
> > > > config
> > > > > defines the key and value converters that will be used for all
> > > > connectors,
> > > > > except when a connector defines its own key and value converters.
> > > > >
> > > > > > I don't see how controlling topic creation based on topic name is
> > > > > something
> > > > > > we should support across all connectors, as if it is some
> > established
> > > > > > pattern or universally useful.
> > > > >
> > > > > Topics are identified by name, and when you create a topic with
> > > specific
> > > > > settings or change a topic's settings you identify the topic by
> name.
> > > The
> > > > > fact that this KIP uses regular expressions to match topic names
> > > doesn't
> > > > > seem surprising, since we use regexes elsewhere.
> > > > >
> > > > > Best regards
> > > > >
> > > > > On Mon, Sep 24, 2018 at 1:24 PM Ryanne Dolan <
> ryannedo...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Randall,
> > > > > >
> > > > > > Say I've got a connector that needs to control topic creation.
> > What I
> > > > > need
> > > > > > is an AdminClient s.t. my connector can do what it knows it needs
> > to
> > > > do.
> > > > > > This KIP doesn't address the issues that have been brought up wrt
> > > > > > configuration, principals, ACL etc, since I'll still need to
> > > construct
> > > > my
> > > > > > own AdminClient.
> > > > > >
> > > > > > Should such a connector ignore your proposed configuration
> > settings?
> > > > > Should
> > > > > > it use it's own principal and it's own configuration properties?
> > How
> > > > does
> > > > > > my AdminClient's settings interact with your proposed settings
> and
> > > the
> > > > > > existing cluster settings?
> > > > > >
> > > > > > What happens when a user specifies topic creation settings in a
> > > > connector
> > > > > > config, but the connector then applies it's own topic creation
> > logic?
> > > > Are
> > > > > > the configurations silently ignored? If not, how can a connector
> > > honor
> > > > > your
> > > > > > proposed settings?
> > > > > >
> > > > > > Do these settings apply to internal topics created by the
> framework
> > > on
> > > > > > bahalf of a connector, e.g. via KafkaConfigBackingStore?
> > > > > >
> > > > > > When do the cluster settings get applied? Only after 3 layers of
> > > > > > fall-through?
> > > > > >
> > > > > > I'd have the same questions if e.g. transformations could be
> > ignored
> > > or
> > > > > > overridden by connectors or if there were multiple places to
> > specify
> > > > what
> > > > > > serde to use.
> > > > > >
> > > > > > I don't see how controlling topic creation based on topic name is
> > > > > something
> > > > > > we should support across all connectors, as if it is some
> > established
> > > > > > pattern or universally useful.
> > > > > >
> > > > > > Ryanne
> > > > > >
> > > > > > On Mon, Sep 24, 2018, 10:14 AM Randall Hauch <rha...@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Hi, Ryanne. My apologies for not responding earlier, as I was
> on
> > a
> > > > long
> > > > > > > holiday.
> > > > > > >
> > > > > > > Thanks for your feedback and questions about this KIP. You've
> > > raised
> > > > > > > several points in the discussion so far, so let me try to
> address
> > > > most
> > > > > of
> > > > > > > them.
> > > > > > >
> > > > > > > IIUC, one of your major concerns is that this KIP introduces a
> > new
> > > > way
> > > > > to
> > > > > > > define configurations for topics. That is true, and the whole
> > > reason
> > > > is
> > > > > > to
> > > > > > > simply the user experience for people using source connectors.
> > You
> > > > > still
> > > > > > > have the freedom to manually pre-create topics before running a
> > > > > > connector,
> > > > > > > or to rely upon the broker automatically creating topics for
> the
> > > > > > connectors
> > > > > > > when those topics don't yet exist -- in both cases, don't
> include
> > > > > > anything
> > > > > > > about topic creation in your connector configurations. In fact,
> > > when
> > > > > you
> > > > > > do
> > > > > > > this, Connect uses the current behavior by assuming the topics
> > > exist
> > > > or
> > > > > > > will be autocreated with the proper configurations.
> > > > > > >
> > > > > > > But for many environments, this existing approach is not
> enough.
> > > > First,
> > > > > > if
> > > > > > > you're relying upon the broker to autocreate topics, then the
> > > brokers
> > > > > > > single set of default topic settings must match the
> requirements
> > of
> > > > > your
> > > > > > > new topics. This can be difficult when your running multiple
> > kinds
> > > of
> > > > > > > connectors with differing expectations. Consider using a CDC
> > > > connector
> > > > > > that
> > > > > > > expects compaction, a high-volume web service connector that
> > should
> > > > not
> > > > > > use
> > > > > > > compaction but expects deletion after 7 days, and another
> > connector
> > > > > with
> > > > > > > lower volume that uses 30 day retention. Or, consider
> connectors
> > > that
> > > > > are
> > > > > > > producing to topics that have very different message
> > > characteristics:
> > > > > > > different sizes, different throughputs, different partitions,
> > etc.
> > > > The
> > > > > > only
> > > > > > > way to work around this is to pre-create the topics, but this
> > adds
> > > > more
> > > > > > > complexity and room for errors, especially when a single
> instance
> > > of
> > > > > some
> > > > > > > source connectors can write to dozens (or even hundreds) of
> > topics.
> > > > > > >
> > > > > > > Second, many operators prefer (or are required) to disable
> topic
> > > > > > > autocreation, since simple mistakes with command line tools can
> > > > result
> > > > > in
> > > > > > > new topics. In this cases, users have no choice but to manually
> > > > > precreate
> > > > > > > the topics that complicates the process of running a connector
> > and,
> > > > as
> > > > > > > mentioned above, increases the risk that something goes wrong.
> > > > > > >
> > > > > > > Third, the reason why this KIP introduces a way for connector
> > > > > > > implementations to override some topic settings is because some
> > > > source
> > > > > > > connectors have very specific requirements. When I wrote the
> > first
> > > > > > Debezium
> > > > > > > CDC connectors, many first-time users didn't precreate the
> topics
> > > as
> > > > > > > recommended in the documentation, and didn't change their
> > brokers'
> > > > > > default
> > > > > > > topic settings. Only after a few days when they tried
> reconsuming
> > > the
> > > > > > full
> > > > > > > streams did they realize that Kafka had deleted messages older
> > than
> > > > the
> > > > > > > default retention period. Debezium expects / requires compacted
> > > > topics,
> > > > > > so
> > > > > > > all kinds of things went wrong. Connect is often one of the
> first
> > > > ways
> > > > > in
> > > > > > > which people get introduced to Kafka, and they simply don't yet
> > > have
> > > > an
> > > > > > > understanding of many of the details that you or I don't have
> to
> > > > think
> > > > > > > twice about.
> > > > > > >
> > > > > > > You suggested that maybe Connect should just expose the Admin
> > API.
> > > > > That's
> > > > > > > possible, but IMO it's very heavyweight and complex. The whole
> > > point
> > > > of
> > > > > > > Connect's design is to abstract the connector developer away
> from
> > > > most
> > > > > of
> > > > > > > the details of Kafka -- it doesn't even expose the producer and
> > > > > consumer
> > > > > > > APIs, which are much simpler. IMO it would be a mistake to
> > require
> > > > > source
> > > > > > > connector developers to deal with the Admin API -- I even have
> > > > trouble
> > > > > > > writing code that uses it to properly create topics, especially
> > > > around
> > > > > > > properly handling all of the potential error conditions.
> > > > > > >
> > > > > > > You also mention that topic settings in a connector
> configuration
> > > > might
> > > > > > not
> > > > > > > reflect the actual topic's settings. This is true, especially
> if
> > > the
> > > > > > topic
> > > > > > > already existed with different settings before the connector
> was
> > > run.
> > > > > > > However, this is also very true of the broker's default topic
> > > > settings,
> > > > > > > which very often don't reflect the actual settings for all of
> the
> > > > > topics
> > > > > > --
> > > > > > > the defaults may have been changed, or topics are created
> > manually
> > > > with
> > > > > > > very different settings. The only way to know the settings of a
> > > > > > particular
> > > > > > > topic are to get them for that topic.
> > > > > > >
> > > > > > > The use of naming rules in the topic creation settings is
> > > > intentional,
> > > > > > and
> > > > > > > it allows connector users to define topic settings for topics
> > based
> > > > > upon
> > > > > > > the names. In some cases this may require several rules to
> handle
> > > the
> > > > > > > different topics, but most of the time a single rule may be all
> > > > that's
> > > > > > > required. I also don't agree that users will start naming
> topics
> > to
> > > > > > > simplify their rules -- many source connectors that write to
> more
> > > > than
> > > > > > one
> > > > > > > topic often don't allow the user to specify the full name of
> the
> > > > topics
> > > > > > > anyway, and when they do they often only write to one topic.
> > > > > > >
> > > > > > > I still think that the proposed KIP provides a simple way for
> > most
> > > > > source
> > > > > > > connector users to control (via configuration) the settings of
> > the
> > > > > topics
> > > > > > > that will be created by Connect for that connector, which works
> > > with
> > > > > all
> > > > > > > existing source connectors out of the box and does not add
> > > additional
> > > > > > > complexities for source connector developers.
> > > > > > >
> > > > > > > Best regards,
> > > > > > >
> > > > > > > Randall
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Wed, Sep 12, 2018 at 12:22 PM Ryanne Dolan <
> > > ryannedo...@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > > > > Rather than go though the connect framework, connectors
> > should
> > > > just
> > > > > > > > create their own AdminClient instance and create their own
> > > topics?
> > > > > > > >
> > > > > > > > Rather, can the framework be improved to expose an
> AdminClient
> > > > ready
> > > > > to
> > > > > > > > use? Then connectors can use this instance without needing
> > > separate
> > > > > > > > identities/principals and associated configuration (which I
> > > totally
> > > > > > > > understand would be a nightmare). I believe that covers all
> the
> > > > > > > use-cases?
> > > > > > > >
> > > > > > > > I just don't see how the "terrible config situation" is
> > remedied
> > > by
> > > > > > > adding
> > > > > > > > even more configuration.
> > > > > > > >
> > > > > > > > Also, I'm not sure I can conceive of a use-case in which a
> > single
> > > > > > > connector
> > > > > > > > would need multiple default topic settings *based on the
> topic
> > > > name*.
> > > > > > Can
> > > > > > > > you give a real-world example? Is this something you've
> > > > encountered,
> > > > > or
> > > > > > > are
> > > > > > > > you just trying for a flexible design?
> > > > > > > >
> > > > > > > > Ryanne
> > > > > > > >
> > > > > > > > On Tue, Sep 11, 2018 at 9:57 PM Gwen Shapira <
> > g...@confluent.io>
> > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hi Ryanne,
> > > > > > > > >
> > > > > > > > > Thanks for the feedback!
> > > > > > > > >
> > > > > > > > > Can you explain a bit more what you mean by "if we allow
> > > > connectors
> > > > > > to
> > > > > > > > make
> > > > > > > > > this
> > > > > > > > > decision, they should have full control of the process."?
> > > > > > > > >
> > > > > > > > > I assume you mean, something like:
> > > > > > > > > Rather than go though the connect framework, connectors
> > should
> > > > just
> > > > > > > > create
> > > > > > > > > their own AdminClient instance and create their own topics?
> > > > > > > > >
> > > > > > > > > The problem with this approach is that connectors currently
> > > don't
> > > > > > have
> > > > > > > > > their own identity (in the authentication/authorization
> > sense).
> > > > All
> > > > > > > > > connectors share the framework identity, if the users need
> to
> > > > start
> > > > > > > > > configuring security for both the framework and connect
> > itself,
> > > > it
> > > > > > gets
> > > > > > > > > messy rather quickly.
> > > > > > > > > We actually already do the thing I'm imagining you
> suggested
> > in
> > > > > some
> > > > > > > > > connectors right now (create AdminClient and configure
> > topics),
> > > > and
> > > > > > we
> > > > > > > > hope
> > > > > > > > > to use the new framework capability to clean-up the
> > > configuration
> > > > > > mess
> > > > > > > > this
> > > > > > > > > has caused. I spent 4 days trying to figure out what a
> > specific
> > > > > > > connector
> > > > > > > > > doesn't work, just to find out that you need to give it its
> > own
> > > > > > > security
> > > > > > > > > config because it has an AdminClient so the configuration
> on
> > > the
> > > > > > > > framework
> > > > > > > > > isn't enough.
> > > > > > > > >
> > > > > > > > > From my experience with rather large number of customers,
> > there
> > > > are
> > > > > > > some
> > > > > > > > > companies where the topics are controlled by a central team
> > > that
> > > > > owns
> > > > > > > all
> > > > > > > > > the machinery to create and configure topics (sometimes via
> > > > gitops,
> > > > > > > > > kubernetes custom resources, etc) and they would indeed be
> > very
> > > > > > > surprised
> > > > > > > > > if a connector suddenly had opinions about topics. There
> are
> > > also
> > > > > > teams
> > > > > > > > > where the application developers feel like they know their
> > data
> > > > and
> > > > > > > > > use-case the best and they are in-charge of making all
> > > > topic-level
> > > > > > > > > decisions, usually automated by the app itself. Admin
> client
> > > was
> > > > > > > created
> > > > > > > > > for those teams and I think they'll appreciate having this
> > > > > capability
> > > > > > > in
> > > > > > > > > connect too. Funny thing is, customers who work with one
> > model
> > > > > > usually
> > > > > > > > > can't believe the other model even exists.
> > > > > > > > >
> > > > > > > > > I'd love to propose a compromise and suggest that we'll
> allow
> > > > this
> > > > > > > > > functionality in Connect but also give ops teams the option
> > to
> > > > > > disable
> > > > > > > it
> > > > > > > > > and avoid surprises. But I'm afraid this wont work - too
> > often
> > > > the
> > > > > > > > defaults
> > > > > > > > > are just terrible for specific connectors (CDC connectors
> > > > sometimes
> > > > > > > need
> > > > > > > > a
> > > > > > > > > single partition to maintain consistency) and if there is a
> > > > chance
> > > > > > the
> > > > > > > > > connector preference won't be used, connectors will have to
> > > force
> > > > > it
> > > > > > > via
> > > > > > > > > admin client which brings us back to the terrible config
> > > > situation
> > > > > we
> > > > > > > > > currently have with Admin client.
> > > > > > > > >
> > > > > > > > > Gwen
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > On Tue, Sep 11, 2018 at 7:23 PM, Ryanne Dolan <
> > > > > ryannedo...@gmail.com
> > > > > > >
> > > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > Randall,
> > > > > > > > > >
> > > > > > > > > > I have some concerns with this proposal.
> > > > > > > > > >
> > > > > > > > > > Firstly, I don't believe it is the job of a connector to
> > > > > configure
> > > > > > > > > topics,
> > > > > > > > > > generally, nor for topic-specific settings to hang out in
> > > > > connector
> > > > > > > > > > configurations. Automatic creation of topics with default
> > > > > settings
> > > > > > is
> > > > > > > > an
> > > > > > > > > > established pattern elsewhere, and I don't think
> connectors
> > > > need
> > > > > to
> > > > > > > > > diverge
> > > > > > > > > > from this.
> > > > > > > > > >
> > > > > > > > > > I agree there are cases where the default settings don't
> > make
> > > > > sense
> > > > > > > and
> > > > > > > > > > it'd be nice to override them. But if we allow connectors
> > to
> > > > make
> > > > > > > this
> > > > > > > > > > decision, they should have full control of the process.
> > > > > > > > > >
> > > > > > > > > > Some concerns:
> > > > > > > > > > - I'd expect the cluster's default settings to apply to
> > newly
> > > > > > created
> > > > > > > > > > topics, regardless of who created them. I wouldn't expect
> > > > source
> > > > > > > > > connectors
> > > > > > > > > > to be a special case. In particular, I'd be surprised if
> > > Kafka
> > > > > > > Connect
> > > > > > > > > were
> > > > > > > > > > to ignore my cluster's default settings and apply its own
> > > > > defaults.
> > > > > > > > > > - It will be possible to add a specific topic to this
> > > > > > configuration,
> > > > > > > in
> > > > > > > > > > which case any reader would expect the topic to have the
> > > > > specified
> > > > > > > > > > settings. But this will not generally be true. Thus, the
> > > > > > > configuration
> > > > > > > > > will
> > > > > > > > > > end up lying and misleading, and there won't be any
> > > indication
> > > > > that
> > > > > > > the
> > > > > > > > > > configuration is lying.
> > > > > > > > > > - Connectors that want to control settings will end up
> > naming
> > > > > > topics
> > > > > > > > > > accordingly. For example, a connector that wants to
> control
> > > the
> > > > > > > number
> > > > > > > > of
> > > > > > > > > > partitions would need a bunch of creation rules for 1
> > > > partition,
> > > > > 2
> > > > > > > > > > partitions and so on. This is a bad pattern to
> establish. A
> > > > > better
> > > > > > > > > pattern
> > > > > > > > > > is to let the connector control the number of partitions
> > > > directly
> > > > > > > when
> > > > > > > > > that
> > > > > > > > > > feature is required.
> > > > > > > > > > - The proposal introduces 2 new interfaces to control
> topic
> > > > > > creation
> > > > > > > > > > (configuration rules and TopicSettings), where there is
> > > > already a
> > > > > > > > > perfectly
> > > > > > > > > > good one (AdminClient).
> > > > > > > > > >
> > > > > > > > > > Ryanne
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > >
> > > > > > > > > > On Tue, Sep 4, 2018 at 5:08 PM Randall Hauch <
> > > rha...@gmail.com
> > > > >
> > > > > > > wrote:
> > > > > > > > > >
> > > > > > > > > > > Okay, I think I cleaned up the formatting issues in the
> > KIP
> > > > > wiki
> > > > > > > > page.
> > > > > > > > > > And
> > > > > > > > > > > while implementing I realized that it'd be helpful to
> be
> > > able
> > > > > to
> > > > > > > > limit
> > > > > > > > > > via
> > > > > > > > > > > the connector configuration and the rules which topics
> > are
> > > > > > > created. I
> > > > > > > > > > added
> > > > > > > > > > > the `topic.creation.${ruleName}.policy` behavior, with
> > > > possible
> > > > > > > > values
> > > > > > > > > > of
> > > > > > > > > > > "create" (the default), "autocreate" (to specify that
> > > Connect
> > > > > > > should
> > > > > > > > > let
> > > > > > > > > > > the broker autocreate any matching topics), and "fail"
> > (to
> > > > > > specify
> > > > > > > > that
> > > > > > > > > > > Connect should not allow the creation of topics whose
> > names
> > > > > match
> > > > > > > the
> > > > > > > > > > > rule's regular expression).
> > > > > > > > > > >
> > > > > > > > > > > Let me know what you think. I'd like to start voting
> > soon,
> > > > but
> > > > > > > > because
> > > > > > > > > I
> > > > > > > > > > > made the above change I'll wait a few days.
> > > > > > > > > > >
> > > > > > > > > > > Best regards,
> > > > > > > > > > >
> > > > > > > > > > > Randall
> > > > > > > > > > >
> > > > > > > > > > > On Wed, Aug 29, 2018 at 9:41 PM Randall Hauch <
> > > > > rha...@gmail.com>
> > > > > > > > > wrote:
> > > > > > > > > > >
> > > > > > > > > > > > Thanks, Magesh.
> > > > > > > > > > > >
> > > > > > > > > > > > All, I've made a few very minor changes to some
> > JavaDocs
> > > > and
> > > > > > the
> > > > > > > > > > > > signatures of the name-value pair methods in
> > > TopicSettings
> > > > > > > > > interface. I
> > > > > > > > > > > > also described as a fifth rejected alternative why
> this
> > > KIP
> > > > > > does
> > > > > > > > not
> > > > > > > > > > > modify
> > > > > > > > > > > > any topic settings for existing topics. All of these
> > are
> > > > > pretty
> > > > > > > > > minor,
> > > > > > > > > > > but
> > > > > > > > > > > > I'm happy to hear about issues or suggestions.
> > > > > > > > > > > >
> > > > > > > > > > > > Since the above changes were very minor, I'll kick
> off
> > a
> > > > vote
> > > > > > to
> > > > > > > > > accept
> > > > > > > > > > > > this KIP unless I hear something in the next day or
> > two.
> > > > Note
> > > > > > > that
> > > > > > > > > this
> > > > > > > > > > > KIP
> > > > > > > > > > > > has been around in virtually the exact form for over
> a
> > > > year.
> > > > > > > > > > > >
> > > > > > > > > > > > Best regards,
> > > > > > > > > > > >
> > > > > > > > > > > > Randall
> > > > > > > > > > > >
> > > > > > > > > > > > On Wed, Aug 29, 2018 at 9:17 PM Magesh Nandakumar <
> > > > > > > > > > mage...@confluent.io>
> > > > > > > > > > > > wrote:
> > > > > > > > > > > >
> > > > > > > > > > > >> Randall,
> > > > > > > > > > > >>
> > > > > > > > > > > >> I originally thought that this proposal was a config
> > > only
> > > > > > topic
> > > > > > > > > > settings
> > > > > > > > > > > >> and hence made the comment about configs being pass
> > > > > through. I
> > > > > > > > just
> > > > > > > > > > > >> realized that the connectors can also override and
> > > provide
> > > > > the
> > > > > > > > > > > >> TopicSettings. With that in mind, I think the
> proposal
> > > > looks
> > > > > > > > great.
> > > > > > > > > > > >> Looking forward to the feature.
> > > > > > > > > > > >>
> > > > > > > > > > > >> Thanks,
> > > > > > > > > > > >> Magesh
> > > > > > > > > > > >>
> > > > > > > > > > > >> On Tue, Aug 28, 2018 at 8:53 PM Magesh Nandakumar <
> > > > > > > > > > mage...@confluent.io
> > > > > > > > > > > >
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >>
> > > > > > > > > > > >> > I was wondering if it would be much simpler to
> just
> > > do a
> > > > > > > > > > pass-through
> > > > > > > > > > > so
> > > > > > > > > > > >> > that we can support any topic setting added in
> Kafka
> > > > > without
> > > > > > > any
> > > > > > > > > > code
> > > > > > > > > > > >> > changes in connect. Since these are for topics
> that
> > > will
> > > > > > have
> > > > > > > > the
> > > > > > > > > > > actual
> > > > > > > > > > > >> > data stream, users might possibly need more
> > > flexibility
> > > > in
> > > > > > > terms
> > > > > > > > > of
> > > > > > > > > > > how
> > > > > > > > > > > >> the
> > > > > > > > > > > >> > topics get created.
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > Thanks
> > > > > > > > > > > >> > Magesh
> > > > > > > > > > > >> >
> > > > > > > > > > > >> > On Tue, Aug 28, 2018 at 4:56 PM Randall Hauch <
> > > > > > > rha...@gmail.com
> > > > > > > > >
> > > > > > > > > > > wrote:
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >> Do you think we should support name-value pairs,
> > too?
> > > > > > > > > > > >> >>
> > > > > > > > > > > >> >> On Tue, Aug 28, 2018 at 6:41 PM Magesh
> Nandakumar <
> > > > > > > > > > > >> mage...@confluent.io>
> > > > > > > > > > > >> >> wrote:
> > > > > > > > > > > >> >>
> > > > > > > > > > > >> >> > Randall,
> > > > > > > > > > > >> >> >
> > > > > > > > > > > >> >> > Thanks a lot for the KIP. I think this would
> be a
> > > > great
> > > > > > > > > addition
> > > > > > > > > > > for
> > > > > > > > > > > >> >> many
> > > > > > > > > > > >> >> > source connectors.
> > > > > > > > > > > >> >> > One clarification I had was regarding the topic
> > > > > settings
> > > > > > > that
> > > > > > > > > can
> > > > > > > > > > > be
> > > > > > > > > > > >> >> > configured. Is it limited to the setting
> exposed
> > in
> > > > the
> > > > > > > > > > > TopicSettings
> > > > > > > > > > > >> >> > interface?
> > > > > > > > > > > >> >> >
> > > > > > > > > > > >> >> > Thanks
> > > > > > > > > > > >> >> > Magesh
> > > > > > > > > > > >> >> >
> > > > > > > > > > > >> >> > On Tue, Aug 21, 2018 at 7:59 PM Randall Hauch <
> > > > > > > > > rha...@gmail.com>
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >> >> >
> > > > > > > > > > > >> >> > > Okay, after much delay let's try this again
> for
> > > AK
> > > > > 2.1.
> > > > > > > Has
> > > > > > > > > > > anyone
> > > > > > > > > > > >> >> found
> > > > > > > > > > > >> >> > > any concerns? Stephane suggested that we
> allow
> > > > > updating
> > > > > > > > topic
> > > > > > > > > > > >> >> > > configurations (everything but partition
> > count).
> > > > I'm
> > > > > > > > > > unconvinced
> > > > > > > > > > > >> that
> > > > > > > > > > > >> >> > it's
> > > > > > > > > > > >> >> > > worth the additional complexity in the
> > > > implementation
> > > > > > and
> > > > > > > > the
> > > > > > > > > > > >> >> > documentation
> > > > > > > > > > > >> >> > > to explain the behavior. Changing several of
> > the
> > > > > > > > > topic-specific
> > > > > > > > > > > >> >> > > configurations have significant impact on
> > broker
> > > > > > > behavior /
> > > > > > > > > > > >> >> > functionality,
> > > > > > > > > > > >> >> > > so IMO we need to proceed more cautiously.
> > > > > > > > > > > >> >> > >
> > > > > > > > > > > >> >> > > Stephane, do you have a particular use case
> in
> > > mind
> > > > > for
> > > > > > > > > > updating
> > > > > > > > > > > >> topic
> > > > > > > > > > > >> >> > > configurations on an existing topic?
> > > > > > > > > > > >> >> > >
> > > > > > > > > > > >> >> > > Randall
> > > > > > > > > > > >> >> > >
> > > > > > > > > > > >> >> > >
> > > > > > > > > > > >> >> > > On Fri, Jan 26, 2018 at 4:20 PM Randall
> Hauch <
> > > > > > > > > > rha...@gmail.com>
> > > > > > > > > > > >> >> wrote:
> > > > > > > > > > > >> >> > >
> > > > > > > > > > > >> >> > > > The KIP deadline for 1.1 has already
> passed,
> > > but
> > > > > I'd
> > > > > > > like
> > > > > > > > > to
> > > > > > > > > > > >> restart
> > > > > > > > > > > >> >> > this
> > > > > > > > > > > >> >> > > > discussion so that we make the next
> release.
> > > I've
> > > > > not
> > > > > > > yet
> > > > > > > > > > > >> addressed
> > > > > > > > > > > >> >> the
> > > > > > > > > > > >> >> > > > previous comment about *existing* topics,
> but
> > > > I'll
> > > > > > try
> > > > > > > to
> > > > > > > > > do
> > > > > > > > > > > that
> > > > > > > > > > > >> >> over
> > > > > > > > > > > >> >> > > the
> > > > > > > > > > > >> >> > > > next few weeks. Any other
> > > > > > > comments/suggestions/questions?
> > > > > > > > > > > >> >> > > >
> > > > > > > > > > > >> >> > > > Best regards,
> > > > > > > > > > > >> >> > > >
> > > > > > > > > > > >> >> > > > Randall
> > > > > > > > > > > >> >> > > >
> > > > > > > > > > > >> >> > > > On Thu, Oct 5, 2017 at 12:13 AM, Randall
> > Hauch
> > > <
> > > > > > > > > > > rha...@gmail.com
> > > > > > > > > > > >> >
> > > > > > > > > > > >> >> > wrote:
> > > > > > > > > > > >> >> > > >
> > > > > > > > > > > >> >> > > >> Oops. Yes, I meant “replication factor”.
> > > > > > > > > > > >> >> > > >>
> > > > > > > > > > > >> >> > > >> > On Oct 4, 2017, at 7:18 PM, Ted Yu <
> > > > > > > > yuzhih...@gmail.com
> > > > > > > > > >
> > > > > > > > > > > >> wrote:
> > > > > > > > > > > >> >> > > >> >
> > > > > > > > > > > >> >> > > >> > Randall:
> > > > > > > > > > > >> >> > > >> > bq. AdminClient currently allows
> changing
> > > the
> > > > > > > > > replication
> > > > > > > > > > > >> >> factory.
> > > > > > > > > > > >> >> > > >> >
> > > > > > > > > > > >> >> > > >> > By 'replication factory' did you mean
> > > > > 'replication
> > > > > > > > > > factor' ?
> > > > > > > > > > > >> >> > > >> >
> > > > > > > > > > > >> >> > > >> > Cheers
> > > > > > > > > > > >> >> > > >> >
> > > > > > > > > > > >> >> > > >> >> On Wed, Oct 4, 2017 at 9:58 AM, Randall
> > > > Hauch <
> > > > > > > > > > > >> rha...@gmail.com
> > > > > > > > > > > >> >> >
> > > > > > > > > > > >> >> > > >> wrote:
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >> >> Currently the KIP's scope is only
> topics
> > > that
> > > > > > don't
> > > > > > > > yet
> > > > > > > > > > > >> exist,
> > > > > > > > > > > >> >> and
> > > > > > > > > > > >> >> > we
> > > > > > > > > > > >> >> > > >> have
> > > > > > > > > > > >> >> > > >> >> to cognizant of race conditions between
> > > tasks
> > > > > > with
> > > > > > > > the
> > > > > > > > > > same
> > > > > > > > > > > >> >> > > connector.
> > > > > > > > > > > >> >> > > >> I
> > > > > > > > > > > >> >> > > >> >> think it is worthwhile to consider
> > whether
> > > > the
> > > > > > > KIP's
> > > > > > > > > > scope
> > > > > > > > > > > >> >> should
> > > > > > > > > > > >> >> > > >> expand to
> > > > > > > > > > > >> >> > > >> >> also address *existing* partitions,
> > though
> > > it
> > > > > may
> > > > > > > not
> > > > > > > > > be
> > > > > > > > > > > >> >> > appropriate
> > > > > > > > > > > >> >> > > to
> > > > > > > > > > > >> >> > > >> >> have as much control when changing the
> > > topic
> > > > > > > settings
> > > > > > > > > for
> > > > > > > > > > > an
> > > > > > > > > > > >> >> > existing
> > > > > > > > > > > >> >> > > >> >> topic. For example, changing the number
> > of
> > > > > > > partitions
> > > > > > > > > > > (which
> > > > > > > > > > > >> the
> > > > > > > > > > > >> >> > KIP
> > > > > > > > > > > >> >> > > >> >> considers a "topic-specific setting"
> even
> > > > > though
> > > > > > > > > > > technically
> > > > > > > > > > > >> it
> > > > > > > > > > > >> >> is
> > > > > > > > > > > >> >> > > not)
> > > > > > > > > > > >> >> > > >> >> shouldn't be done blindly due to the
> > > > > partitioning
> > > > > > > > > > impacts,
> > > > > > > > > > > >> and
> > > > > > > > > > > >> >> IIRC
> > > > > > > > > > > >> >> > > you
> > > > > > > > > > > >> >> > > >> >> can't reduce them (which we could
> verify
> > > > before
> > > > > > > > > > applying).
> > > > > > > > > > > >> >> Also, I
> > > > > > > > > > > >> >> > > >> don't
> > > > > > > > > > > >> >> > > >> >> think the AdminClient currently allows
> > > > changing
> > > > > > the
> > > > > > > > > > > >> replication
> > > > > > > > > > > >> >> > > >> factory. I
> > > > > > > > > > > >> >> > > >> >> think changing the topic configs is
> less
> > > > > > > problematic
> > > > > > > > > both
> > > > > > > > > > > >> from
> > > > > > > > > > > >> >> what
> > > > > > > > > > > >> >> > > >> makes
> > > > > > > > > > > >> >> > > >> >> sense for connectors to verify/change
> and
> > > > from
> > > > > > what
> > > > > > > > the
> > > > > > > > > > > >> >> AdminClient
> > > > > > > > > > > >> >> > > >> >> supports.
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >> >> Even if we decide that it's not
> > appropriate
> > > > to
> > > > > > > change
> > > > > > > > > the
> > > > > > > > > > > >> >> settings
> > > > > > > > > > > >> >> > on
> > > > > > > > > > > >> >> > > >> an
> > > > > > > > > > > >> >> > > >> >> existing topic, I do think it's
> > > advantageous
> > > > to
> > > > > > at
> > > > > > > > > least
> > > > > > > > > > > >> notify
> > > > > > > > > > > >> >> the
> > > > > > > > > > > >> >> > > >> >> connector (or task) prior to the first
> > > record
> > > > > > sent
> > > > > > > > to a
> > > > > > > > > > > given
> > > > > > > > > > > >> >> topic
> > > > > > > > > > > >> >> > > so
> > > > > > > > > > > >> >> > > >> that
> > > > > > > > > > > >> >> > > >> >> the connector can fail or issue a
> warning
> > > if
> > > > it
> > > > > > > > doesn't
> > > > > > > > > > > meet
> > > > > > > > > > > >> its
> > > > > > > > > > > >> >> > > >> >> requirements.
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >> >> Best regards,
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >> >> Randall
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >> >> On Wed, Oct 4, 2017 at 12:52 AM,
> Stephane
> > > > > Maarek
> > > > > > <
> > > > > > > > > > > >> >> > > >> >> steph...@simplemachines.com.au> wrote:
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >> >>> Hi Randall,
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>> Thanks for the KIP. I like it
> > > > > > > > > > > >> >> > > >> >>> What happens when the target topic is
> > > > already
> > > > > > > > created
> > > > > > > > > > but
> > > > > > > > > > > >> the
> > > > > > > > > > > >> >> > > configs
> > > > > > > > > > > >> >> > > >> do
> > > > > > > > > > > >> >> > > >> >>> not match?
> > > > > > > > > > > >> >> > > >> >>> i.e. wrong RF, num partitions, or
> > missing
> > > /
> > > > > > > > additional
> > > > > > > > > > > >> configs?
> > > > > > > > > > > >> >> > Will
> > > > > > > > > > > >> >> > > >> you
> > > > > > > > > > > >> >> > > >> >>> attempt to apply the necessary changes
> > or
> > > > > throw
> > > > > > an
> > > > > > > > > > error?
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>> Thanks!
> > > > > > > > > > > >> >> > > >> >>> Stephane
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>> On 24/5/17, 5:59 am, "Mathieu
> Fenniak"
> > <
> > > > > > > > > > > >> >> > > mathieu.fenn...@replicon.com
> > > > > > > > > > > >> >> > > >> >
> > > > > > > > > > > >> >> > > >> >>> wrote:
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>    Ah, yes, I see you a highlighted
> part
> > > > that
> > > > > > > > > should've
> > > > > > > > > > > made
> > > > > > > > > > > >> >> this
> > > > > > > > > > > >> >> > > >> clear
> > > > > > > > > > > >> >> > > >> >>>    to me the first read. :-)  Much
> > clearer
> > > > > now!
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>    By the way, enjoyed your Debezium
> > talk
> > > in
> > > > > > NYC.
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>    Looking forward to this Kafka
> Connect
> > > > > change;
> > > > > > > it
> > > > > > > > > will
> > > > > > > > > > > >> allow
> > > > > > > > > > > >> >> me
> > > > > > > > > > > >> >> > to
> > > > > > > > > > > >> >> > > >> >>>    remove a post-deployment tool that
> I
> > > > hacked
> > > > > > > > > together
> > > > > > > > > > > for
> > > > > > > > > > > >> the
> > > > > > > > > > > >> >> > > >> purpose
> > > > > > > > > > > >> >> > > >> >>>    of ensuring auto-created topics
> have
> > > the
> > > > > > right
> > > > > > > > > > config.
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>    Mathieu
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>    On Tue, May 23, 2017 at 11:38 AM,
> > > Randall
> > > > > > > Hauch <
> > > > > > > > > > > >> >> > > rha...@gmail.com>
> > > > > > > > > > > >> >> > > >> >>> wrote:
> > > > > > > > > > > >> >> > > >> >>>> Thanks for the quick feedback,
> Mathieu.
> > > > Yes,
> > > > > > the
> > > > > > > > > first
> > > > > > > > > > > >> >> > > >> >> configuration
> > > > > > > > > > > >> >> > > >> >>> rule
> > > > > > > > > > > >> >> > > >> >>>> whose regex matches will be applied,
> > and
> > > no
> > > > > > other
> > > > > > > > > rules
> > > > > > > > > > > >> will
> > > > > > > > > > > >> >> be
> > > > > > > > > > > >> >> > > >> >>> used. I've
> > > > > > > > > > > >> >> > > >> >>>> updated the KIP to try to make this
> > more
> > > > > clear,
> > > > > > > but
> > > > > > > > > let
> > > > > > > > > > > me
> > > > > > > > > > > >> >> know
> > > > > > > > > > > >> >> > if
> > > > > > > > > > > >> >> > > >> >>> it's
> > > > > > > > > > > >> >> > > >> >>>> still not clear.
> > > > > > > > > > > >> >> > > >> >>>>
> > > > > > > > > > > >> >> > > >> >>>> Best regards,
> > > > > > > > > > > >> >> > > >> >>>>
> > > > > > > > > > > >> >> > > >> >>>> Randall
> > > > > > > > > > > >> >> > > >> >>>>
> > > > > > > > > > > >> >> > > >> >>>> On Tue, May 23, 2017 at 10:07 AM,
> > Mathieu
> > > > > > > Fenniak <
> > > > > > > > > > > >> >> > > >> >>>> mathieu.fenn...@replicon.com> wrote:
> > > > > > > > > > > >> >> > > >> >>>>
> > > > > > > > > > > >> >> > > >> >>>>> Hi Randall,
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>>> Awesome, very much looking forward
> to
> > > > this.
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>>> It isn't 100% clear from the KIP how
> > > > > multiple
> > > > > > > > > > > config-based
> > > > > > > > > > > >> >> rules
> > > > > > > > > > > >> >> > > >> >>> would
> > > > > > > > > > > >> >> > > >> >>>>> be applied; it looks like the first
> > > > > > > configuration
> > > > > > > > > rule
> > > > > > > > > > > >> whose
> > > > > > > > > > > >> >> > regex
> > > > > > > > > > > >> >> > > >> >>>>> matches the topic name will be used,
> > and
> > > > no
> > > > > > > other
> > > > > > > > > > rules
> > > > > > > > > > > >> will
> > > > > > > > > > > >> >> be
> > > > > > > > > > > >> >> > > >> >>>>> applied.  Is that correct?  (I
> wasn't
> > > sure
> > > > > if
> > > > > > it
> > > > > > > > > might
> > > > > > > > > > > >> >> cascade
> > > > > > > > > > > >> >> > > >> >>>>> together multiple matching rules...)
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>>> Looks great,
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>>> Mathieu
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>>> On Mon, May 22, 2017 at 1:43 PM,
> > Randall
> > > > > > Hauch <
> > > > > > > > > > > >> >> > rha...@gmail.com>
> > > > > > > > > > > >> >> > > >> >>> wrote:
> > > > > > > > > > > >> >> > > >> >>>>>> Hi, all.
> > > > > > > > > > > >> >> > > >> >>>>>>
> > > > > > > > > > > >> >> > > >> >>>>>> We recently added the ability for
> > Kafka
> > > > > > Connect
> > > > > > > > to
> > > > > > > > > > > create
> > > > > > > > > > > >> >> > > >> >>> *internal*
> > > > > > > > > > > >> >> > > >> >>>>> topics
> > > > > > > > > > > >> >> > > >> >>>>>> using the new AdminClient, but it
> > still
> > > > > would
> > > > > > > be
> > > > > > > > > > great
> > > > > > > > > > > if
> > > > > > > > > > > >> >> Kafka
> > > > > > > > > > > >> >> > > >> >>> Connect
> > > > > > > > > > > >> >> > > >> >>>>>> could do this for new topics that
> > > result
> > > > > from
> > > > > > > > > source
> > > > > > > > > > > >> >> connector
> > > > > > > > > > > >> >> > > >> >>> records.
> > > > > > > > > > > >> >> > > >> >>>>>> I've outlined an approach to do
> this
> > in
> > > > > > > "KIP-158
> > > > > > > > > > Kafka
> > > > > > > > > > > >> >> Connect
> > > > > > > > > > > >> >> > > >> >>> should
> > > > > > > > > > > >> >> > > >> >>>>> allow
> > > > > > > > > > > >> >> > > >> >>>>>> source connectors to set
> > topic-specific
> > > > > > > settings
> > > > > > > > > for
> > > > > > > > > > > new
> > > > > > > > > > > >> >> > > >> >> topics".
> > > > > > > > > > > >> >> > > >> >>>>>>
> > > > > > > > > > > >> >> > > >> >>>>>> *
> > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > > > > > >> >> > > >> >>>>> 158%3A+Kafka+Connect+should+
> > > > > > > > > > allow+source+connectors+to+
> > > > > > > > > > > >> >> > > >> >>>>>
> > > set+topic-specific+settings+for+new+topics
> > > > > > > > > > > >> >> > > >> >>>>>> <
> > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > > > > > > > > >> >> > > >> >>>>> 158%3A+Kafka+Connect+should+
> > > > > > > > > > allow+source+connectors+to+
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > set+topic-specific+settings+for+new+topics>*
> > > > > > > > > > > >> >> > > >> >>>>>>
> > > > > > > > > > > >> >> > > >> >>>>>> Please take a look and provide
> > > feedback.
> > > > > > > Thanks!
> > > > > > > > > > > >> >> > > >> >>>>>>
> > > > > > > > > > > >> >> > > >> >>>>>> Best regards,
> > > > > > > > > > > >> >> > > >> >>>>>>
> > > > > > > > > > > >> >> > > >> >>>>>> Randall
> > > > > > > > > > > >> >> > > >> >>>>>
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>>
> > > > > > > > > > > >> >> > > >> >>
> > > > > > > > > > > >> >> > > >>
> > > > > > > > > > > >> >> > > >
> > > > > > > > > > > >> >> > > >
> > > > > > > > > > > >> >> > >
> > > > > > > > > > > >> >> >
> > > > > > > > > > > >> >>
> > > > > > > > > > > >> >
> > > > > > > > > > > >>
> > > > > > > > > > > >
> > > > > > > > > > >
> > > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > --
> > > > > > > > > *Gwen Shapira*
> > > > > > > > > Product Manager | Confluent
> > > > > > > > > 650.450.2760 <(650)%20450-2760> | @gwenshap
> > > > > > > > > Follow us: Twitter <https://twitter.com/ConfluentInc> |
> blog
> > > > > > > > > <http://www.confluent.io/blog>
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to