[
https://issues.apache.org/jira/browse/KAFKA-5386?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16099058#comment-16099058
]
Bart Vercammen edited comment on KAFKA-5386 at 7/25/17 7:03 AM:
----------------------------------------------------------------
[~mjsax] {{<application.id>-<storeName>-changelog}} that's exactly the problem
in our case ;-)
We have some strict rules about the naming of kafka topics in our project and
would like to have full control over them, including the ACLs on each
kafka-topic.
So basically we do not want applications to start creating whatever topics all
over the place.
This being said, {{<application.id>-<storeName>-changelog}} would work if we
could add some wildcard ACLs on the creation of Kafka topics, but that would
imply writing an own authorizer in Kafka allowing only the creation of topics
that comply to a specific naming schema.
Changing the default {{<application.id>-<storeName>-changelog}} schema into
something more configurable would be easier for KafkaStreams applications.
The (dirty) patch I made was simply to allow {{ProcessorStateManager}} to
accept a {{static}} _prefix_ and _suffix_ configuration and just replace the
{{storeChangelogTopic}} function to return {{<prefix>.<storeName>.<suffix>}}
when they are defined. This would of course not work for the dynamically
created repartitioning topics, but that could then be solved with your tip to
use {{through("...")}} ...
Let me see how far I get to make this cleaner and more configurable without
impacting the public API too much ... ;-)
was (Author: cloutrix):
[~mjsax] {{<application.id>-<storeName>-changelog}} that's exactly the problem
in our case ;-)
We have some strict rules about the naming of kafka topics in our project and
would like to have full control over them, including the ACLs on each
kafka-topic.
So basically we do not want applications to start creating whatever topics all
over the place.
This being said, {{<application.id>-<storeName>-changelog}} would work if we
could add some wildcard ACLs on the creation of Kafka topics, but that would
imply writing an own authorizer in Kafka allowing only the creation of topics
that comply to a specific naming schema.
Changing the default {{<application.id>-<storeName>-changelog}} schema into
something more configurable would be easier for KafkaStreams applications.
The (dirty) patch I made was simply to allow {{ProcessorStateManager}} to
accept a {{static}} _prefix_ and _suffix_ configuration and just replace the
{{storeChangelogTopic}} function to return {{<prefix>.<storeName>.<suffix>}}
when they are defined. This would of course not work for the dynamically
created repartitioning topics, but that could then be solved with your tip to
use {{through("...")}} ...
Let me see how far I get to make this cleaner and more configurable without
impacting the public API to much ... ;-)
> [Kafka Streams] - custom name for state-store change-log topic
> --------------------------------------------------------------
>
> Key: KAFKA-5386
> URL: https://issues.apache.org/jira/browse/KAFKA-5386
> Project: Kafka
> Issue Type: Improvement
> Components: streams
> Affects Versions: 0.10.2.1
> Reporter: Bart Vercammen
> Labels: needs-kip
>
> Currently, when working with Kafka backed state stores in Kafka Streams,
> these log compacted topics are given a hardcoded name :
> _my.app.id-storename-changelog_
> {noformat} public static String storeChangelogTopic(String applicationId,
> String storeName) {
> return applicationId + "-" + storeName + STATE_CHANGELOG_TOPIC_SUFFIX;
> }{noformat}
> It would be nice if somehow I would be able to override this functionality
> and provide the topic-name myself when creating the state-store.
> Any comments?
> Would it be OK to submit a PR for this?
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)