[ https://issues.apache.org/jira/browse/KAFKA-7654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16692959#comment-16692959 ]
Bruno Bieth commented on KAFKA-7654: ------------------------------------ bq. However, I want to point out, that in upcoming `2.1` we actually pass `Serdes` downstream. This can be achieved by having 2 different APIs (using the fluent style), while still retaining the possibility of passing only what's really needed. bq. 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 I agree, same type, but not same instance (i.e format). bq. the other one would fall back to the Serde from the StreamsConfig that is of a different type That's your root cause. {code:java} public interface ProcessorContext { /** * Returns the default key serde * * @return the key serializer */ Serde<?> keySerde(); /** * Returns the default value serde * * @return the value serializer */ Serde<?> valueSerde(); {code} Having untyped `Serde`s opens the door to lots of runtime errors as you end-up casting all over the place: {code:java} (Serde<K>) context.keySerde() {code} So you need to remember that some of that casting is ultimately going to happen and defensively guard against such incorrect casts. This leaks to the API in unfortunate ways. I would suggest to remove the default serdes. bq. `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 In this specific case, the serializers shouldn't be optional, but I think I understand your point, you want a convenient API that's easy to navigate. Back to my suggestion (a fluent API with 2 paths), this may be achieved with something like this: {code:java} class Builder { /** although a `KVDeser` is only needed, one can already pass in a `KVSerde` that's going to be used further down the chain **/ SerdeStream<K,V> stream<K,V>(KVSerde<K,V> serde) DeserStream<K,V> stream<K,V>(KVDeser<K,V> deser) } /** Carries a serde */ class SerdeStream<K,V> { void to(topicName: String) /** changes the format to `ser` */ void to(topicName: String, ser: KVSer<K,V>) } /** Carries a deserializer */ class DeserStream<K,V> { // Note that `#to(String)` isn't available as we DON'T have defaults void to(topicName: String, ser: KVSer<K,V>) } {code} > 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)