Status

Current state: Accepted

Discussion thread: here

JIRAKAFKA-18023

Motivation

Kafka Streams applications typically rely on internal topics, such as changelog and repartition topics. By default, Kafka Streams auto-generates names for these internal topics if the user has not explicitly defined them.

This auto-generation process includes a numeric index based on each topic’s position within the topology (e.g., <application.id>-KSTREAM-KEY-SELECT-0000000003-repartition).

While convenient, relying on generated names can lead to issues during topology updates, as changes in the topology structure may alter these generated names, potentially breaking compatibility and causing the loss of previously built-up state. To avoid these issues, it is often beneficial to enforce explicit naming of internal topics, especially in production environments where stability and consistency are critical.

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, thereby enforcing explicit naming for all internal topics and enhancing the stability of the application’s topology.

Note: Enforcing explicit naming of changelog topics also enforces explicit naming of state stores, as these are closely related.

Public Interfaces

Users will be able to ensure that internal topic names  are explicity named with a new configuration in StreamsConfig :

public static final String  ENSURE_EXPLICIT_INTERNAL_RESOURCE_NAMING_CONFIG = "ensure.explicit.internal.resource.naming";

.define( ENSURE_EXPLICIT_INTERNAL_RESOURCE_NAMING_CONFIG,
        Type.BOOLEAN,
        false,
        Importance.HIGH,         
        ENSURE_EXPLICIT_INTERNAL_RESOURCE_NAMING_DOC)

To not break compatibility with existing applications, the default value is false.

The new configuration is passed to StreamsBuilder.build(properties)

When using the no-arg StreamsBuilder.build() method, Kafka Streams will not check if it must ensure explicit internal topic names.

Proposed Changes

The implementation will update InternalTopologyBuilder class to validate the names of internal topics during the topology-building phase.


If an internal topic with an auto-generated name is detected, the application will throw a TopologyException with a clear message indicating which internal topics do not have an explicit name. The message will also reference the ensure.explicit.internal.resource.naming configuration parameter.

Additionally, if ensure.explicit.internal.resource.naming is set to false, a warning-level log can be added to alert users about the presence of auto-generated names.

Compatibility, Deprecation, and Migration Plan

Backward Compatibility:
The default value of ensure.explicit.internal.resource.naming is set to false, preserving the current behavior and allowing applications to start with auto-generated internal topic names by default.

No Deprecation Required:
This feature introduces a new configuration parameter without deprecating any existing configurations.

Migration Plan:
No migration is required for existing Kafka Streams applications. Users who wish to enforce explicit naming can set ensure.explicit.internal.resource.naming to true in their application configurations. However, they should note that enabling this setting may result in a loss of state for changelog topics if previous topologies relied on auto-generated names.

Test Plan

  • Add unit tests to InternalTopologyBuilderTest and InternalStreamsBuilderTest

Rejected Alternatives

  • Add enforcement also on the processor names

As an additional measure, an option could be introduced to require users to assign explicit names to all processors (both stateful and stateless). This new configuration, ensure.explicit.internal.resource.naming, could take a String value:

    • TOPIC to check only internal topic names.
    • ALL to check both internal topic names and processor names.

This approach was rejected to keep the proposal simple and to limit its scope.


  • No labels