For what it's worth the use case for auto-creation isn't using a dynamic
set of topics, but rather letting apps flow through different
dev/staging/prod/integration_testing/unit_testing environments without
having the app configure appropriate replication/partitioning stuff in each
environment and having complex logic to check if the topic is there.
Basically if you leave this up to individual apps you get kind of a mess,
it's better to have cluster defaults that are reasonable and controlled by
an admin and then pre-provision anything that is weird (super big, unusual
perms, whatever). Usually in the pre-prod environments you don't really
care about the settings at all, and in prod you can pre-provision.

This raises an important point about how we handle defaults, which I don't
think we talked about. I do think it is really important that we allow a
way to create topics with the "cluster defaults". I know this is possible
for configs since if you omit them they inherit default values, but I think
we should be able to do it with replication factor and partition count too.
I think the Java API should expose this and maybe even encourage it.

I don't have a super strong opinion on how this is exposed, though I kind
of prefer one of two options:
1. Keep the approach we have now with a config option to allow auto create,
but using this option just gives you a plain vanilla topic with no custom
configs, for anything custom you need to use AdminClient "manually"
2. Just throw an exception and let you use AdminClient. This may be a bit
of a transition for people relying on the current behavior.

I kind of feel once you start adding AdminClient methods to the producer
and consumer it's not really clear where to stop--e.g. if I can create I
should be able to delete, list, etc.

-Jay

On Tue, Jun 28, 2016 at 9:26 AM, Grant Henke <ghe...@cloudera.com> wrote:

> With the KIP-4 create topic schema voted and passed and a PR available
> upstream. I wanted to discuss moving the auto topic creation from the
> broker side to the client side (KAFKA-2410
> <https://issues.apache.org/jira/browse/KAFKA-2410>).
>
> This change has many benefits
>
>    - Remove the need for failed messages until a topic is created
>    - Client can define the auto create parameters instead of a global
>    cluster setting
>    - Errors can be communicated back to the client more clearly
>
> Overall auto create is not my favorite feature, since topic creation is a
> highly critical piece for Kafka, and with authorization added it becomes
> even more involved. When creating a topic a user needs:
>
>    - The access to create topics
>    - To set the correct partition count and replication factor for their
>    use case
>    - To set who has access to the topic
>    - Knowledge of how a new topic may impact regex consumers or mirrormaker
>
> Often I find use cases that look like they need auto topic creation, can
> often be handled with a few pre made topics. That said, we still should
> support the feature for the cases that need it (mirrormaker, streams).
>
> The question is how we should expose auto topic creation in the client. A
> few options are:
>
>    - Add configs like the broker configs today, and let the client
>    automatically create the topics if enabled
>       - Both producer and consumer?
>    - Throw an error to the user and let them use a separate AdminClient
>    (KIP-4) api to create the topic
>    - Throw an error to the user and add a create api to the producer so
>    they can easily handle by creating a topic
>
> I am leaning towards the last 2 options but wanted to get some others
> thoughts on the matter. Especially if you have use cases that use auto
> topic creation today.
>
> Thanks,
> Grant
>
> --
> Grant Henke
> Software Engineer | Cloudera
> gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
>

Reply via email to