Versions Compared

Key

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

Table of Contents

Status

Current stateUnder Discussion

Discussion threadhttps://sematext.com/opensee/m/Kafka/uyzND1VU1Ou1y0Lbh?subj=+DISCUSS+KIP+466+Add+support+for+List+lt+T+gt+serialization+and+deserialization

JIRA

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-8326

Motivation

I believe there are many use cases where List Serde could be used. Ex. https://stackoverflow.com/questions/41427174/aggregate-java-objects-in-a-list-with-kafka-streams-dsl-windowshttps://stackoverflow.com/questions/46365884/issue-with-arraylist-serde-in-kafka-streams-api

For instance, aggregate grouped (by key) values together in a list to do other subsequent operations on the collection.

Public Interfaces


  • New class org.apache.kafka.common.serialization.ListSerializer which implements the Serializer<List<T>> interface
  • New class org.apache.kafka.common.serialization.ListDeserializer which implements Deserializer<List<T>> interface
  • New subclass ListSerde<T> in org.apache.kafka.common.serialization.Serdes which creates new serde based on ListSerializer and ListDeserializer classes
  • New method public static <T> Serde<List<T>> ListSerde(Serde<T> innerSerde) in org.apache.kafka.common.serialization.Serdes class

Proposed Changes

This KIP proposes adding new ListSerializer and ListDeserializer classes as well as support for the new classes into the Serdes class. This will allow using List<T> Serde directly from Consumers, Producers and Streams.

List<T> serialization and deserialization will be done through repeatedly calling a serializer/deserializer for each entry provided by passed generic T's Serde. For example, if you want to create List of Strings serde, then serializer/deserializer of StringSerde will be used to serialize/deserialize each entry in `List<String>`.

Compatibility, Deprecation, and Migration Plan

Does not apply

Rejected Alternatives

Not known