Bosco,

Currently, if acl is enabled, auto topic creation will succeed if the
client has the CREATE permission.

Thanks,

Jun

On Mon, Jul 11, 2016 at 6:26 PM, Don Bosco Durai <bo...@apache.org> wrote:

> Jun, my understanding is, currently if ACLs are enabled, then auto topic
> creation is disabled. Is this going to change with this requirement?
>
> Thanks
>
> Bosco
>
>
> On 7/11/16, 1:14 PM, "Jun Rao" <j...@confluent.io> wrote:
>
> Ismael,
>
> We could change the existing behavior if it's bad for most of the users. In
> the case of auto topic creation in the producer, it seems that it's at
> least convenient in a testing environment. So, I am not sure if that
> behavior is universally bad.
>
> Also, I am not sure if we can rely on the client to set the configuration
> properly to disable auto topic creation. It seems that a safer way is to do
> that on the broker side (e.g., only allow the admin to create topics
> through ACL). Once you do that, I am not sure if we need a configuration to
> enable topic creation in the producer. A producer will just error out if
> the topic doesn't exist and the topic creation is disabled on the broker.
>
> Thanks,
>
> Jun
>
> On Fri, Jul 8, 2016 at 6:06 AM, Ismael Juma <ism...@juma.me.uk> wrote:
>
> > Hi Jun,
> >
> > I agree that it's closer to the existing behaviour, which some people may
> > be used to by now. However, I am not sure that it won't surprise people.
> As
> > Grant said, auto-topic creation is a common source of confusion and it
> > interacts badly with topic deletion.
> >
> > If we need to provide auto-topic creation in the client as a migration
> path
> > for people who rely on it and so that we can remove the server based one
> > (after a suitable deprecation period), then can we at least have it false
> > by default? This way it's more likely that people who enable it would be
> > aware of the pitfalls and it would reduce the number of confused users.
> >
> > Ismael
> >
> > On Thu, Jul 7, 2016 at 9:47 PM, Jun Rao <j...@confluent.io> wrote:
> >
> > > It seems that it makes sense for the writer to trigger auto topic
> > creation,
> > > but not the reader. So, my preference is Jay's option #1: add a new
> > > configuration to enable topic creation on the producer side and
> defaults
> > to
> > > true. If the topic doesn't exist, the producer will send a
> > > createTopicRequest and pick up the broker side defaults for #partitions
> > and
> > > replication factor. This matches the current behavior and won't
> surprise
> > > people. People who want to enforce manual topic creation can disable
> auto
> > > topic creation on the producer.
> > >
> > > On the consumer side, throwing an exception to the client when a topic
> > > doesn't exist probably works for most cases. I am wondering if there
> is a
> > > case where a user really wants to start the consumer before the topic
> is
> > > created.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Fri, Jul 1, 2016 at 4:09 AM, Ismael Juma <ism...@juma.me.uk> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I think there are a few things being discussed and it would be good
> to
> > > make
> > > > that explicit:
> > > >
> > > > 1. If and how we expose auto-topic creation in the client (under the
> > > > assumption that the server auto-topic creation will be deprecated and
> > > > eventually removed)
> > > > 2. The ability to create topics with the cluster defaults for
> > replication
> > > > factor and partition counts
> > > > 3. Support for topic "specs"
> > > > 4. The fact that some exceptions are retriable in some cases, but not
> > > > others
> > > >
> > > > My thoughts on each:
> > > >
> > > > 1. I prefer the approach where we throw an exception and let the
> > clients
> > > > create the topic via `AdminClient` if that's what they need.
> > > > 2. Like Grant, I'm unsure that will generally be used in a positive
> > way.
> > > > However, if this is what we need to be able to deprecate server
> > > auto-topic
> > > > creation, the benefits outweigh the costs in my opinion.
> > > > 3. Something like this would be good to have and could potentially
> > > provide
> > > > a better solution than 2. However, it needs a separate KIP and may
> > take a
> > > > while for the final design to be agreed. So, it should not prevent
> > > progress
> > > > from being made in my opinion.
> > > > 4. This has come up before. Encoding whether an exception is
> retriable
> > or
> > > > not via inheritance is a bit restrictive. Also, something that should
> > be
> > > > discussed separately, probably.
> > > >
> > > > Ismael
> > > >
> > > > On Wed, Jun 29, 2016 at 10:37 PM, Grant Henke <ghe...@cloudera.com>
> > > wrote:
> > > >
> > > > > Hi Roger and Constantine,
> > > > >
> > > > > Thanks for the feedback.
> > > > >
> > > > > I agree that configuration to maintain guarantees is commonly
> spread
> > > > across
> > > > > enterprise teams, making it difficult to get right. That said its
> > also
> > > > hard
> > > > > to solve for every company structure too. I think there is room for
> > an
> > > > open
> > > > > discussion about what configs should be able to be
> > > > > validated/enforced/overridden and where configurations should
> live. I
> > > > think
> > > > > thats big enough for a whole new KIP and would like to push that
> > > > discussion
> > > > > out until that KIP is opened. These discussions would also make
> sense
> > > in
> > > > > KIP-37
> > > > > - Add Namespaces to Kafka
> > > > > <
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-37+-+Add+Namespaces+to+Kafka
> > > > > >.
> > > > > To ensure we allow validation and overrides at the namespace level.
> > > > >
> > > > > That said, KIP-4 will be introducing a config request/response
> > protocol
> > > > >  and adding call to get/alter configs to the admin api. You could
> > > > leverage
> > > > > that to do some of the client validation and defaulting based on
> your
> > > > > needs. Look for a discussion thread from me on that soon.
> > > > >
> > > > > As far as auto topic creation goes, it sounds like failing fast and
> > > > > allowing the client application to create the topic would provide
> the
> > > > most
> > > > > flexibility to ensure the topic matches its needed specifications.
> > > > >
> > > > > Thanks,
> > > > > Grant
> > > > >
> > > > > On Wed, Jun 29, 2016 at 3:02 PM, Konstantin Zadorozhny <
> > > > > konstantin.zadoroz...@tubemogul.com> wrote:
> > > > >
> > > > > > Roger,
> > > > > >
> > > > > > I concur with everything you said.
> > > > > >
> > > > > > Couple more use cases to prove the point:
> > > > > >
> > > > > >    1. Some topics should always have 1 and only one partition.
> > > > > >    2. CDC application based on Kafka Connect. Those type of
> > > application
> > > > > >    absolutely must know how to create properly configured topics:
> > > > > > compacted, 1
> > > > > >    partition, replication factor 3, 2 min in sync replicas. In
> many
> > > > cases
> > > > > > per
> > > > > >    table or per database configuration overrides will be useful
> > too.
> > > > > >
> > > > > > If producer and consumer are able to verify topic configuration
> on
> > > > > startup
> > > > > > would be really useful. A spec would be great way to document the
> > > > intent
> > > > > of
> > > > > > the code. A lot of silly (but quite hard to pin down) production
> > > issues
> > > > > > could have been prevented by having producer to fail fast on
> > > > > misconfigured
> > > > > > topics.
> > > > > >
> > > > > > To add to the auto-creation configuration tally. We do have topic
> > > > > > auto-creation disabled on all our clusters.
> > > > > >
> > > > > > *Konstantin Zadorozhny*
> > > > > > www.tubemogul.com
> > > > > >
> > > > > > On Wed, Jun 29, 2016 at 11:17 AM, Roger Hoover <
> > > roger.hoo...@gmail.com
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > My comments go a bit beyond just topic creation but I'd like to
> > see
> > > > > Kafka
> > > > > > > make it easier for application developers to specify their
> > > > requirements
> > > > > > > declaratively in a single place.  Today, for example, if your
> > > > > application
> > > > > > > requires strong guarantees against data loss, you must set a
> mix
> > of
> > > > > > > topic-level configs (replication factor, min.in.sync.replicas,
> > > > > > > retention.ms)
> > > > > > > and client configs (acks=all and
> > > > > > > possibly max.in.flight.requests.per.connection if you care
> about
> > > > > > > ordering).  This can be complicated by organizational structure
> > > where
> > > > > you
> > > > > > > have a different team (SREs) responsible for the cluster
> configs
> > > and
> > > > > > > perhaps topic creation and application teams responsible for
> the
> > > > client
> > > > > > > settings.  Let's say that you get all the settings right up
> > front.
> > > > How
> > > > > > > would you know if they later were changed incorrectly?  How do
> > > admins
> > > > > > know
> > > > > > > which topics are ok to add more partitions are which are not?
> > How
> > > do
> > > > > > > downstream applications know how much retention they can rely
> on
> > > for
> > > > > > > re-processing in their upstream topics.
> > > > > > >
> > > > > > > I think it's useful to consider the typical roles in an
> > > organization.
> > > > > > Say
> > > > > > > we have an SRE team responsible for overall cluster health,
> > > capacity,
> > > > > > etc.
> > > > > > > This team likely has elevated privileges and perhaps wants to
> > > > > > > review/approve settings for new topics to make sure they're
> sane.
> > > > > > >
> > > > > > > The application developer may not care about some of the
> details
> > of
> > > > > topic
> > > > > > > creation but does care in as much as they affect the
> application
> > > > > > > correctness and SLAs.  It's more than just number of partitions
> > and
> > > > > > > replication factor.  The application may require
> > > > > > > 1) some of it's topics to be compacted to function correctly
> and
> > > > > > > min.compaction.lag.ms (KIP-58) set correctly
> > > > > > > 2) retention.ms set correctly on some of it's topics to
> satisfy
> > > it's
> > > > > > > failure/re-processing SLAs
> > > > > > > 3) partitioning of it's input topics to match it's expectations
> > > > > > > 4) the data format to match expectations
> > > > > > >
> > > > > > > I realize that #3 and #4 are unrelated to topic creation but
> > > they're
> > > > > part
> > > > > > > of a set of invariants that the application needs enforced and
> > > should
> > > > > > fail
> > > > > > > early if their requirements are not met.  For example, with
> > > > > semantically
> > > > > > > partitioned topics, the application may break if new partitions
> > are
> > > > > > added.
> > > > > > > The issue is that there is no standard mechanism or convention
> to
> > > > > > > communicate application requirements so that admins and
> > application
> > > > > teams
> > > > > > > can verify that they continue to be met over time.
> > > > > > >
> > > > > > > Imagine for a second that Kafka allowed arbitrary tags to be
> > > > associated
> > > > > > to
> > > > > > > topics.  An application could now define a specification for
> it's
> > > > > > > interaction with Kafka including topic names, min replication
> > > > factors,
> > > > > > > fault tolerance settings (replication factors,
> > > min.in.sync.replicas,
> > > > > > > producer acks), compacted yes/no, topic retention settings, can
> > > > > > add/remove
> > > > > > > partitions, partition key, and data format.  Some of these
> > > > requirements
> > > > > > map
> > > > > > > onto topics configs and some (like acks=all) are producer
> > settings
> > > > and
> > > > > > some
> > > > > > > (like partition key and data format) could be organizational
> > > > > conventions
> > > > > > > stored as tags (format:avro).
> > > > > > >
> > > > > > > For organizations where only SREs/admins can create/modify
> > topics,
> > > > this
> > > > > > > spec allows them to do their job while being sure they're not
> > > > breaking
> > > > > > the
> > > > > > > application.  The application can verify on startup that it's
> > > > > > requirements
> > > > > > > are satisfied and fail early if not.  If the application has
> > > > > permissions
> > > > > > to
> > > > > > > create it's own topics then the spec is a declarative format
> for
> > > > doing
> > > > > > that
> > > > > > > require and will not require the same topic creation
> boilerplate
> > > code
> > > > > to
> > > > > > be
> > > > > > > duplicated in every application.
> > > > > > >
> > > > > > > If people like this approach, perhaps we could define a topic
> > spec
> > > > (if
> > > > > > all
> > > > > > > fields besides topic name are empty it use "cluster defaults").
> > > Then
> > > > > the
> > > > > > > AdminClient would have an idempotent create method that takes a
> > > spec
> > > > > and
> > > > > > > verifies that the spec is already met, tries to create topics
> to
> > > meet
> > > > > the
> > > > > > > spec, or fails saying it cannot be met.  Perhaps the producer
> and
> > > > > > consumer
> > > > > > > APIs would only have a verify() method which checks if the spec
> > is
> > > > > > > satisfied.
> > > > > > >
> > > > > > > Cheers,
> > > > > > >
> > > > > > > Roger
> > > > > > >
> > > > > > > On Wed, Jun 29, 2016 at 8:50 AM, Grant Henke <
> > ghe...@cloudera.com>
> > > > > > wrote:
> > > > > > >
> > > > > > > > Thanks for the discussion, below are some thoughts and
> > responses.
> > > > > > > >
> > > > > > > > One of the problems that we currently have with
> > > > > > > > > the clients is that we retry silently on unknown topics
> under
> > > the
> > > > > > > > > expectation that they will eventually be created
> > (automatically
> > > > or
> > > > > > > not).
> > > > > > > > > This makes it difficult to detect misconfiguration without
> > > > looking
> > > > > > for
> > > > > > > > > warnings in the logs. This problem is compounded if the
> > client
> > > > > isn't
> > > > > > > > > authorized to the topic since then we don't actually know
> if
> > > the
> > > > > > topic
> > > > > > > > > exists or not and whether it is reasonable to keep
> retrying.
> > > > > > > >
> > > > > > > >
> > > > > > > > Yeah this is a problem thats difficult and opaque to the
> user.
> > I
> > > > > think
> > > > > > > any
> > > > > > > > of the proposed solutions would help solve this issue. Since
> > the
> > > > > create
> > > > > > > > would be done at the metadata request phase, instead of in
> the
> > > > > produce
> > > > > > > > response handling. And if the create fails, the user would
> > > receive
> > > > a
> > > > > > > munch
> > > > > > > > more clear authorization error.
> > > > > > > >
> > > > > > > > The current auto creation of topic by the broker appear to be
> > the
> > > > > only
> > > > > > > > > reason an unknown topic error is retriable
> > > > > > > > > which leads to bugs (like
> > > > > > > > https://issues.apache.org/jira/browse/KAFKA-3727
> > > > > > > > > ) where the consumer hangs forever (or until woken up) and
> > only
> > > > > debug
> > > > > > > > > tracing shows what's going on.
> > > > > > > > >
> > > > > > > >
> > > > > > > > I agree this is related, but should be solvable even with
> > > retriable
> > > > > > > > exceptions. I think UnknownTopicOrPartitionException needs to
> > > > remain
> > > > > > > > generally retriable because it could occur due to outdated
> > > metadata
> > > > > and
> > > > > > > not
> > > > > > > > because a topic needs to be created. In the case of message
> > > > > production
> > > > > > or
> > > > > > > > consumption it could be explicitly handled differently in the
> > > > client.
> > > > > > > >
> > > > > > > > Do we clearly define the expected behavior of subscribe and
> > > assign
> > > > in
> > > > > > the
> > > > > > > > case of a missing topic? I can see reasons to fail early
> > > (partition
> > > > > > will
> > > > > > > > never exist, typo in topic name) and reasons to keep
> returning
> > > > empty
> > > > > > > record
> > > > > > > > sets until the topic exists (consumer with a preconfigured
> list
> > > of
> > > > > > topics
> > > > > > > > that may or may not exist). Though I think failing and
> > insisting
> > > > > topics
> > > > > > > > exist is the most predictable. Especially since the Admin API
> > > will
> > > > > make
> > > > > > > > creating topics easier.
> > > > > > > >
> > > > > > > > Usually in the pre-prod environments you don't really
> > > > > > > > > care about the settings at all, and in prod you can
> > > > pre-provision.
> > > > > > > >
> > > > > > > >
> > > > > > > > I like the recommendations, developer/ops experience and
> > required
> > > > > > > exercises
> > > > > > > > to be fairly consistent between dev, qa, and prod. If you
> need
> > to
> > > > > > > > pre-provision and think about the settings in prod. Its best
> to
> > > put
> > > > > > some
> > > > > > > > effort into building that logic in dev or qa too. Otherwise
> you
> > > get
> > > > > > ready
> > > > > > > > to deploy and everything changes and all your earlier testing
> > is
> > > > not
> > > > > as
> > > > > > > > relevant.
> > > > > > > >
> > > > > > > > For what it's worth the use case for auto-creation isn't
> using
> > a
> > > > > > dynamic
> > > > > > > > > set of topics, but rather letting apps flow through
> different
> > > > > > > > > dev/staging/prod/integration_testing/unit_testing
> > environments
> > > > > > without
> > > > > > > > > having the app configure appropriate
> replication/partitioning
> > > > stuff
> > > > > > in
> > > > > > > > each
> > > > > > > > > environment and having complex logic to check if the topic
> is
> > > > > there.
> > > > > > > > >
> > > > > > > >
> > > > > > > > The problem I have seen here is that the cluster default is
> > > global,
> > > > > at
> > > > > > > > least until we have some concept of namespaces and can
> > configure
> > > > > > defaults
> > > > > > > > for each. Since picking a good number of partitions varies
> > based
> > > on
> > > > > > > volume,
> > > > > > > > use case, etc a default that works for most topics is a hard
> to
> > > > find.
> > > > > > > >
> > > > > > > > I feel like because app developers think they don't need to
> > think
> > > > > about
> > > > > > > > topic creation, often they don't. And that leads to a mess
> > where
> > > > they
> > > > > > > don't
> > > > > > > > know how may partitions and what replication factor they
> have.
> > > > > Instead
> > > > > > > > migrating environments with a setup script that creates the
> > > needed
> > > > > > topics
> > > > > > > > allows them to source control those setting and create
> > > predictable,
> > > > > > > > repeatable deployments.
> > > > > > > >
> > > > > > > > I have also seen a lot of issues where users are confused
> about
> > > > why a
> > > > > > > topic
> > > > > > > > is coming back or can't be deleted. This is often a result
> > > > > > > > of auto.create.topics.enable being defaulted to true. And
> they
> > > > never
> > > > > > > expect
> > > > > > > > that a feature like that would exist, much less be the
> default.
> > > > > > > >
> > > > > > > > On a side note, the best dynamic use case I could think of is
> > > > > > > MirrorMaker.
> > > > > > > > But the cluster defaults here don't really work since its
> they
> > > are
> > > > > not
> > > > > > > very
> > > > > > > > flexible. Pushing creation to the client would allow tools
> like
> > > > > > > MirrorMaker
> > > > > > > > to create topics that match the upstream cluster, or provide
> > its
> > > > own
> > > > > > > logic
> > > > > > > > for sizing downstream topics.
> > > > > > > >
> > > > > > > > This raises an important point about how we handle defaults,
> > > which
> > > > I
> > > > > > > don't
> > > > > > > > > think we talked about. I do think it is really important
> that
> > > we
> > > > > > allow
> > > > > > > a
> > > > > > > > > way to create topics with the "cluster defaults". I know
> this
> > > is
> > > > > > > possible
> > > > > > > > > for configs since if you omit them they inherit default
> > values,
> > > > > but I
> > > > > > > > think
> > > > > > > > > we should be able to do it with replication factor and
> > > partition
> > > > > > count
> > > > > > > > too.
> > > > > > > > > I think the Java API should expose this and maybe even
> > > encourage
> > > > > it.
> > > > > > > > >
> > > > > > > >
> > > > > > > > We could make the create topic request num_partitions and
> > > > > > > > replication_factor fields optional and if unset use the
> cluster
> > > > > > defaults.
> > > > > > > > This allows a user to opt into the cluster defaults at create
> > > > time. I
> > > > > > > have
> > > > > > > > rarely seen good defaults set in my experience though,
> > especially
> > > > > since
> > > > > > > the
> > > > > > > > default is 1 in both cases.
> > > > > > > >
> > > > > > > > I kind of feel once you start adding AdminClient methods to
> the
> > > > > > producer
> > > > > > > > > and consumer it's not really clear where to stop--e.g. if I
> > can
> > > > > > create
> > > > > > > I
> > > > > > > > > should be able to delete, list, etc.
> > > > > > > >
> > > > > > > >
> > > > > > > > I agree this gets weird and could lead to duplicate client
> code
> > > and
> > > > > > > > inconsistent behavior across clients. The one thing I don't
> > like
> > > > > about
> > > > > > > > requiring a separate client is it maintains all its own
> > > connections
> > > > > and
> > > > > > > > metadata. Perhaps sometime down the road if we see a lot of
> > mixed
> > > > > usage
> > > > > > > we
> > > > > > > > could break out the core cluster connection code into a
> > > > > KafkaConnection
> > > > > > > > class and instantiate clients with that. That way clients
> could
> > > > share
> > > > > > the
> > > > > > > > same KafkaConnection.
> > > > > > > >
> > > > > > > > Thanks,
> > > > > > > > Grant
> > > > > > > >
> > > > > > > >
> > > > > > > > On Wed, Jun 29, 2016 at 9:29 AM, Jay Kreps <j...@confluent.io
> >
> > > > wrote:
> > > > > > > >
> > > > > > > > > For what it's worth the use case for auto-creation isn't
> > using
> > > a
> > > > > > > dynamic
> > > > > > > > > set of topics, but rather letting apps flow through
> different
> > > > > > > > > dev/staging/prod/integration_testing/unit_testing
> > environments
> > > > > > without
> > > > > > > > > having the app configure appropriate
> replication/partitioning
> > > > stuff
> > > > > > in
> > > > > > > > each
> > > > > > > > > environment and having complex logic to check if the topic
> is
> > > > > there.
> > > > > > > > > Basically if you leave this up to individual apps you get
> > kind
> > > > of a
> > > > > > > mess,
> > > > > > > > > it's better to have cluster defaults that are reasonable
> and
> > > > > > controlled
> > > > > > > > by
> > > > > > > > > an admin and then pre-provision anything that is weird
> (super
> > > > big,
> > > > > > > > unusual
> > > > > > > > > perms, whatever). Usually in the pre-prod environments you
> > > don't
> > > > > > really
> > > > > > > > > care about the settings at all, and in prod you can
> > > > pre-provision.
> > > > > > > > >
> > > > > > > > > This raises an important point about how we handle
> defaults,
> > > > which
> > > > > I
> > > > > > > > don't
> > > > > > > > > think we talked about. I do think it is really important
> that
> > > we
> > > > > > allow
> > > > > > > a
> > > > > > > > > way to create topics with the "cluster defaults". I know
> this
> > > is
> > > > > > > possible
> > > > > > > > > for configs since if you omit them they inherit default
> > values,
> > > > > but I
> > > > > > > > think
> > > > > > > > > we should be able to do it with replication factor and
> > > partition
> > > > > > count
> > > > > > > > too.
> > > > > > > > > I think the Java API should expose this and maybe even
> > > encourage
> > > > > it.
> > > > > > > > >
> > > > > > > > > I don't have a super strong opinion on how this is exposed,
> > > > though
> > > > > I
> > > > > > > kind
> > > > > > > > > of prefer one of two options:
> > > > > > > > > 1. Keep the approach we have now with a config option to
> > allow
> > > > auto
> > > > > > > > create,
> > > > > > > > > but using this option just gives you a plain vanilla topic
> > with
> > > > no
> > > > > > > custom
> > > > > > > > > configs, for anything custom you need to use AdminClient
> > > > "manually"
> > > > > > > > > 2. Just throw an exception and let you use AdminClient.
> This
> > > may
> > > > > be a
> > > > > > > bit
> > > > > > > > > of a transition for people relying on the current behavior.
> > > > > > > > >
> > > > > > > > > I kind of feel once you start adding AdminClient methods to
> > the
> > > > > > > producer
> > > > > > > > > and consumer it's not really clear where to stop--e.g. if I
> > can
> > > > > > create
> > > > > > > I
> > > > > > > > > should be able to delete, list, etc.
> > > > > > > > >
> > > > > > > > > -Jay
> > > > > > > > >
> > > > > > > > > On Tue, Jun 28, 2016 at 9:26 AM, Grant Henke <
> > > > ghe...@cloudera.com>
> > > > > > > > wrote:
> > > > > > > > >
> > > > > > > > > > With the KIP-4 create topic schema voted and passed and a
> > PR
> > > > > > > available
> > > > > > > > > > upstream. I wanted to discuss moving the auto topic
> > creation
> > > > from
> > > > > > the
> > > > > > > > > > broker side to the client side (KAFKA-2410
> > > > > > > > > > <https://issues.apache.org/jira/browse/KAFKA-2410>).
> > > > > > > > > >
> > > > > > > > > > This change has many benefits
> > > > > > > > > >
> > > > > > > > > >    - Remove the need for failed messages until a topic is
> > > > created
> > > > > > > > > >    - Client can define the auto create parameters instead
> > of
> > > a
> > > > > > global
> > > > > > > > > >    cluster setting
> > > > > > > > > >    - Errors can be communicated back to the client more
> > > clearly
> > > > > > > > > >
> > > > > > > > > > Overall auto create is not my favorite feature, since
> topic
> > > > > > creation
> > > > > > > > is a
> > > > > > > > > > highly critical piece for Kafka, and with authorization
> > added
> > > > it
> > > > > > > > becomes
> > > > > > > > > > even more involved. When creating a topic a user needs:
> > > > > > > > > >
> > > > > > > > > >    - The access to create topics
> > > > > > > > > >    - To set the correct partition count and replication
> > > factor
> > > > > for
> > > > > > > > their
> > > > > > > > > >    use case
> > > > > > > > > >    - To set who has access to the topic
> > > > > > > > > >    - Knowledge of how a new topic may impact regex
> > consumers
> > > or
> > > > > > > > > mirrormaker
> > > > > > > > > >
> > > > > > > > > > Often I find use cases that look like they need auto
> topic
> > > > > > creation,
> > > > > > > > can
> > > > > > > > > > often be handled with a few pre made topics. That said,
> we
> > > > still
> > > > > > > should
> > > > > > > > > > support the feature for the cases that need it
> > (mirrormaker,
> > > > > > > streams).
> > > > > > > > > >
> > > > > > > > > > The question is how we should expose auto topic creation
> in
> > > the
> > > > > > > > client. A
> > > > > > > > > > few options are:
> > > > > > > > > >
> > > > > > > > > >    - Add configs like the broker configs today, and let
> the
> > > > > client
> > > > > > > > > >    automatically create the topics if enabled
> > > > > > > > > >       - Both producer and consumer?
> > > > > > > > > >    - Throw an error to the user and let them use a
> separate
> > > > > > > AdminClient
> > > > > > > > > >    (KIP-4) api to create the topic
> > > > > > > > > >    - Throw an error to the user and add a create api to
> the
> > > > > > producer
> > > > > > > so
> > > > > > > > > >    they can easily handle by creating a topic
> > > > > > > > > >
> > > > > > > > > > I am leaning towards the last 2 options but wanted to get
> > > some
> > > > > > others
> > > > > > > > > > thoughts on the matter. Especially if you have use cases
> > that
> > > > use
> > > > > > > auto
> > > > > > > > > > topic creation today.
> > > > > > > > > >
> > > > > > > > > > Thanks,
> > > > > > > > > > Grant
> > > > > > > > > >
> > > > > > > > > > --
> > > > > > > > > > Grant Henke
> > > > > > > > > > Software Engineer | Cloudera
> > > > > > > > > > gr...@cloudera.com | twitter.com/gchenke |
> > > > > > > linkedin.com/in/granthenke
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > Grant Henke
> > > > > > > > Software Engineer | Cloudera
> > > > > > > > gr...@cloudera.com | twitter.com/gchenke |
> > > > > linkedin.com/in/granthenke
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Grant Henke
> > > > > Software Engineer | Cloudera
> > > > > gr...@cloudera.com | twitter.com/gchenke |
> > linkedin.com/in/granthenke
> > > > >
> > > >
> > >
> >
>
>
>
>

Reply via email to