Table of Contents |
---|
Current state: AcceptedAdopted
Discussion thread:
JIRA: KAFKA-12313
...
- StreamsConfig
- Deprecate default.windowed.key.serde.inner and default.windowed.value.serde.inner.
- Introduce a new config called windowed.inner.class.serde. This way, the config comes closer to the window.size.ms config introduced in KIP-659 and it also emphasises that the config isn't really a default one.
- ConsoleConsumer
- It would be mandatory to pass windowed.inner.class.serde and window.size.ms config.
- The following table discusses the proposed changes for the Windowed Serdes and proposed changes based on Client Type.
Windowed Serde Type Client Type Constructor Invoked Proposed Change 1 TimeWindowedDeserializer
Console Consumer Default constructor Ensure both windowSize and Serde class configs are set. Throw error if not. Plain Consumer Default OR Parameterised Ok to use either as long as the configs supplied don't conflict. Kafka Streams Parameterised User supplies parameters by constructing a TimeWindowedSerde object. No change 2 TimeWindowedSerializer Console Producer Default constructor Ensure Serde class config is set. Throw error if not. Plain Producer Default OR Parameterised Ok to use either as long as the configs supplied don't conflict. Kafka Streams Parameterised User supplies parameters by constructing a TimeWindowedSerde object. No change 3 SessionWindowedDeserializer Console consumer Default constructor Ensure Serde class config is set. Throw error if not. Plain consumer Default OR Parameterised Ok to use either as long as the configs supplied don't conflict. Kafka Streams Parameterised User supplies parameters by constructing a SessionWindowedSerde object. No change 4 SessionWindowedSerializer Console Producer Default constructor Ensure Serde class config is set. Throw error if not. Plain Producer Default OR Parameterised Ok to use either as long as the configs supplied don't conflict. Kafka Streams Parameterised User supplies parameters by constructing a SessionWindowedSerde object. No change
Public Interfaces
- StreamsConfig:
- Deprecate default.windowed.key.serde.inner and default.windowed.value.serde.inner in StreamConfig.
- Introduce a new config called windowed.inner.class.serde.
Code Block language java theme Midnight public static final String WINDOWED_INNER_CLASS_SERDE = "windowed.inner.class.serde"; private static final String WINDOWED_INNER_CLASS_SERDE_DOC = " Serde for the inner class of a windowed record. Must implement the " + "<code>org.apache.kafka.common.serialization.Serde</code> interface. Note that setting this config in KafkaStreams application would result " + "in an error as it is meant to be used only from Plain consumer client."; @Deprecated public static final String DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS = "default.windowed.key.serde.inner"; @Deprecated public static final String DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS = "default.windowed.value.serde.inner";
- Introduce a new config called windowed.inner.class.serde.
- Deprecate default.windowed.key.serde.inner and default.windowed.value.serde.inner in StreamConfig.
...