Versions Compared

Key

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

...

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 a 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.

...

If configuration APPLICATION_INITIALIZATION is set to USER_INITIALIZATION, the internal topics will not be set up during a rebalance but users need to call KafkaStreams.init() to setup the internal topics. If the internal topics do not exist during a rebalance because KafkaStreams.init() was not called or one or more internal topics were deleted, a MissingInternalTopicException is thrown in each Kafka Streams client.

If KafkaStreams.init() is called and one or more internal topics already exist, then only the missing internal topics are created as empty topics. Additionally to creating internal topics, KafkaStreams.init() will make all checks that are currently done during a rebalance.

Compatibility, Deprecation, and Migration Plan

Since metrics are only added and no other metrics are modifiedwe introduce configuration APPLICATION_INITIALIZATION with default value AUTOMATIC_INITIALIZATION that ensures the current Kafka Streams behavior, this KIP should not affect backward-compatibility

We do not need to deprecate any public interfaces

...

since we propose to add a new method to the public API that does not replace another method.

Migrating from the current behavior to APPLICATION_INITIALIZATION set to USER_INITIALIZATION can be done without any specific migration plan. Users need to set APPLICATION_INITIALIZATION to USER_INITIALIZATION. They can additionally also call KafkaStreams.init() if they think it is safe to re-create missing internal topics or if they know that no internal topics are missing.     

Rejected Alternatives

  • Persist 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.

...