Table of Contents |
---|
Current state: Under DiscussionAccepted
Voting thread: here
Discussion thread: here
JIRA:
Jira | ||||||
---|---|---|---|---|---|---|
|
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
...
Create a new org.apache.kafka.streams.scala.serialization.Serdes
. It will be like an old one, but with other implicits names. The old version of Serdes should be marked as deprecated.
Also, some default Java serdes are missing from the org.apache.kafka.streams.scala.Serdes
. Since we are creating a new version of Serdes
, it also makes sense to add the missing default Java serdes to it.
Changed public Interfaces
...
Code Block | ||||||
---|---|---|---|---|---|---|
| ||||||
object Serdes { implicit def stringSerde: Serde[String] = ??? implicit def longSerde: Serde[Long] = ??? implicit def javaLongSerde: Serde[java.lang.Long] = ??? implicit def byteArraySerde: Serde[Array[Byte]] = ??? implicit def bytesSerde: Serde[org.apache.kafka.common.utils.Bytes] = ??? implicit def byteBufferSerde: Serde[ByteBuffer] = ??? implicit def shortSerde: Serde[Short] = ??? implicit def javaShortSerde: Serde[java.lang.Short] = ??? implicit def floatSerde: Serde[Float] = ??? implicit def javaFloatSerde: Serde[java.lang.Float] = ??? implicit def doubleSerde: Serde[Double] = ??? implicit def javaDoubleSerde: Serde[java.lang.Double] = ??? implicit def intSerde: Serde[Int] = ??? implicit def javaIntegerSerde: Serde[java.lang.Integer] = ??? implicit def uuidSerde: Serde[UUID] = ??? implicit def sessionWindowedSerde[T](implicit tSerde: Serde[T]): WindowedSerdes.SessionWindowedSerde[T] = ??? def fromFn[T >: Null](serializer: T => Array[Byte], deserializer: Array[Byte] => Option[T]): Serde[T] = ??? def fromFn[T >: Null](serializer: (String, T) => Array[Byte], deserializer: (String, Array[Byte]) => Option[T]): Serde[T] = ??? } |
Not all old names have name clash, but I think it worth to rename all implicits for consistency. Also, new Serdes will contain uuidSerde, shortSerde, javaShortSerde
and byteBufferSerde,
which are missing in the old org.apache.kafka.streams.scala.Serdes.
Note, we're not providing an implicit for timeWindowedSerde
because there's no way to automatically configure it properly (which requires knowing the window size). See KIP-659: Improve TimeWindowedDeserializer and TimeWindowedSerde to handle window size .
Migration Plan, Compatibility and Deprecation
...
An old org.apache.kafka.streams.scala.Serdes
will be marked as deprecated and could be deleted with the major 3.0 release.
Rejected Alternatives
Rename serdes in the old org.apache.kafka.streams.scala.Serdes
...