[
https://issues.apache.org/jira/browse/KAFKA-5245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16049511#comment-16049511
]
Guozhang Wang commented on KAFKA-5245:
--------------------------------------
Here is my take on the serde issues:
We currently follow the pattern that "there is a global default serde specified
in config; and whenever users want to override them, they need to do that
specifically"; the suggested pattern which is "inheriting the overridden serde
from the source, i.e. builder.stream/table to the downstream operators". I'm
not sure which one is more intuitive, given that whenever you have some
operators in between like "map", the serdes cannot be inherited any more and
users are still required to specify them. So that by changing this pattern we
cannot actually remove those overloads that requires serdes, and users need to
have a deeper understanding on "when do I need to override the serde, and when
I do not need to", this seems trickier for normal users than the somewhat-dumb
but easy-to-understand pattern we have now. We have also considered another way
of handling serdes long time ago, to infer data types with registered serdes so
that users would "NEVER" need to specify the serde along with the topology, but
unfortunately due to Java type erasure this is not 100 percent feasible:
https://cwiki.apache.org/confluence/display/KAFKA/Discussion%3A+Serialization+and+Deserialization+Options
That being said, for the specific issue that Yeva rasied, I think for
{{groupByKey}} and {{selectKey}}, it is indeed counter-intuitive to even
require a serde, only because we may need to repartition since the key has
changed, and hence needing the serde to read / write to Kafka. That is, we are
sort of exposing the internal implementation of the DSL that we want to hide
from the users in its interface. I do not have a better solution on top of my
head for this specific issue since again we cannot always inherit the serde
from the source stream if there is some operators in between that changed the
key / value types, like {{map}}.
> KStream builder should capture serdes
> --------------------------------------
>
> Key: KAFKA-5245
> URL: https://issues.apache.org/jira/browse/KAFKA-5245
> Project: Kafka
> Issue Type: Improvement
> Components: streams
> Affects Versions: 0.10.2.0, 0.10.2.1
> Reporter: Yeva Byzek
> Assignee: anugrah
> Priority: Minor
> Labels: beginner, newbie
>
> Even if one specifies a serdes in `builder.stream`, later a call to
> `groupByKey` may require the serdes again if it differs from the configured
> streams app serdes. The preferred behavior is that if no serdes is provided
> to `groupByKey`, it should use whatever was provided in `builder.stream` and
> not what was in the app.
> From the current docs:
> “When to set explicit serdes: Variants of groupByKey exist to override the
> configured default serdes of your application, which you must do if the key
> and/or value types of the resulting KGroupedStream do not match the
> configured default serdes.”
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)