Thanks for the reply Daniyar, That makes much more sense! I thought I must be missing something, but I couldn't for the life of me figure it out.
What do you think about just taking an argument, instead of for a Comparator, for the Serde of the inner type? That way, the user can control how exactly the inner data gets serialized, while also bounding the generic parameter properly. As for the order, since the list is already in a specific order, which the user themselves controls, it doesn't seem strictly necessary to offer an option to sort the data during serialization. Thanks, -John On Mon, May 6, 2019 at 8:47 PM Development <d...@yeralin.net> wrote: > Hi John, > > I’m really sorry for the confusion. I cloned that JIRA ticket from an old > one about introducing UUID Serde, and I guess was too hasty while editing > the copy to notice the mistake. Just edited the ticket. Sorry for any > inconvenience . > > As per comparator, I agree. Let’s make user be responsible for > implementing comparable interface. I was just thinking to make the serde a > little more flexible (i.e. let user decide in which order records is going > to be inserted into a change log topic). > > Thank you! > > Best, > Daniyar Yeralin > > > > On May 6, 2019, at 5:37 PM, John Roesler <j...@confluent.io> wrote: > > > > Hi Daniyar, > > > > Thanks for the proposal! > > > > If I understand the point about the comparator, is it just to capture the > > generic type parameter? If so, then anything that implements a known > > interface would work just as well, right? I've been considering adding > > something like the Jackson TypeReference (or similar classes in many > other > > projects). Would this be a good time to do it? > > > > Note that it's not necessary to actually require that the captured type > is > > Comparable (as this proposal currently does), it's just a way to make > sure > > there is some method that makes use of the generic type parameter, to > force > > the compiler to capture the type. > > > > Just to make sure I understand the motivation... You expressed a desire > to > > be able to serialize UUIDs, which I didn't follow, since there is a > > built-in UUID serde: org.apache.kafka.common.serialization.Serdes#UUID, > and > > also, a UUID isn't a List. Did you mean that you need to use *lists of* > > UUIDs? > > > > Thanks, > > -John > > > > On Mon, May 6, 2019 at 11:49 AM Development <d...@yeralin.net> wrote: > > > >> Hello, > >> > >> Starting a discussion for KIP-466 adding support for List Serde. PR is > >> created under https://github.com/apache/kafka/pull/6592 < > >> https://github.com/apache/kafka/pull/6592> > >> > >> There are two topics I would like to discuss: > >> 1. Since type for List serve needs to be declared before hand, I could > not > >> create a static method for List Serde under > >> org.apache.kafka.common.serialization.Serdes. I addressed it in the KIP: > >> P.S. Static method corresponding to ListSerde under > >> org.apache.kafka.common.serialization.Serdes (something like static > public > >> Serde<List<T>> List() {...} > inorg.apache.kafka.common.serialization.Serdes) > >> class cannot be added because type needs to be defined beforehand. > That's > >> why one needs to create List Serde in the following fashion: > >> new Serdes.ListSerde<String>(Serdes.String(), > >> Comparator.comparing(String::length)); > >> (can possibly be simplified by declaring import static > >> org.apache.kafka.common.serialization.Serdes.ListSerde) > >> > >> 2. @miguno Michael G. Noll <https://github.com/miguno> is questioning > >> whether I need to pass a comparator to ListDeserializer. This certainly > is > >> not required. Feel free to add your input: > >> https://github.com/apache/kafka/pull/6592#discussion_r281152067 > >> > >> Thank you! > >> > >> Best, > >> Daniyar Yeralin > >> > >>> On May 6, 2019, at 11:59 AM, Daniyar Yeralin (JIRA) <j...@apache.org> > >> wrote: > >>> > >>> Daniyar Yeralin created KAFKA-8326: > >>> -------------------------------------- > >>> > >>> Summary: Add List<T> Serde > >>> Key: KAFKA-8326 > >>> URL: https://issues.apache.org/jira/browse/KAFKA-8326 > >>> Project: Kafka > >>> Issue Type: Improvement > >>> Components: clients, streams > >>> Reporter: Daniyar Yeralin > >>> > >>> > >>> I propose adding serializers and deserializers for the java.util.List > >> class. > >>> > >>> I have many use cases where I want to set the key of a Kafka message to > >> be a UUID. Currently, I need to turn UUIDs into strings or byte arrays > and > >> use their associated Serdes, but it would be more convenient to > serialize > >> and deserialize UUIDs directly. > >>> > >>> I believe there are many use cases where one would want to have a List > >> serde. Ex. [ > >> > https://stackoverflow.com/questions/41427174/aggregate-java-objects-in-a-list-with-kafka-streams-dsl-windows > ], > >> [ > >> > https://stackoverflow.com/questions/46365884/issue-with-arraylist-serde-in-kafka-streams-api > >> ] > >>> > >>> > >>> > >>> KIP Link: [ > >> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-466%3A+Add+support+for+List%3CT%3E+serialization+and+deserialization > >> ] > >>> > >>> > >>> > >>> -- > >>> This message was sent by Atlassian JIRA > >>> (v7.6.3#76005) > >> > >> > >