Thanks for the discussion, below are some thoughts and responses.

One of the problems that we currently have with
> the clients is that we retry silently on unknown topics under the
> expectation that they will eventually be created (automatically or not).
> This makes it difficult to detect misconfiguration without looking for
> warnings in the logs. This problem is compounded if the client isn't
> authorized to the topic since then we don't actually know if the topic
> exists or not and whether it is reasonable to keep retrying.


Yeah this is a problem thats difficult and opaque to the user. I think any
of the proposed solutions would help solve this issue. Since the create
would be done at the metadata request phase, instead of in the produce
response handling. And if the create fails, the user would receive a munch
more clear authorization error.

The current auto creation of topic by the broker appear to be the only
> reason an unknown topic error is retriable
> which leads to bugs (like https://issues.apache.org/jira/browse/KAFKA-3727
> ) where the consumer hangs forever (or until woken up) and only debug
> tracing shows what's going on.
>

I agree this is related, but should be solvable even with retriable
exceptions. I think UnknownTopicOrPartitionException needs to remain
generally retriable because it could occur due to outdated metadata and not
because a topic needs to be created. In the case of message production or
consumption it could be explicitly handled differently in the client.

Do we clearly define the expected behavior of subscribe and assign in the
case of a missing topic? I can see reasons to fail early (partition will
never exist, typo in topic name) and reasons to keep returning empty record
sets until the topic exists (consumer with a preconfigured list of topics
that may or may not exist). Though I think failing and insisting topics
exist is the most predictable. Especially since the Admin API will make
creating topics easier.

Usually in the pre-prod environments you don't really
> care about the settings at all, and in prod you can pre-provision.


I like the recommendations, developer/ops experience and required exercises
to be fairly consistent between dev, qa, and prod. If you need to
pre-provision and think about the settings in prod. Its best to put some
effort into building that logic in dev or qa too. Otherwise you get ready
to deploy and everything changes and all your earlier testing is not as
relevant.

For what it's worth the use case for auto-creation isn't using a dynamic
> set of topics, but rather letting apps flow through different
> dev/staging/prod/integration_testing/unit_testing environments without
> having the app configure appropriate replication/partitioning stuff in each
> environment and having complex logic to check if the topic is there.
>

The problem I have seen here is that the cluster default is global, at
least until we have some concept of namespaces and can configure defaults
for each. Since picking a good number of partitions varies based on volume,
use case, etc a default that works for most topics is a hard to find.

I feel like because app developers think they don't need to think about
topic creation, often they don't. And that leads to a mess where they don't
know how may partitions and what replication factor they have. Instead
migrating environments with a setup script that creates the needed topics
allows them to source control those setting and create predictable,
repeatable deployments.

I have also seen a lot of issues where users are confused about why a topic
is coming back or can't be deleted. This is often a result
of auto.create.topics.enable being defaulted to true. And they never expect
that a feature like that would exist, much less be the default.

On a side note, the best dynamic use case I could think of is MirrorMaker.
But the cluster defaults here don't really work since its they are not very
flexible. Pushing creation to the client would allow tools like MirrorMaker
to create topics that match the upstream cluster, or provide its own logic
for sizing downstream topics.

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

We could make the create topic request num_partitions and
replication_factor fields optional and if unset use the cluster defaults.
This allows a user to opt into the cluster defaults at create time. I have
rarely seen good defaults set in my experience though, especially since the
default is 1 in both cases.

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


I agree this gets weird and could lead to duplicate client code and
inconsistent behavior across clients. The one thing I don't like about
requiring a separate client is it maintains all its own connections and
metadata. Perhaps sometime down the road if we see a lot of mixed usage we
could break out the core cluster connection code into a KafkaConnection
class and instantiate clients with that. That way clients could share the
same KafkaConnection.

Thanks,
Grant


On Wed, Jun 29, 2016 at 9:29 AM, Jay Kreps <j...@confluent.io> wrote:

> For what it's worth the use case for auto-creation isn't using a dynamic
> set of topics, but rather letting apps flow through different
> dev/staging/prod/integration_testing/unit_testing environments without
> having the app configure appropriate replication/partitioning stuff in each
> environment and having complex logic to check if the topic is there.
> Basically if you leave this up to individual apps you get kind of a mess,
> it's better to have cluster defaults that are reasonable and controlled by
> an admin and then pre-provision anything that is weird (super big, unusual
> perms, whatever). Usually in the pre-prod environments you don't really
> care about the settings at all, and in prod you can pre-provision.
>
> This raises an important point about how we handle defaults, which I don't
> think we talked about. I do think it is really important that we allow a
> way to create topics with the "cluster defaults". I know this is possible
> for configs since if you omit them they inherit default values, but I think
> we should be able to do it with replication factor and partition count too.
> I think the Java API should expose this and maybe even encourage it.
>
> I don't have a super strong opinion on how this is exposed, though I kind
> of prefer one of two options:
> 1. Keep the approach we have now with a config option to allow auto create,
> but using this option just gives you a plain vanilla topic with no custom
> configs, for anything custom you need to use AdminClient "manually"
> 2. Just throw an exception and let you use AdminClient. This may be a bit
> of a transition for people relying on the current behavior.
>
> I kind of feel once you start adding AdminClient methods to the producer
> and consumer it's not really clear where to stop--e.g. if I can create I
> should be able to delete, list, etc.
>
> -Jay
>
> On Tue, Jun 28, 2016 at 9:26 AM, Grant Henke <ghe...@cloudera.com> wrote:
>
> > With the KIP-4 create topic schema voted and passed and a PR available
> > upstream. I wanted to discuss moving the auto topic creation from the
> > broker side to the client side (KAFKA-2410
> > <https://issues.apache.org/jira/browse/KAFKA-2410>).
> >
> > This change has many benefits
> >
> >    - Remove the need for failed messages until a topic is created
> >    - Client can define the auto create parameters instead of a global
> >    cluster setting
> >    - Errors can be communicated back to the client more clearly
> >
> > Overall auto create is not my favorite feature, since topic creation is a
> > highly critical piece for Kafka, and with authorization added it becomes
> > even more involved. When creating a topic a user needs:
> >
> >    - The access to create topics
> >    - To set the correct partition count and replication factor for their
> >    use case
> >    - To set who has access to the topic
> >    - Knowledge of how a new topic may impact regex consumers or
> mirrormaker
> >
> > Often I find use cases that look like they need auto topic creation, can
> > often be handled with a few pre made topics. That said, we still should
> > support the feature for the cases that need it (mirrormaker, streams).
> >
> > The question is how we should expose auto topic creation in the client. A
> > few options are:
> >
> >    - Add configs like the broker configs today, and let the client
> >    automatically create the topics if enabled
> >       - Both producer and consumer?
> >    - Throw an error to the user and let them use a separate AdminClient
> >    (KIP-4) api to create the topic
> >    - Throw an error to the user and add a create api to the producer so
> >    they can easily handle by creating a topic
> >
> > I am leaning towards the last 2 options but wanted to get some others
> > thoughts on the matter. Especially if you have use cases that use auto
> > topic creation today.
> >
> > Thanks,
> > Grant
> >
> > --
> > Grant Henke
> > Software Engineer | Cloudera
> > gr...@cloudera.com | twitter.com/gchenke | linkedin.com/in/granthenke
> >
>



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

Reply via email to