Status

Current state: Adopted

Discussion thread

JIRA: KAFKA-12313

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


Motivation

KIP-659 introduced a config windowSize to TimeWindowedDeserializer which discourages the setting of windowSize to Long.MAX_VALUE as the default value. However, there is still a possibility of setting the config only for innerClassSerde or only for windowSize. This KIP aims at standardising the way the serialisers and deserializer object can be created for the usage in console consumer, plain consumer and via the DSL. 

Proposed Changes

Here are the changes being proposed in the KIP:

  1. StreamsConfig
    1. Deprecate default.windowed.key.serde.inner and default.windowed.value.serde.inner
    2. 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.
  2. ConsoleConsumer
    1. It would be mandatory to pass windowed.inner.class.serde and window.size.ms config. 
  3. The following table discusses the proposed changes for the Windowed Serdes and proposed changes based on Client Type.

    Windowed Serde TypeClient TypeConstructor InvokedProposed Change
    1

    TimeWindowedDeserializer

    Console ConsumerDefault constructorEnsure both windowSize and Serde class configs are set. Throw error if not.
    Plain ConsumerDefault OR ParameterisedOk to use either as long as the configs supplied don't conflict.
    Kafka StreamsParameterisedUser supplies parameters by constructing a TimeWindowedSerde object. No change
    2TimeWindowedSerializerConsole ProducerDefault constructorEnsure Serde class config is set. Throw error if not.
    Plain ProducerDefault OR ParameterisedOk to use either as long as the configs supplied don't conflict.
    Kafka StreamsParameterisedUser supplies parameters by constructing a TimeWindowedSerde object. No change
    3SessionWindowedDeserializerConsole consumerDefault constructorEnsure Serde class config is set. Throw error if not.
    Plain consumerDefault OR ParameterisedOk to use either as long as the configs supplied don't conflict.
    Kafka StreamsParameterisedUser supplies parameters by constructing a SessionWindowedSerde object. No change
    4SessionWindowedSerializerConsole ProducerDefault constructorEnsure Serde class config is set. Throw error if not.
    Plain ProducerDefault OR ParameterisedOk to use either as long as the configs supplied don't conflict.
    Kafka StreamsParameterisedUser 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
            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";

Compatibility, Deprecation, and Migration Plan

This KIP deprecates following 2 configs in StreamConfig : default.windowed.key.serde.inner and default.windowed.value.serde.inner

Rejected Alternatives

N/A



  • No labels