[ https://issues.apache.org/jira/browse/KAFKA-7654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16692570#comment-16692570 ]
Matthias J. Sax commented on KAFKA-7654: ---------------------------------------- Thanks for sharing your thoughts. I think it basically makes sense, however, there are couple of things to consider: {quote}then coming up with a deserializer for a non-trivial format is way too much work for something that's not going to be used {quote} For most cases, any type you use in KafkaStreams, most likely needs to be both, serialized and deserialized. I think it's only corner cases, for which one reads or writes a type only. Because of this assumption, the believe is, that all types will have Serdes anyway, and it should not be any extra work to create the serdes for `Consumed` (of course, there are always exception). {quote}So it's two-fold, first the API is puzzling (a `Produced` requiring a `Deserializer` is a smell to me) Any good serialization library actually separate the 2 concerns (serialization/deserialization) and acknowledge the fact that you don't always need both. {quote} I see your point here, and even with my argument from above (ie, not expected overhead as a serde is created anyway most likely), a source or sink could still just accept a serializer or deserializer. However, I want to point out, that in upcoming `2.1` we actually pass `Serdes` downstream. For example, if you read a topic as KStream and call `stream.groupByKey().reduce().toStream().to()`, you don't need to specify the serde on reduce() a second time, or on `to()` a third time, because the type did no change and the `builder.stream()` provided serde will be reused. Thus, if we change the API and only get a deserializer in `builder.stream()`, we cannot do this any longer what seems to be a step backward. {quote}The materialized is mutated to get the serdes from the consumed, which is counter-intuitive (I guess that's why it's explained in the documentation). I don't understand why the materialized need to have the exact same serde as the consumed used to read `topic`, but assuming there's a good reason there are ways to enforce this at the type level rather than overriding pre-existing values. {quote} The idea is, to avoid specifying the Serde twice (what would be redundant)—yes, this must be documented because the API allows two ways to specify serdes (admittedly not optimal). The data from the input topic is of the same type as the data stored in the table and thus, both Serdes (Consumed and Materialized) must be of the same type. The type system does enforce this, but if we would required that users specify the same Serde twice (on Consumed to read data from the topic and on Materialized to read/write data to/from the store) it would be error prone (assume the user only specified it once, the other one would fall back to the Serde from the StreamsConfig that is of a different type—otherwise, it would not be required to overwrite it in the first place). I agree, that for the `table()` example, the API offers two ways to specify Serde what is not optimal. However, because `Materialized` is used as parameter for other methods, too, it needs the ability to accept serdes as Materialized might be the only config parameter specified (we could have introduced a new class for both cases, but decided to accept the "hit" to avoid too many different config types—we have already many... — it's debatable if this was the best decision). For your suggestion about `KStream#to(String, Serializer<K>, Serializer)`—this was the API in older releases, but had the major drawback, that methods with a lot of optional parameters, resulted in a lot of overloads what confused a lot of users and was not easy to use. Thus, with [KIP-182|https://cwiki.apache.org/confluence/display/KAFKA/KIP-182%3A+Reduce+Streams+DSL+overloads+and+allow+easier+use+of+custom+storage+engines], we change the API to reduce the number of overloads: one with all mandatory arguments, and a second one with all mandatory argument plus a config parameter that takes all optional arguments and that follows a builder-like pattern (some method still have more then two overloads unfortunately: we tried to reuse config classes instead of creating more and more—again a design decision that is debatable). To have a unique API experience, we decided to use this pattern for all methods (also for those, with fewer optional parameters for which the change is not necessarily an improvement). A unique API experience seemed to be more valuable. Thus, while I see your point, there are many trade-offs to consider and I am not convinced atm, that your suggestion would improve the API from a holistic point of view. Let me know what you think about this. I just wanted to give you some context on why the API is design in the current way, and that changing something (even if it seems valid from an isolated point of view) impacts other parts of the API (thus an improvement for one operator might imply an negative impact on some other operator—it's all a trade-off). Maybe there is a good middle ground, that addressed your valid concerns, but still aligns with the current API design philosophy. Happy to discus this. If we can find a better way, to balance the difference concerns, I am happy to work on an improved API design. > 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)