Hi everybody, I just want to ping this discussion. Any thoughts?
- Yuriy On Fri, Jun 5, 2020 at 10:22 AM Yuriy Badalyantc <lmne...@gmail.com> wrote: > Hi Matthias, > > I think you misunderstood my diagram. I wanted to show the next > relationships: > - Serde is KeySerde and ValueSerde > - KeySerde is KeySerializer and KeyDeserializer > - ValueSerde is ValueSerializer and ValueDeserializer > > So, `Serde` is is most "powerfull" class. It could do anything. `KeySerde` > and `ValueSerde` are not specialized `Serde`, but a component of a `Serde`. > > Implementation of these relationships through inheritance will look like > this (pseudocode): > class KeySerializer > class KeyDeserializer > class ValueSerializer > class ValueDeserializer > class KeySerde extends KeySerializer with KeyDeserializer > class ValueSerde extends ValueSerializer with ValueDeserializer > class Serde extends KeySerde with ValueSerde > > So, everywhere where you need, for example, `KeyDeserializer`, you could > use `KeyDeserializer`, `KeySerde` or `Serde`. > > -Yuriy > > On Fri, Jun 5, 2020 at 7:10 AM Matthias J. Sax <mj...@apache.org> wrote: > >> As you say "hierarchy" I read this really as "class hierarchy". For this >> case, I think that we need to do it differently. >> >> I agree to this part: >> >> KeySerde extends Serde >> ValueSerde extends Serde >> >> However, >> >> KeySerializer extends KeySerde (etc) >> >> does not make sense IMHO, because a `KeySerializer` is no a special >> KeySerde; it's only a serializer but it's not a deserializer. >> >> In fact the hierarchy goes the other direction: >> >> Serde extends Serializerd, Deserializer >> >> >> Atm, a Serde is just a "container" for serializers and desrialzier though. >> >> >> -Matthias >> >> >> >> On 6/2/20 10:40 PM, Yuriy Badalyantc wrote: >> > Hi. >> > >> > I'm the author of the KIP-616. I got acquainted with the KIP-513 and I >> > think overall it's a good idea. But I think workaround only on the scala >> > side is not enough. We could consider moving a bit further and change >> serde >> > hierarchy on the java side to something like this: >> > >> > KeySerializer-----↘ >> > KeySerde---↘ >> > KeyDeserializer---↗ ↓ >> > Serde >> > ValueSerializer---↘ ↑ >> > ValueSerde-↗ >> > ValueDeserializer-↗ >> > >> > I understand that this is a bit too revolutionary, but I think this is >> the >> > right direction. >> > >> > Regarding KIP-616 and KIP-513 connection. It's hard to say at the >> current >> > moment how should we implement these kips: together or separately. It >> looks >> > like there is no consensus on the implementation details for these kips. >> > >> > If we decided to create a new `KeySerde` and `ValueSerde` on the scala >> side >> > (current plan without my proposal above), we could use their companions >> for >> > default instances. In this case, it looks like we don't need to do >> KIP-616. >> > But what about backward compatibility? What should we do with >> > `org.apache.kafka.streams.scala.Serdes`? Deprecate it? >> > >> > - Yuriy >> > >> > On Sun, May 31, 2020 at 1:24 AM John Roesler <vvcep...@apache.org> >> wrote: >> > >> >> Hi Mykhailo, >> >> >> >> Wow, I really dropped the ball here. I have just looked over your KIP >> >> again, and now I see how you don’t need to change every dsl method, >> only >> >> Consumed, Materialized, etc. >> >> >> >> I think this would be a good addition. Yuriy has just proposed KIP-616 >> to >> >> fix some other problems with the implicit serdes. I’m wondering if >> these >> >> two kips have any joint opportunities we should consider, or if it’s >> better >> >> to continue to consider them separately. >> >> >> >> Thanks, >> >> John >> >> >> >> On Wed, Jan 22, 2020, at 16:18, Михаил Ерёменко wrote: >> >>> Hi, John! >> >>> >> >>> Sorry for the late reply. I am not really familiar with this mail list >> >>> discussions, so I have not seen your mails. >> >>> >> >>> Regarding your question: >> >>>> I guess what >> >>> I'm struggling with is why you actually want to have different key >> and >> >>> serdes for the same type >> >>> >> >>> I think good example will be (and it is actually what we do in ours >> >>> project) using confluent schema registry in conjunction with kafka >> >>> streams. Some models can be used as keys as well as values. When we >> >>> define schema registry compatible serde, we have to specify is it for >> >>> key or not. We can of course create two serdes for the same model, but >> >>> in this case implicit semantic will not work because scala doesn’t >> know >> >>> which implicit to pick. And things become even more complicated in >> case >> >>> if you will try to derive your serdes (we derive serdes in ours >> >>> project). >> >>> >> >>> One more thing: >> >>>> every method in the streams-scala DSL. >> >>> >> >>> So far we've just changed >> >>> org.apache.kafka.streams.scala.ImplicitConversions and >> >>> org.apache.kafka.streams.scala.kstream.Materialized and it works for >> >>> us. Also we did introduce default serdes for primitive types. >> >>> >> >>> Regards, >> >>> Mykhailo >> >> >> > >> >>