One last minor question:

You propose to add this new config with importance level "LOW". I am wondering if this is appropriate?

From my understanding, the rule-of-thumb is, that "low" means, most likely not necessary to change, while "high" mean, recommended to change for production.

Thus, to me, this new config might qualify as "high" priority config? Thoughts?

Overall it's not a big deal, because the "priority value" is not really part of public API, and we can change it any time, but I though asking can't hurt.



-Matthias

On 12/5/24 7:20 AM, Lucas Brutschy wrote:
Hi,

agreeing with Bill here. I'm still not sure, I like `resource` but I
won't insist - ready for a vote.

Cheers,
Lucas

On Thu, Dec 5, 2024 at 12:31 AM Bill Bejeck <bbej...@gmail.com> wrote:

Hi All,

Thanks for the KIP, Sébastien, this will be a very useful addition.

I don't have any additional comments at this point, but I'd like to
add a +1 for current naming of `ensure.explicit.internal.resource.naming`.
I think it strikes the correct balance among all the suggestions on this
thread.
At this point, I'm thinking this could be ready to put to a vote in the
next day or two.

Regards,
Bill

On Wed, Dec 4, 2024 at 3:01 PM Sebastien Viale <sebastien.vi...@michelin.com>
wrote:

Hi,

BC+1:
I have updated the KIP with ensure.explicit.internal.resource.naming
(default: false) as Bruno suggested.


BC+2:
Implementation details have been removed.


Regarding the issue with "DSL/StreamsBuilder" configs, I will open a
separate KIP once this one is definitively accepted or rejected.

Regarding Almog's idea, I believe it is feasible to account for existing
auto-generated topics. However, I also prefer to focus on new applications
or recommend that users reset their streams before enabling the
configuration.

Thanks for your feedback


Cheers
Sébastien






________________________________
De : Bruno Cadonna <cado...@apache.org>
Envoyé : mardi 3 décembre 2024 09:14
À : dev@kafka.apache.org <dev@kafka.apache.org>
Objet : [SUSPICIOUS EXTERNAL MESSAGE] Re: [DISCUSS] KIP-1111: Enforcing
Explicit Naming for Kafka Streams Internal Topics

Warning This might be a fraudulent message! When clicking REPLY, your
answers will NOT go to the sender (cado...@apache.org). Instead, replies
will be sent to dev@kafka.apache.org. Be cautious!

Hi all,

BC+1:
Just to be clear, I did not mean state stores and their changelogs with
"persistence", but every resource a Streams app needs to persists on the
brokers as well as locall to be able to work (e.g., changelog topics and
repartition topics). Since this seems to be confusing, we should use
some other term to describe those resources. Maybe:

- ensure.invariant.internal.resource.naming
- ensure.explicit.internal.resource.naming

Regarding "resources" being too generic, I think being generic is good
in this case, because this config can then be used for all resources
that are persisted on the brokers (repartition topics and changelog
topics) as well as locally (state store names) that need to be invariant
to changes to the topology. Maybe there will be more in the future.


BC+2:
I agree with Matthias that we should remove the implementation details
from the KIP.


BC+3:
Sébastien, you also need to consider the time ordered key value buffers
used in suppressions and stream-table join with grace periods. Those
also use changelog topics.


Best,
Bruno

This email was screened for spam and malicious content but exercise
caution anyway.



On 03.12.24 03:05, Matthias J. Sax wrote:
Hi,

I lost track of all proposals for the config name... But from the ones I
recall, and from what people commented, I would like to propose:

   - enable.internal.topic.name.generation (default: true)

Given that we have repartition and changelog topics, it seem using
"topic" in the name is more generic (compared to store). Also, changelog
topic and store name are coupled, and thus "topic" does cover store
names implicitly.

I find other term like "resources" too generic and hard to understand
personally.


About the related question about working on the issue about "DSL/
StreamsBuilder" configs, I am happy either way to either include it in
this KIP or separate it out. It might be cleaner to have two KIPs, but
given that there is natural overlap between both, it can also become
confusing... I guess it will be your own decision Sebastian, which way
you prefer personally.


The other discussion about how to ensure topics name are not auto-
generated seems to go pretty much into implementation details... Not
sure if we really need to discuss this on the mailing list? I have a few
of my own ideas about it, but not sure if this is relevant at this stage?


In the end, if we keep the KIP scope as-is (and don't extend it to cover
the general DSL/StreamsBuilder config question), we only need to agree
on a name for the config? And consider the upgrade path.

I am not sure if Almog's idea to pass in custom names to mimic auto-
generated ones would actually work? Btw: I am also happy if we say, the
config can only be enabled for _new_ applications in a safe manner, and
if people enable it for existing ones, it's their own risk (while it
should be safe to enable for topologies for which everything is explicit
named already).


-Matthias




On 12/2/24 3:25 AM, Sebastien Viale wrote:
A1 / BC+1
What do you think of these options for taking internal topics and
state store names into consideration?

    *   ensure.explicit.internal.names
    *   ensure.explicit.resource.names
    *   ensure.consistent.resource.naming
    *   ensure.invariant.persistence.naming

BC+2
To not rely only on a pattern, this is what can be done:

- For repartition topics, they are all created in the
KStreamImpl.createRepartitionedSource(...) static method.
The method either receives a name explicitly provided by the user or
null and builds the final repartition topic name.
Here, I can easily determine if a name has been provided.
- For changelog topics and state store names, I identified two
situations where they are created:

    1.  In the MaterializedInternal constructor, which receives a
Materialized object with a name or not.
    2.  When a KStream/KStream join is made. In this case, stores are
not built using Materialized but are instead created in the
KStreamImplJoin.join() method.

In all cases, I can access the InternalTopologyBuilder from the
InternalStreamBuilder object and add the unprovided topics or store
names to a list property:
List<String> UnprovidedInternalNames.
I will then be able to check the content of this list to enforce the
explicit naming requirement if needed.
Thanks for your feedback.

regards,
Sébastien

________________________________
De : Lucas Brutschy <lbruts...@confluent.io.INVALID>
Envoyé : jeudi 28 novembre 2024 11:51
À : dev@kafka.apache.org <dev@kafka.apache.org>
Objet : [SUSPICIOUS EXTERNAL MESSAGE] Re: [DISCUSS] KIP-1111:
Enforcing Explicit Naming for Kafka Streams Internal Topics

Warning This might be a fraudulent message! When clicking REPLY, your
answers will NOT go to the sender (lbruts...@confluent.io.invalid).
Instead, replies will be sent to dev@kafka.apache.org. Be cautious!

Hi all,

Thanks for the KIP! Super useful.

No new comments, let me just voice my opinion on the suggestions being
made.

A1: When I read `require.auto.generated.topic.names`, it sounds like
explicit naming is not allowed if the config is true. This is not what
we are doing here. So to avoid the negative, I'd use
`require.explicit.topic.names`, default false.

BC+1: I think `invariant` is quite indirect, because it depends on
what changes are being made for the naming to change or not. Also,
it's not only about changelog topics but also repartition topics, so
`persistance` seems misleading as well. I agree, IQ can benefit, but
it seems that it is more of a side effect of the feature?

BC+2: Big +1 on this.

Cheers,
Lucas

This email was screened for spam and malicious content but exercise
caution anyway.






On Tue, Nov 26, 2024 at 9:37 PM Sebastien Viale
<sebastien.vi...@michelin.com> wrote:

Thanks Bruno for your comments:

A1 .
BC+1.
I let other people give their advice, and each of them makes sense.

BC+2.
I believe you're right. I think I can manage to check whether names
are set for repartition or changelog topics across different DSL
operators in KStreamImpl. The complexity arises because repartition
topic names can depend on the operator, such as selectKey before a
join.
If I detect unnamed topics, I could introduce a property like
unNamedInternalTopics in InternalStreamBuilder to maintain a list of
unnamed topics that I can check when topology is built. This approach
would allow me to focus not only on sequence numbers but also on
ensuring explicit topic naming for improved stability. I hope this is
what you imagine.
Let me know if you need further refinements!

Sébastien,



________________________________
De : Bruno Cadonna <cado...@apache.org>
Envoyé : mardi 26 novembre 2024 19:12
À : dev@kafka.apache.org <dev@kafka.apache.org>
Objet : [EXT] Re: [DISCUSS] KIP-1111: Enforcing Explicit Naming for
Kafka Streams Internal Topics

Warning External sender Do not click on any links or open any
attachments unless you trust the sender and know the content is safe.

Hi,

Thanks for the KIP!

A1.
I find the "auto.generated.topic.names" part a bit misleading, because
actually the topic names are always auto-generated no matter if the
processors and state stores are explicitly named or not.

which leads me to

BC+1.
IMO this is not only about topic names, explicitly naming state stores
would also benefit existing IQ queries. So the config should not
necessarily be only focused on internal topics.
Maybe a config name like invariant.persistence.naming or
ensure.invariant.persistence.naming or
enabled.invariant.persistence.naming. I am not too happy with the names
but I hope you get the idea.

BC+2.
Can we not keep track of explicit naming within the DSL instead of
relying on the 10-digit sequence number?

Best,
Bruno

This email was screened for spam and malicious content but exercise
caution anyway.



On 22.11.24 22:04, Sebastien Viale wrote:
hi,
A1. Personally, I prefer the suggestion:
require.auto.generated.topic.names.
If everyone agrees, I will update the KIP accordingly.
MJS-1. Sophie, I reviewed the ticket and the sub-tasks, and
everything seems clear to me. I must say that I sometimes felt
confused with the constructors, but now everything is much clearer.
Based on my experience and discussions with my teammates, I believe
this is a good opportunity to simplify and clarify the implementation.
The question is whether this should be part of a separate KIP.
I don’t see any issue with integrating it into the current one, but
if necessary, I’m happy to volunteer to open a new KIP.
In both cases, would it be okay for you if I validate it with you
before publishing?
cheers !



________________________________
De : Sophie Blee-Goldman <sop...@responsive.dev>
Envoyé : vendredi 22 novembre 2024 01:33
À : dev@kafka.apache.org <dev@kafka.apache.org>
Objet : [EXT] Re: [DISCUSS] KIP-1111: Enforcing Explicit Naming for
Kafka Streams Internal Topics

Warning External sender Do not click on any links or open any
attachments unless you trust the sender and know the content is safe.

First off, thanks for the KIP! I think this is a great idea as it's
super
easy to miss naming one thing and end up with a topology that isn't
upgradeable.

A1. I actually had the same reaction as Almog to the name, I feel it's
slightly clearer as a positive instead of a negative, though I think
the
rest of it is fine -- what about "require.auto.generated.topic.names"
instead?

MJS-1. While I can see the advantage of having this stuff be
programmatically configured, I personally would prefer to keep this
and
other topology-related configurations part of StreamsConfig. For one
thing,
I think they would be much more discoverable as true configs. For
another,
this still wouldn't solve the problem for configs that need to be
passed in
up front, that is, to the Topology/StreamsBuilder constructor rather
than
to StreamsBuilder#build. And imo whatever we do should work for all
such
configs that are required for building a topology.

Sebastien -- have you read through that ticket and its subtasks yet?
I'd be
interested in your take on what I'm proposing there, as I think it
could be
mutually beneficial. For context, I'm currently implementing a KIP
that
introduces another such config in this category which needs to be
passed
into the topology.

This email was screened for spam and malicious content but exercise
caution anyway.



On Thu, Nov 21, 2024 at 12:42 PM Matthias J. Sax <mj...@apache.org>
wrote:

Yes, the ticket is related. Also just saw it today.

Did leave a comment on the ticket for Sophie to hopefully chime in on
this KIP discussion.


-Matthias

On 11/21/24 9:06 AM, Sebastien Viale wrote:
Thank you very much for your reviews!

A1 I will keep the disallow.auto.generated.topic.names
configuration in
the KIP for now while waiting for other suggestions.

A2 I thought about making the implementation typesafe; it would, of
course, complicate the implementation, but for me,
this feature is intended for new Kafka Streams applications or those
after a reset. I agree with Matthias.
If a user wants to enable the check, they will simply need to avoid
naming topics like the auto-generated ones.

MJS-1 It might be a good idea to add your proposal to the KIP. I
just
wonder how to distinguish configurations that must be set
'programmatically' from others (e.g., topology optimization and
this one)
I am open to any suggestions.

Is this ticket related to what you are proposing
https://lists.apache.org/thread/<https://lists.apache.org/thread>
dfgd2vcco7d1omjptfqp92kdocnlf3cq<https://lists.apache.org/thread/<
https://lists.apache.org/thread/>
dfgd2vcco7d1omjptfqp92kdocnlf3cq><https://lists.apache.org/thread/<
https://lists.apache.org/thread/>
dfgd2vcco7d1omjptfqp92kdocnlf3cq<https://lists.apache.org/thread/<
https://lists.apache.org/thread/>
dfgd2vcco7d1omjptfqp92kdocnlf3cq>><https://lists.apache.org/<
https://lists.apache.org>
thread/dfgd2vcco7d1omjptfqp92kdocnlf3cq<https://lists.apache.org/<
https://lists.apache.org/>
thread/dfgd2vcco7d1omjptfqp92kdocnlf3cq><https://lists.apache.org/<
https://lists.apache.org/>
thread/dfgd2vcco7d1omjptfqp92kdocnlf3cq<https://lists.apache.org/<
https://lists.apache.org/>
thread/dfgd2vcco7d1omjptfqp92kdocnlf3cq>>>


cheers,
Sébastien



________________________________
De : Matthias J. Sax <mj...@apache.org>
Envoyé : jeudi 21 novembre 2024 02:48
À : dev@kafka.apache.org <dev@kafka.apache.org>
Objet : [EXT] Re: [DISCUSS] KIP-1111: Enforcing Explicit Naming for
Kafka Streams Internal Topics

Warning External sender Do not click on any links or open any
attachments unless you trust the sender and know the content is safe.

Thanks for the KIP. Overall this does sound useful.

About Almog's comments

(A1) -- I don't think that `topics.internal.require.explicit.naming`
would be a good name, as we use `topic.` prefix for actual topic
configs. Thus, KS would pickup `internal.require.explicit.naming`
and
try to apply is as topic config what would either crash or log a
spurious WARN what would be annoying?

(A2) -- We would need to check the code if such a strategy would
work as
expected? If users pass in a name for a previously un-named topic,
we
might get an cascading index shift which might be undesired (and
could
"break" processor level metrics)?

But I am also not sure if we actually need a migration path? If
might be
ok if this new feature only work for new deployments? (Or use can
re-set
their application.)


MJS-1: I have another question though about the risk such a config
implies? Given that configs are often managed from "outside", one
could
easily break an exiting application which does not have this config
enable, by enable the config. Of course, we already have similar
config
which are equally dangerous; however, most people don't like that
one
need to pass in the config into `StreamsBuilder.build(configs)`
method
anyway, so maybe we could make a first step to get rid of this via
this
KIP?

Thus, I am wondering if a config is actually the right way to go?
Should
we instead make it feature of `StreamsBuilder` that one can enable
programmatically? Not 100% how we would do this, but maybe we
could use
a builder pattern (to allow us to add similar thing in the
future), and
deprecate the current constructor of `StreamsBuilder`?

// new way to build a `StreamsBuilder`
StreamsBuilder builder = StreamsBuilder.build();

// if we don't extend the scope of this KIP, we might also need:
StreamsBuilder builder = StreamsBuilder.build(Properties);

Or we do extend this KIP and deprecate existing config (like
topology
optimization) if favor of the new builder pattern:


// enable the new feature
StreamsBuilder builder =
StreamsBuilder.requireExplicitNaming().build();

// or
StreamsBuilder builder =
StreamsBuilder.disableNameGeneration().build();


Thoughts? I am not 100% sure if this is a good idea or not, but
thought
it cannot hurt to throw it out.


-Matthias

This email was screened for spam and malicious content but exercise
caution anyway.





On 11/18/24 12:57 PM, Almog Gavra wrote:
Hi Sebastien,

Thanks for the KIP! In general, I'm a fan of giving users the
tools they
need to protect their organization so I'm supportive of this
proposal. A
few nits and comments:

A1. [nit] consider 'topics.internal.require.explicit.naming' so
that
(a) we
can group anything else we introduce for "topics.internal" with
the same
prefix and (b) it's not a double negative (don't disallow is the
default,
instead of don't require).
A2. I think we can improve on the implementation by making it
typesafe
instead of checking whether the topic matches some pattern. I
think a
migration path for users that want to turn this flag on, but
already
have
some auto-generated names, is to manually specify the
auto-generated
names
for preexisting topics. This would enforce future topics naming,
but not
penalize them for having used auto generated names in the past.
This
makes
the implementation a little more challenging, but I think it's
worthwhile.

Cheers,
Almog

On Fri, Nov 15, 2024 at 12:20 AM Sebastien Viale <
sebastien.vi...@michelin.com> wrote:

Hi Everyone,

I would like to start a discussion on KIP-1111: Enforcing Explicit
Naming
for Kafka Streams Internal Topics<

https://cwiki.apache.org/confluence/display/KAFKA/<
https://cwiki.apache.org/confluence/display/KAFKA>

KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics

<
https://cwiki.apache.org/confluence/display/KAFKA/<
https://cwiki.apache.org/confluence/display/KAFKA/>

KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics
<
https://cwiki.apache.org/confluence/display/KAFKA/KIP-1111%3A+Enforcing+Explicit+Naming+for+Kafka+Streams+Internal+Topics



This proposal aims to add a configuration that prevents a Kafka
Streams
application from starting if any of its internal topics have
auto-generated
names.
Regards,

Sébastien





Reply via email to