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

Guozhang Wang commented on KAFKA-7654:
--------------------------------------

[~backuitist] [~mjsax] There seems to have several discussion topics mingled 
together a bit. I'd like to share my thoughts separately:

1) {{Produced}} should not need serializer, and hence we should change the 
key/value serde to key/value serializer only; similarly {{Consumed}} should 
change its key/value serde to key/value deserializer only.

I admit that the deserializer / serializer are not needed for produced / 
consumed, and the current proposed API is because a) we want to have a 
consistent serde API on Serdes throughout the Streams library (I agree it's 
debatable which is better for API cleanness, consistency or succinctness; my 
personal preference is the former), and b) the assumption is that in practice, 
a single or very few serde classes will be used (Avro, Crice, etc) for which 
users will implement both ser and deser anyways, and hence it should not be 
much of a development burden. 

2) {{StreamsBuilder.table}} requires both a Consumed and a Materialized, and 
the serdes passed in Consumed can override the ones passed in Materialized. 
This is a bad API design.

I think I agree with this argument. There are some historical reasons that we 
accept a Consumed and Materialized, as we do not want to disallow users from 
serializing the object read from Kafka bytes into a different form of bytes 
when writing to the materialized state store, but later there are some 
optimization proposals to not deserialize / serialize the bytes read from Kafka 
at all but write them directly to the state store, and hence the overridden 
above. Thinking about this twice, I felt the serde benefits may not worth such 
overridden mess, and hence I'd like to propose changing the serde semantics as:

{{table(String)}} : default serde for reading from Kafka and writing to state 
store (if needed).

{{table(String, Consumed)}} : specified serde from Consumed for reading from 
Kafka and writing to state store (if needed).

{{table(String, Materialized)}} : specified serde from Consumed for reading 
from Kafka and writing to state store (if needed).

{{table(String, Consumed Materialized)}} : specified serde from Consumed for 
reading from Kafka and specified serde from Materialized writing to state 
store, which can be different (if needed).

3) The global configured serde is bad as it is not type safe, and hence we 
should consider removing it and always enforce users to specify the serdes on 
the streams / topology builder.

First of all note that even for JSON or Avro, the serdes can be type-safe as 
well, e.g. there is specificRecord and generalRecord in Avro serde, for 
specific record you indeed need to do a cast for deserializer (but from what I 
read at 
https://index.scala-lang.org/nequissimus/circe-kafka/circe-kafka/2.0.0?target=_2.12
 it seems circe is acting similarly?) Secondly, from my interactions with users 
of Streams, there are users who do want type-safety and those who do not. So I 
think we cannot simply say that we should enforce all users to embrace 
type-safety with our own taste.



> Relax requirements on serializing-only methods.
> -----------------------------------------------
>
>                 Key: KAFKA-7654
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7654
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>            Reporter: Bruno Bieth
>            Priority: Major
>
> Methods such as KStream#to shouldn't require a Produced as only the 
> serializing part is ever used.



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

Reply via email to