You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Grant Henke <gh...@cloudera.com> on 2016/06/28 16:26:14 UTC

[DISCUSS] Client Side Auto Topic Creation

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
grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Guozhang Wang <wa...@gmail.com>.
I agree with Grant and Ismael regarding 1 / 2 / 3, and for 2) I would
prefer having the default configs on the client side (i.e. like what we did
in the kafka-topics script today) than on the cluster side, where the
AdminClient will auto-set the configs if they are not specified by the user
triggering its function, so that:

a) when you use the AdminClient in a command line tool / etc, you would
more likely to be aware of the default setting you are living with even
when you do not care.

b) when you use the AdminClient programmably in your client development,
you are more likely to think of the configs to choose with the exposed APIs.


Guozhang

On Fri, Jul 1, 2016 at 4:09 AM, Ismael Juma <is...@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 <gh...@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.zadorozhny@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.hoover@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 <gh...@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 <ja...@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 <
> ghenke@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
> > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > linkedin.com/in/granthenke
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Grant Henke
> > > > > Software Engineer | Cloudera
> > > > > grant@cloudera.com | twitter.com/gchenke |
> > linkedin.com/in/granthenke
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Grant Henke
> > Software Engineer | Cloudera
> > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> >
>



-- 
-- Guozhang

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Grant Henke <gh...@cloudera.com>.
Thanks for all the feedback guys.

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.


I agree that testing has a unique set of needs and tooling. Some things
that are critical at scale or in real environments can be automated and
ignored. However, those things are often critical in a real environment and
allowing users to ignore them is often a trap they don't know they are
walking into.

I have often though a public testing package for Kafka would be really
useful for third party developers and integrations to leverage. I think a
test package like that would be a great place for "unsafe but convenient"
functions like auto create.
Given all the feedback I should be able to write up a straw-man KIP.

Thank you,
Grant



On Mon, Jul 11, 2016 at 3:22 PM, Guozhang Wang <wa...@gmail.com> wrote:

> I'd prefer to have the auto-creation on producer to false by default.
>
> Also, I would suggest not have a broker-side default configs for created
> topics, but rather on the admin-client side, this way users may be more
> aware of the default config values.
>
>
> Guozhang
>
>
> On Fri, Jul 8, 2016 at 6:06 AM, Ismael Juma <is...@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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <
> > ghenke@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 <jay@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 <
> > > > ghenke@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
> > > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > > > linkedin.com/in/granthenke
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > Grant Henke
> > > > > > > > Software Engineer | Cloudera
> > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > linkedin.com/in/granthenke
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Grant Henke
> > > > > Software Engineer | Cloudera
> > > > > grant@cloudera.com | twitter.com/gchenke |
> > linkedin.com/in/granthenke
> > > > >
> > > >
> > >
> >
>
>
>
> --
> -- Guozhang
>



-- 
Grant Henke
Software Engineer | Cloudera
grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Guozhang Wang <wa...@gmail.com>.
I'd prefer to have the auto-creation on producer to false by default.

Also, I would suggest not have a broker-side default configs for created
topics, but rather on the admin-client side, this way users may be more
aware of the default config values.


Guozhang


On Fri, Jul 8, 2016 at 6:06 AM, Ismael Juma <is...@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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <
> ghenke@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 <ja...@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 <
> > > ghenke@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
> > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > > linkedin.com/in/granthenke
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > Grant Henke
> > > > > > > Software Engineer | Cloudera
> > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > linkedin.com/in/granthenke
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Grant Henke
> > > > Software Engineer | Cloudera
> > > > grant@cloudera.com | twitter.com/gchenke |
> linkedin.com/in/granthenke
> > > >
> > >
> >
>



-- 
-- Guozhang

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Don Bosco Durai <bo...@apache.org>.
Jun, thanks. I will check again.

Regards

Bosco


On 7/17/16, 9:36 PM, "Jun Rao" <ju...@confluent.io> wrote:

    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" <ju...@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 <is...@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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <
    > > ghenke@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 <jay@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 <
    > > > > ghenke@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
    > > > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
    > > > > > > > linkedin.com/in/granthenke
    > > > > > > > > > >
    > > > > > > > > >
    > > > > > > > >
    > > > > > > > >
    > > > > > > > >
    > > > > > > > > --
    > > > > > > > > Grant Henke
    > > > > > > > > Software Engineer | Cloudera
    > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
    > > > > > linkedin.com/in/granthenke
    > > > > > > > >
    > > > > > > >
    > > > > > >
    > > > > >
    > > > > >
    > > > > >
    > > > > > --
    > > > > > Grant Henke
    > > > > > Software Engineer | Cloudera
    > > > > > grant@cloudera.com | twitter.com/gchenke |
    > > linkedin.com/in/granthenke
    > > > > >
    > > > >
    > > >
    > >
    >
    >
    >
    >
    



Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Jun Rao <ju...@confluent.io>.
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" <ju...@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 <is...@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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <
> > ghenke@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 <jay@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 <
> > > > ghenke@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
> > > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > > > linkedin.com/in/granthenke
> > > > > > > > > >
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > --
> > > > > > > > Grant Henke
> > > > > > > > Software Engineer | Cloudera
> > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > linkedin.com/in/granthenke
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Grant Henke
> > > > > Software Engineer | Cloudera
> > > > > grant@cloudera.com | twitter.com/gchenke |
> > linkedin.com/in/granthenke
> > > > >
> > > >
> > >
> >
>
>
>
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Don Bosco Durai <bo...@apache.org>.
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" <ju...@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 <is...@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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <
> ghenke@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 <ja...@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 <
> > > ghenke@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
> > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > > linkedin.com/in/granthenke
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > Grant Henke
> > > > > > > Software Engineer | Cloudera
> > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > linkedin.com/in/granthenke
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Grant Henke
> > > > Software Engineer | Cloudera
> > > > grant@cloudera.com | twitter.com/gchenke |
> linkedin.com/in/granthenke
> > > >
> > >
> >
>




Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Jun Rao <ju...@confluent.io>.
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 <is...@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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <
> ghenke@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 <ja...@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 <
> > > ghenke@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
> > > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > > linkedin.com/in/granthenke
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > Grant Henke
> > > > > > > Software Engineer | Cloudera
> > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > linkedin.com/in/granthenke
> > > > > > >
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Grant Henke
> > > > Software Engineer | Cloudera
> > > > grant@cloudera.com | twitter.com/gchenke |
> linkedin.com/in/granthenke
> > > >
> > >
> >
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Ismael Juma <is...@juma.me.uk>.
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 <ju...@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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <gh...@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 <ja...@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 <
> > ghenke@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
> > > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > > linkedin.com/in/granthenke
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Grant Henke
> > > > > > Software Engineer | Cloudera
> > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > linkedin.com/in/granthenke
> > > > > >
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Grant Henke
> > > Software Engineer | Cloudera
> > > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> > >
> >
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Tommy Becker <to...@tivo.com>.
I think the use case for not blowing up the consumer is simply to not create an implicit ordering in which your services have to come up.

On 07/07/2016 04:47 PM, Jun Rao 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 <is...@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 <gh...@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.zadorozhny@tubemogul.com<ma...@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<http://www.tubemogul.com>

On Wed, Jun 29, 2016 at 11:17 AM, Roger Hoover <ro...@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 <gh...@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 <ja...@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 <


ghenke@cloudera.com<ma...@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><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
grant@cloudera.com<ma...@cloudera.com> | twitter.com/gchenke |


linkedin.com/in/granthenke











--
Grant Henke
Software Engineer | Cloudera
grant@cloudera.com<ma...@cloudera.com> | twitter.com/gchenke |


linkedin.com/in/granthenke














--
Grant Henke
Software Engineer | Cloudera
grant@cloudera.com<ma...@cloudera.com> | twitter.com/gchenke | linkedin.com/in/granthenke









--
Tommy Becker
Senior Software Engineer

Digitalsmiths
A TiVo Company

www.digitalsmiths.com<http://www.digitalsmiths.com>
tobecker@tivo.com<ma...@tivo.com>

________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Jun Rao <ju...@confluent.io>.
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 <is...@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 <gh...@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.zadorozhny@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.hoover@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 <gh...@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 <ja...@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 <
> ghenke@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
> > > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > > linkedin.com/in/granthenke
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > > Grant Henke
> > > > > Software Engineer | Cloudera
> > > > > grant@cloudera.com | twitter.com/gchenke |
> > linkedin.com/in/granthenke
> > > > >
> > > >
> > >
> >
> >
> >
> > --
> > Grant Henke
> > Software Engineer | Cloudera
> > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> >
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Ismael Juma <is...@juma.me.uk>.
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 <gh...@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.zadorozhny@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 <ro...@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 <gh...@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 <ja...@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 <gh...@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
> > > > > > grant@cloudera.com | twitter.com/gchenke |
> > > linkedin.com/in/granthenke
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Grant Henke
> > > > Software Engineer | Cloudera
> > > > grant@cloudera.com | twitter.com/gchenke |
> linkedin.com/in/granthenke
> > > >
> > >
> >
>
>
>
> --
> Grant Henke
> Software Engineer | Cloudera
> grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Grant Henke <gh...@cloudera.com>.
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.zadorozhny@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 <ro...@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 <gh...@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 <ja...@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 <gh...@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
> > > > > grant@cloudera.com | twitter.com/gchenke |
> > linkedin.com/in/granthenke
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > > Grant Henke
> > > Software Engineer | Cloudera
> > > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> > >
> >
>



-- 
Grant Henke
Software Engineer | Cloudera
grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Konstantin Zadorozhny <ko...@tubemogul.com>.
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 <ro...@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 <gh...@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 <ja...@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 <gh...@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
> > > > grant@cloudera.com | twitter.com/gchenke |
> linkedin.com/in/granthenke
> > > >
> > >
> >
> >
> >
> > --
> > Grant Henke
> > Software Engineer | Cloudera
> > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> >
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Tommy Becker <to...@tivo.com>.
We currently run with auto topic creation enabled, largely to ensure that our topics all get created with the cluster defaults. My understanding is that this is the only to ensure this, since the defaults are not accessible to clients. We run a cluster per deployment, with the defaults are set by our administrators; they are not guaranteed to be the same everywhere. One use case we have in particular is to create topics with log compaction enabled. Currently, doing this from the client requires the use of AdminUtils, which in turn requires that you specify pretty much the entire config. I would be very much in favor of having to use an AdminClient to explicitly create topics if it was possible to only override specific settings (e.g. enable log compaction). I think any solution that requires developers to "think harder" about things like partition count would need to be accompanied by some actual guidance on how to determine such things. Actually, such guidance would be nice regardless.

I think Roger's suggestion of having named topic configurations for specific use cases is a great one. Being able to make these decisions once and then have applications be able to simply create a topic for "max-redundancy" or "high-parallelism" would be nice.



On 06/29/2016 02:17 PM, Roger Hoover 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 <gh...@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 <ja...@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 <gh...@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><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
grant@cloudera.com<ma...@cloudera.com> | twitter.com/gchenke | linkedin.com/in/granthenke









--
Grant Henke
Software Engineer | Cloudera
grant@cloudera.com<ma...@cloudera.com> | twitter.com/gchenke | linkedin.com/in/granthenke






--
Tommy Becker
Senior Software Engineer

Digitalsmiths
A TiVo Company

www.digitalsmiths.com<http://www.digitalsmiths.com>
tobecker@tivo.com<ma...@tivo.com>

________________________________

This email and any attachments may contain confidential and privileged material for the sole use of the intended recipient. Any review, copying, or distribution of this email (or any attachments) by others is prohibited. If you are not the intended recipient, please contact the sender immediately and permanently delete this email and any attachments. No employee or agent of TiVo Inc. is authorized to conclude any binding agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo Inc. may only be made by a signed written agreement.

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Roger Hoover <ro...@gmail.com>.
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 <gh...@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 <ja...@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 <gh...@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
> > > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> > >
> >
>
>
>
> --
> Grant Henke
> Software Engineer | Cloudera
> grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Grant Henke <gh...@cloudera.com>.
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 <ja...@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 <gh...@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
> > grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> >
>



-- 
Grant Henke
Software Engineer | Cloudera
grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Jay Kreps <ja...@confluent.io>.
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 <gh...@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
> grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Edoardo Comar <EC...@uk.ibm.com>.
Thanks Jason, I'm 100% with you on this subject,

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.

As Jason, my favourite option is #2 - client receives an exception and may 
uses an admin client to try resolve.
This is a very clear flow control - an unknown topic becomes an error 
solvable only through client action when the client has the necessary 
authorizations.

Disclaimer - our deployments of Kafka as a service (IBM MessageHub) have 
auto creation on the broker disabled :-)

cheers,
Edo
--------------------------------------------------
Edoardo Comar
MQ Cloud Technologies
ecomar@uk.ibm.com
+44 (0)1962 81 5576 
IBM UK Ltd, Hursley Park, SO21 2JN

IBM United Kingdom Limited Registered in England and Wales with number 
741598 Registered office: PO Box 41, North Harbour, Portsmouth, Hants. PO6 
3AU



From:   Jason Gustafson <ja...@confluent.io>
To:     dev@kafka.apache.org
Date:   28/06/2016 21:53
Subject:        Re: [DISCUSS] Client Side Auto Topic Creation



Hey Grant,

I'm also inclined to the latter two options you've suggested (and probably
more to the one which uses AdminClient since it encourages separation of
privileges). Raising an exception gives the user the most control over how
to handle unknown topics. 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. I think a
safer default would be to treat unknown topics as fatal errors. Of course,
this is slightly orthogonal to the question of autocreation. We could
change the behavior to raise an exception only if autocreation is not
enabled, but I'd rather push users to creating topics with the AdminClient
since it saves us from needing to add a bunch of default topic
configuration options to the client configurations.

Thanks,
Jason

On Tue, Jun 28, 2016 at 9:26 AM, Grant Henke <gh...@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
> grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>



Unless stated otherwise above:
IBM United Kingdom Limited - Registered in England and Wales with number 
741598. 
Registered office: PO Box 41, North Harbour, Portsmouth, Hampshire PO6 3AU

Re: [DISCUSS] Client Side Auto Topic Creation

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Grant,

I'm also inclined to the latter two options you've suggested (and probably
more to the one which uses AdminClient since it encourages separation of
privileges). Raising an exception gives the user the most control over how
to handle unknown topics. 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. I think a
safer default would be to treat unknown topics as fatal errors. Of course,
this is slightly orthogonal to the question of autocreation. We could
change the behavior to raise an exception only if autocreation is not
enabled, but I'd rather push users to creating topics with the AdminClient
since it saves us from needing to add a bunch of default topic
configuration options to the client configurations.

Thanks,
Jason

On Tue, Jun 28, 2016 at 9:26 AM, Grant Henke <gh...@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
> grant@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>