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

Bruno Bieth commented on KAFKA-7654:
------------------------------------

Hi,

[~bbejeck] this is future proofing and in the meantime the API forces us to 
provide things that aren't used.

[~mjsax] if you (really) need a `StreamsPartitioner` then you can create 
another class that wraps a `StreamsPartitioner` and a pair of `Serializer`s.

The concrete issue that I'm facing is that I'm forced to provide something that 
isn't used.
So it's two-fold, first the API is puzzling (a `Produced` requiring a 
`Deserializer` is a smell to me), 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. Any good serialization library actually separate the 2 concerns 
(serialization/deserialization) and acknowledge the fact that you don't always 
need both.

I would suggest 2 possible changes:
1) a stop-gap overload: `KStream#to(String, Serializer<K>, Serializer<V>)`

2) fix the API in backward incompatible ways - change `Produced` and `Consumed` 
so that they respectively need `Serializer`s and `Deserializer`s only. For the 
cases where you need a `Serde` (say for a `Materialized`) then you'll have to 
change the API a bit further. Let's take a look at this:

{code:java}
    public synchronized <K, V> KTable<K, V> table(final String topic,
                                                  final Consumed<K, V> consumed,
                                                  final Materialized<K, V, 
KeyValueStore<Bytes, byte[]>> materialized) {
        // ...
        
materialized.withKeySerde(consumedInternal.keySerde()).withValueSerde(consumedInternal.valueSerde());
 // <---- HERE ----
        // ...
        return internalStreamsBuilder.table(topic, consumedInternal, 
materializedInternal);
    }
{code}

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. Once this is done then consumed is only used for it's deserializers.
I believe the same reasoning can be applied throughout the whole API (`stream`, 
`through`, `to`) where serializers and deserializers are conflated. I can help 
more if you're willing to go down that road :)

> 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