Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current state: Under Discussion

...

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

Kafka Streams uses repartition topics to repartition the data when a key-based operation, e.g., aggregation or join, follows a key-changing operation, e.g., map. Additionally, Kafka Streams uses changelog topics to replicate the data in its state store for fault-tolerance. If any required repartition topic or changelog topic does not exist, it is created during a rebalance. More precisely these internal topics are created during the computation of the assignment. Consequently, if one of the internal topics is deleted between rebalances, it will be silently recreated as an empty topic and data might be lost. Kafka Streams users would either not notice the loss at all or notice it too late to limit the damage by stopping processing. Deletion of internal topics may happen by mistake. This silent recreation of internal topics could be avoided by creating the internal topics only once during the first-ever rebalance of the application (or after an application reset). However, determining the first-ever rebalance of an application is not always straightforward. For example, if all Kafka Streams clients of a Kafka Streams application are stopped, an internal topic is deleted, and then the Kafka Streams application is restarted, the deleted topic would be silently recreated as an empty topic because Kafka Streams does not have any information that it could leverage to recognize that the first rebalance after restarting is not the first-ever rebalance of the application.

We propose to move the creation of the internal topics to an initialization method that users can call before they start their application for the first time. That way, users have full control over when the internal topics are created and Kafka Streams can reliably notify the users about possible data loss by throwing an exception when internal topics do unexpectedly not exist. Additionally, we propose to add a configuration that allows to turn automatic creation of internal topics on or off. The configuration is needed for backward compatibility but also to keep the first steps with Kafka Streams simple for new users.

Public Interfaces

We propose to add a new exception that helps to discriminate errors originating from missing internal topics from other errors in the uncaught exception handler. For example, reacting on a missing source topic (i.e., MissingSourceTopicException) might be different from reacting on missing internal topics, because the process for re-creating source topics might differ from the process for re-creating internal topics. Furthermore, source topics might be owned by a different team than the internal topics, consequently different people need to be paged.

...

Code Block
languagejava
public class StreamsConfig {

	// possible values
	public static final String AUTOMATIC_SETUP = "automatic";
	public static final String MANUAL_SETUP = "manual";

	// configuration
    public static final String INTERNAL_TOPIC_SETUP = "internal.topics.setup";  // default is AUTOMATIC_SETUP 
 
}


Proposed Changes

If configuration INTERNAL_TOPIC_SETUP is set to AUTOMATIC_SETUP, the internal topics will be set up during a rebalance. If the internal topics do not exist during a rebalance, they will be setup. That corresponds to the current behavior of Kafka Streams. Users can also call KafkaStreams#init() to setup the internal topics when INTERNAL_TOPIC_SETUP is set to AUTOMATIC_SETUP, but the call is not necessary since the internal topics would be created anyways during the next rebalance.

...

In addition to setup internal topics, KafkaStreams#init() will make all checks that are currently done during a rebalance including checks for source and sink topics.

Compatibility, Deprecation, and Migration Plan

Since we introduce configuration INTERNAL_TOPIC_SETUP with default value AUTOMATIC_SETUP that ensures the current Kafka Streams behavior, this KIP should not affect backward-compatibility. 

...

Migrating from the current behavior to INTERNAL_TOPIC_SETUP set to MANUAL_SETUP can be done without any specific migration plan. Users need to set INTERNAL_TOPIC_SETUP to MANUAL_SETUP.  

Rejected Alternatives

  • Persist a flag for the first-ever rebalance broker side: This approach was rejected because that would imply changes on the brokers which we thought we can avoid and still get a good solution with the approach proposed in this KIP.
  • Use committed offsets for a repartition topic to verify if a repartition topic existed: This approach would not work since committed offsets are removed when a topic is deleted.

...