Status
Current state: Under Discussion
Discussion thread: TODO
JIRA: TODO
Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).
Motivation
Currently org.apache.kafka.streams.scala.Serdes
contains implicit Serde
instances. And some of these implicits are named the same as its Serde
type. For example:
implicit def Long: Serde[Long] = ???
This naming leads to a name clash after wildcard import:
import org.apache.kafka.streams.scala.Serdes._ val x = String.valueOf(5) // Error:(7, 18) value valueOf is not a member of org.apache.kafka.common.serialization.Serde[String] println(x)
Because wildcard import is the way for getting implicits in the current scope, there is no way to use these implicits without possible naming issues.
Personally, I faced this issue on the first day of using kafka-streams-scala library. And many of my colleagues faced it too.
This KIP was created as a result of this pull request.
Proposed Change
The only way to make it work without naming clash is to rename these implicits. In scala implicits are usually named like these:
implicit def longSerde: Serde[Long] = ???
Changed public Interfaces
New serde names should looks like these:
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 floatSerde: Serde[Float] = ??? implicit def javaFloatSerde: Serde[java.lang.Float] = ??? implicit def doubleSerde: Serde[Double] = ??? implicit def javaDoubleSerde: Serde[java.lang.Double] = ??? implicit def integerSerde: Serde[Int] = ??? implicit def javaIntegerSerde: Serde[java.lang.Integer] = ???
Not all old names have name clash, but I think it worth to rename all implicits for consistency.
Migration Plan, Compatibility and Deprecation
These changes are not binary and code compatible with the current 2.5 version. Also, users could use these implicits by full its full name (org.apache.kafka.streams.scala.Serdes.String
). It means that we should pass these changes through deprecation cycle. I propose the next plan:
First release:
- Rename instances.
- Near with the renamed instances (in the
org.apache.kafka.streams.scala.Serdes
) add serdes with old naming, but withoutimplicit
modifier. These serdes will point to the new implicits. These step will allow to us to maintain compatibility for users, who point to the old serdes by its full name. - Mark new serdes without
implicit
modifier as deprecated.
It will looks like these (example for the single serde):
implicit def longSerde: Serde[Long] = ??? @deprecated def Long: Serde[Long] = longSerde
- Second release:
- Remove deprecated serdes.
The original problem will be solved only after the second release.