Bump

> On May 24, 2019, at 2:09 PM, Development <d...@yeralin.net> wrote:
> 
> Hey,
> 
> - did we consider to make the return type (ie, ArrayList, vs
> LinkesList) configurable or encode it the serialized bytes?
> 
> Not sure about this one. Could you elaborate?
> 
> - atm the size of each element is encoded individually; did we consider
> an optimization for fixed size elements (like Long) to avoid this overhead?
> 
> I cannot think of any clean way to do so. How would you see it?
> 
> Btw I resolved all your comments under PR
> 
> Best,
> Daniyar Yeralin
> 
>> On May 24, 2019, at 12:01 AM, Matthias J. Sax <matth...@confluent.io> wrote:
>> 
>> Thanks for the KIP. I also had a look into the PR and have two follow up
>> question:
>> 
>> 
>> - did we consider to make the return type (ie, ArrayList, vs
>> LinkesList) configurable or encode it the serialized bytes?
>> 
>> - atm the size of each element is encoded individually; did we consider
>> an optimization for fixed size elements (like Long) to avoid this overhead?
>> 
>> 
>> 
>> -Matthias
>> 
>> On 5/15/19 6:05 PM, John Roesler wrote:
>>> Sounds good!
>>> 
>>> On Tue, May 14, 2019 at 9:21 AM Development <d...@yeralin.net> wrote:
>>>> 
>>>> Hey,
>>>> 
>>>> I think it the proposal is finalized, no one raised any concerns. Shall we 
>>>> call it for a [VOTE]?
>>>> 
>>>> Best,
>>>> Daniyar Yeralin
>>>> 
>>>>> On May 10, 2019, at 10:17 AM, John Roesler <j...@confluent.io> wrote:
>>>>> 
>>>>> Good observation, Daniyar.
>>>>> 
>>>>> Maybe we should just not implement support for serdeFrom.
>>>>> 
>>>>> We can always add it later, but I think you're right, we need some
>>>>> kind of more sophisticated support, or at least a second argument for
>>>>> the inner class.
>>>>> 
>>>>> For now, it seems like most use cases would be satisfied without
>>>>> serdeFrom(...List...)
>>>>> 
>>>>> -John
>>>>> 
>>>>> On Fri, May 10, 2019 at 8:57 AM Development <d...@yeralin.net> wrote:
>>>>>> 
>>>>>> Hi,
>>>>>> 
>>>>>> I was trying to add some test cases for the list serde, and it led me to 
>>>>>> this class `org.apache.kafka.common.serialization.SerializationTest`. I 
>>>>>> saw that it relies on method 
>>>>>> `org.apache.kafka.common.serialization.serdeFrom(Class<T> type)`
>>>>>> 
>>>>>> Now, I’m not sure how to adapt List<T> serde for this method, since it 
>>>>>> will be a “nested class”. What is the best approach in this case?
>>>>>> 
>>>>>> I remember that in Jackson for example, one uses a TypeFactory, and 
>>>>>> constructs “collectionType” of two classes. For example, 
>>>>>> `constructCollectionType(List.class, String.class).getClass()`. I don’t 
>>>>>> think it applies here.
>>>>>> 
>>>>>> Any ideas?
>>>>>> 
>>>>>> Best,
>>>>>> Daniyar Yeralin
>>>>>> 
>>>>>>> On May 9, 2019, at 2:10 PM, Development <d...@yeralin.net> wrote:
>>>>>>> 
>>>>>>> Hey Sophie,
>>>>>>> 
>>>>>>> Thank you for your input. I think I’d rather finish this KIP as is, and 
>>>>>>> then open a new one for the Collections (if everyone agrees). I don’t 
>>>>>>> want to extend the current KIP-466, since most of the work is already 
>>>>>>> done for it.
>>>>>>> 
>>>>>>> Meanwhile, I’ll start adding some test cases for this new list serde 
>>>>>>> since this discussion seems to be approaching its logical end.
>>>>>>> 
>>>>>>> Best,
>>>>>>> Daniyar Yeralin
>>>>>>> 
>>>>>>>> On May 9, 2019, at 1:35 PM, Sophie Blee-Goldman <sop...@confluent.io> 
>>>>>>>> wrote:
>>>>>>>> 
>>>>>>>> Good point about serdes for other Collections. On the one hand I'd 
>>>>>>>> guess
>>>>>>>> that non-List Collections are probably relatively rare in practice (if
>>>>>>>> anyone disagrees please correct me!) but on the other hand, a) even if 
>>>>>>>> just
>>>>>>>> a small number of people benefit I think it's worth the extra effort 
>>>>>>>> and b)
>>>>>>>> if we do end up needing/wanting them in the future it would save us a 
>>>>>>>> KIP
>>>>>>>> to just add them now. Personally I feel it would make sense to expand 
>>>>>>>> the
>>>>>>>> scope of this KIP a bit to include all Collections as a logical unit, 
>>>>>>>> but
>>>>>>>> the ROI could be low..
>>>>>>>> 
>>>>>>>> (I know of at least one instance in the unit tests where a Set serde 
>>>>>>>> could
>>>>>>>> be useful, and there may be more)
>>>>>>>> 
>>>>>>>> On Thu, May 9, 2019 at 7:27 AM Development <d...@yeralin.net> wrote:
>>>>>>>> 
>>>>>>>>> Hey,
>>>>>>>>> 
>>>>>>>>> I don’t see any replies. Seems like this proposal can be finalized and
>>>>>>>>> called for a vote?
>>>>>>>>> 
>>>>>>>>> Also I’ve been thinking. Do we need more serdes for other Collections?
>>>>>>>>> Like queue or set for example
>>>>>>>>> 
>>>>>>>>> Best,
>>>>>>>>> Daniyar Yeralin
>>>>>>>>> 
>>>>>>>>>> On May 8, 2019, at 2:28 PM, John Roesler <j...@confluent.io> wrote:
>>>>>>>>>> 
>>>>>>>>>> Hi Daniyar,
>>>>>>>>>> 
>>>>>>>>>> No worries about the procedural stuff. Prior experience with KIPs is
>>>>>>>>>> not required :)
>>>>>>>>>> 
>>>>>>>>>> I was just trying to help you propose this stuff in a way that the
>>>>>>>>>> others will find easy to review.
>>>>>>>>>> 
>>>>>>>>>> Thanks for updating the KIP. Thanks to the others for helping out 
>>>>>>>>>> with
>>>>>>>>>> the syntax.
>>>>>>>>>> 
>>>>>>>>>> Given these updates, I'm curious if anyone else has feedback about
>>>>>>>>>> this proposal. Personally, I think it sounds fine!
>>>>>>>>>> 
>>>>>>>>>> -John
>>>>>>>>>> 
>>>>>>>>>> On Wed, May 8, 2019 at 1:01 PM Development <d...@yeralin.net> wrote:
>>>>>>>>>>> 
>>>>>>>>>>> Hey,
>>>>>>>>>>> 
>>>>>>>>>>> That worked! I certainly lack Java generics knowledge. Thanks for 
>>>>>>>>>>> the
>>>>>>>>> snippet. I’ll update KIP again.
>>>>>>>>>>> 
>>>>>>>>>>> Best,
>>>>>>>>>>> Daniyar Yeralin
>>>>>>>>>>> 
>>>>>>>>>>>> On May 8, 2019, at 1:39 PM, Chris Egerton <chr...@confluent.io> 
>>>>>>>>>>>> wrote:
>>>>>>>>>>>> 
>>>>>>>>>>>> Hi Daniyar,
>>>>>>>>>>>> 
>>>>>>>>>>>> I think you may want to tweak your syntax a little:
>>>>>>>>>>>> 
>>>>>>>>>>>> public static <T> Serde<List<T>> List(Serde<T> innerSerde) {
>>>>>>>>>>>> return new ListSerde<T>(innerSerde);
>>>>>>>>>>>> }
>>>>>>>>>>>> 
>>>>>>>>>>>> Does that work?
>>>>>>>>>>>> 
>>>>>>>>>>>> Cheers,
>>>>>>>>>>>> 
>>>>>>>>>>>> Chris
>>>>>>>>>>>> 
>>>>>>>>>>>> On Wed, May 8, 2019 at 10:29 AM Development <d...@yeralin.net 
>>>>>>>>>>>> <mailto:
>>>>>>>>> d...@yeralin.net>> wrote:
>>>>>>>>>>>> Hi John,
>>>>>>>>>>>> 
>>>>>>>>>>>> I updated JIRA and KIP.
>>>>>>>>>>>> 
>>>>>>>>>>>> I didn’t know about the process, and created PR before I knew about
>>>>>>>>> KIPs :)
>>>>>>>>>>>> 
>>>>>>>>>>>> As per static declaration, I don’t think Java allows that:
>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>>> Best,
>>>>>>>>>>>> Daniyar Yeralin
>>>>>>>>>>>> 
>>>>>>>>>>>>> On May 7, 2019, at 2:22 PM, John Roesler <j...@confluent.io 
>>>>>>>>>>>>> <mailto:
>>>>>>>>> j...@confluent.io>> wrote:
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thanks for that update. Do you mind making changes primarily on 
>>>>>>>>>>>>> the
>>>>>>>>>>>>> KIP document ? (
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-466%3A+Add+support+for+List%3CT%3E+serialization+and+deserialization
>>>>>>>>> <
>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-466%3A+Add+support+for+List%3CT%3E+serialization+and+deserialization
>>>>>>>>>> )
>>>>>>>>>>>>> 
>>>>>>>>>>>>> This is the design document that we have to agree on and vote 
>>>>>>>>>>>>> for, the
>>>>>>>>>>>>> PR comes later. It can be nice to have an implementation to look 
>>>>>>>>>>>>> at,
>>>>>>>>>>>>> but the KIP is the main artifact for this discussion.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> With this in mind, it will help get more reviewers to look at it 
>>>>>>>>>>>>> if
>>>>>>>>>>>>> you can tidy up the KIP document so that it stands on its own. 
>>>>>>>>>>>>> People
>>>>>>>>>>>>> shouldn't have to look at any other document to understand the
>>>>>>>>>>>>> motivation of the proposal, and they shouldn't have to look at a 
>>>>>>>>>>>>> PR to
>>>>>>>>>>>>> see what the public API will look like. If it helps, you can take 
>>>>>>>>>>>>> a
>>>>>>>>>>>>> look at some other recent KIPs.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Given that the list serde needs an inner serde, I agree you can't 
>>>>>>>>>>>>> have
>>>>>>>>>>>>> a zero-argument static factory method for it, but it seems you 
>>>>>>>>>>>>> could
>>>>>>>>>>>>> still have a static method:
>>>>>>>>>>>>> `public static Serde<List<T>> List(Serde<T> innerSerde)`.
>>>>>>>>>>>>> 
>>>>>>>>>>>>> Thoughts?
>>>>>>>>>>>>> 
>>>>>>>>>>>>> On Tue, May 7, 2019 at 12:18 PM Development <d...@yeralin.net 
>>>>>>>>>>>>> <mailto:
>>>>>>>>> d...@yeralin.net>> wrote:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Absolutely agree. Already pushed another commit to remove 
>>>>>>>>>>>>>> comparator
>>>>>>>>> argument: https://github.com/apache/kafka/pull/6592 <
>>>>>>>>> https://github.com/apache/kafka/pull/6592> <
>>>>>>>>> https://github.com/apache/kafka/pull/6592 <
>>>>>>>>> https://github.com/apache/kafka/pull/6592>>
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> Thank you for your input John! I really appreciate it.
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> What about this point I made:
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 1. Since type for List serde 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)
>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> On May 7, 2019, at 11:50 AM, John Roesler <j...@confluent.io
>>>>>>>>> <mailto:j...@confluent.io>> wrote:
>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>> 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
>>>>>>>>> <mailto: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
>>>>>>>>> <mailto: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
>>>>>>>>> <mailto: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> <
>>>>>>>>>>>>>>>>>> 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 <
>>>>>>>>> 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
>>>>>>>>> <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 <mailto: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 <
>>>>>>>>> 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/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
>>>>>>>>> <
>>>>>>>>> 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
>>>>>>>>> <
>>>>>>>>> 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)
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>>>> 
>>>>>>>>>>>>>> 
>>>>>>>>>>>> 
>>>>>>>>>>> 
>>>>>>>>> 
>>>>>>>>> 
>>>>>>> 
>>>>>> 
>>>> 
>> 
> 

Reply via email to