[ 
https://issues.apache.org/jira/browse/KAFKA-7386?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16611367#comment-16611367
 ] 

ASF GitHub Bot commented on KAFKA-7386:
---------------------------------------

guozhangwang closed pull request #5622: KAFKA-7386: streams-scala should not 
cache serdes
URL: https://github.com/apache/kafka/pull/5622
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala
 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala
index 8bfb083909b..02e5380e710 100644
--- 
a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala
+++ 
b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala
@@ -25,17 +25,17 @@ import org.apache.kafka.common.serialization.{Deserializer, 
Serde, Serializer, S
 import org.apache.kafka.streams.kstream.WindowedSerdes
 
 object Serdes {
-  implicit val String: Serde[String] = JSerdes.String()
-  implicit val Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]]
-  implicit val JavaLong: Serde[java.lang.Long] = JSerdes.Long()
-  implicit val ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray()
-  implicit val Bytes: Serde[org.apache.kafka.common.utils.Bytes] = 
JSerdes.Bytes()
-  implicit val Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]]
-  implicit val JavaFloat: Serde[java.lang.Float] = JSerdes.Float()
-  implicit val Double: Serde[Double] = 
JSerdes.Double().asInstanceOf[Serde[Double]]
-  implicit val JavaDouble: Serde[java.lang.Double] = JSerdes.Double()
-  implicit val Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]]
-  implicit val JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer()
+  implicit def String: Serde[String] = JSerdes.String()
+  implicit def Long: Serde[Long] = JSerdes.Long().asInstanceOf[Serde[Long]]
+  implicit def JavaLong: Serde[java.lang.Long] = JSerdes.Long()
+  implicit def ByteArray: Serde[Array[Byte]] = JSerdes.ByteArray()
+  implicit def Bytes: Serde[org.apache.kafka.common.utils.Bytes] = 
JSerdes.Bytes()
+  implicit def Float: Serde[Float] = JSerdes.Float().asInstanceOf[Serde[Float]]
+  implicit def JavaFloat: Serde[java.lang.Float] = JSerdes.Float()
+  implicit def Double: Serde[Double] = 
JSerdes.Double().asInstanceOf[Serde[Double]]
+  implicit def JavaDouble: Serde[java.lang.Double] = JSerdes.Double()
+  implicit def Integer: Serde[Int] = JSerdes.Integer().asInstanceOf[Serde[Int]]
+  implicit def JavaInteger: Serde[java.lang.Integer] = JSerdes.Integer()
 
   implicit def timeWindowedSerde[T]: WindowedSerdes.TimeWindowedSerde[T] = new 
WindowedSerdes.TimeWindowedSerde[T]()
   implicit def sessionWindowedSerde[T]: WindowedSerdes.SessionWindowedSerde[T] 
=


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Streams Scala wrapper should not cache serdes
> ---------------------------------------------
>
>                 Key: KAFKA-7386
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7386
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 2.0.0
>            Reporter: John Roesler
>            Assignee: John Roesler
>            Priority: Major
>             Fix For: 2.0.1, 2.1.0
>
>
> for example, 
> [https://github.com/apache/kafka/blob/trunk/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/Serdes.scala#L28]
>  invokes Serdes.String() once and caches the result.
> However, the implementation of the String serde has a non-empty configure 
> method that is variant in whether it's used as a key or value serde. So we 
> won't get correct execution if we create one serde and use it for both keys 
> and values.
> The fix is simple: change all the `val` declarations in scala.Serdes to 
> `def`. Thanks to the referential transparency for parameterless methods in 
> scala, no user-facing code will break.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to