Table of Contents |
---|
Status
Current state: Under Accepted
Discussion thread: hereDiscussion
Voting thread: here
JIRA:
Jira | ||||||||
---|---|---|---|---|---|---|---|---|
|
...
KIP-479, implemented in Kafka 2.4.0, added StreamJoined
to Kafka Streams as a way to extend features of Materialized
to stream-stream joins. During the KIP discussion, adding the option to disable logging was discussed but not implemented through KIP-479. Adding this feature allows users more flexibility over their store and fits with the original goal of extending Materialized
features to stream-stream joins.
...
This KIP will add the following two methods as well as a loggingEnabled
variable and a topicConfig
variable to store any configs passed in when enabling logging.
Code Block | ||
---|---|---|
| ||
public StreamJoined<K, V1, V2> withLoggingEnabled(final WindowBytesStoreSupplier otherStoreSupplierMap<String, String> config) {} public StreamJoined<K, V1, V2> withLoggingDisabled(final WindowBytesStoreSupplier otherStoreSupplier) {} |
Proposed Changes
We will add two new methods to StreamJoined
to allow users to determine whether they want logging for their stores. This config sets the logging decision for both stores, meaning that if logging is disabled for thisWindowStore
, it will also be disabled for otherWindowStore
. The default value for loggingEnabled
will be true
, enabling logging for both stores if the user does not set the values manually.
...