Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-08-06 Thread John Roesler
Thanks, Yuriy, That sounds good to me. -John On Wed, Aug 5, 2020, at 00:24, Yuriy Badalyantc wrote: > Hi John! > > I understand your point. But, I'm still not sure that having a Unit serde > is a good idea because it could be ambiguous for the user. In a KStream > [Long, Unit] valid value

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-08-05 Thread Yuriy Badalyantc
Hi John! I understand your point. But, I'm still not sure that having a Unit serde is a good idea because it could be ambiguous for the user. In a KStream [Long, Unit] valid value could mean one the following: 1. Empty byte array 2. null 3. Any value (value ignored) As a user of a library, I

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-11 Thread John Roesler
Thanks, Yuriy, IIRC, I added VoidSerde because there are times when the key or value is always null, but since ‘null’ has no type in Java, we had used the ByteArraySerde or some other proxy. This is confusing, because then the type is ‘byte[]’ although we intended it to always be null. It also

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Yuriy Badalyantc
Ok, I mentioned adding missing serdes in the Proposed Change paragraph. About VoidSerde. I didn't add it intentionally. The semantic of the Unit (scala's void) type is not clear in terms of the data. If kafka topic contains messages of type Unit, what does it actually means? That there is always

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Matthias J. Sax
Thanks Yuriy! What about `VoidSerde` ? It's not listed. It might also be nice to add a short sentence and state that in addition to fixing the name collisions, the KIP will also close the gap of out-of-the-box serdes and add missing Serdes that are offered in Java to Scala. -Matthias On

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Yuriy Badalyantc
Oh, ok. I have done that. Just didn't know that it was necessary. -Yuriy On Fri, Jul 10, 2020 at 9:30 PM John Roesler wrote: > Ah, thanks Yuriy, > > Sorry if this wasn't clear, but _all_ public API changes have to > be explicitly included in the KIP. Can you just enumerate all > the contents

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread John Roesler
Ah, thanks Yuriy, Sorry if this wasn't clear, but _all_ public API changes have to be explicitly included in the KIP. Can you just enumerate all the contents of the new API? Thanks, John On Fri, Jul 10, 2020, at 04:54, Yuriy Badalyantc wrote: > Hi, Matthias, > > It's not directly mentioned in

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-10 Thread Yuriy Badalyantc
Hi, Matthias, It's not directly mentioned in the KIP, but I added all missing Java serdes. I mentioned it in the pull request description: https://github.com/apache/kafka/pull/8955 And also, this KIP originally was based on a pull request where I added missing java serdes :)

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-09 Thread Matthias J. Sax
Yuriy, thanks for the KIP update. I have one follow up thought: I checked what default Serdes we offer in the Java class `org.apache.kafka.common.serialization.Serdes` and I think it would be good if we could close the gap between the Java and Scala code and add the missing Java Serdes in

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-08 Thread John Roesler
Hi Yuriy, Once it seems like there’s general agreement in the discussion, you can start a voting thread. You can find examples on the mailing list of what to say in the first message. It’s basically just a message with the subject line changed from “[DISCUSS]...” to “[VOTE]...”, and then

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-07 Thread Yuriy Badalyantc
So, what's next? It's my first KIP and I'm not familiar with all processes. -Yuriy On Mon, Jul 6, 2020 at 1:32 AM John Roesler wrote: > Hi Yuriy, > > Thanks for the update! It looks good to me. > > Thanks, > John > > On Sun, Jul 5, 2020, at 03:27, Yuriy Badalyantc wrote: > > Hi John. > > > > I

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-05 Thread John Roesler
Hi Yuriy, Thanks for the update! It looks good to me. Thanks, John On Sun, Jul 5, 2020, at 03:27, Yuriy Badalyantc wrote: > Hi John. > > I updated the KIP. An old proposed implementation is now in the rejected > alternatives. > > - Yuriy > > On Sun, Jul 5, 2020 at 12:03 AM John Roesler

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-05 Thread Yuriy Badalyantc
Hi John. I updated the KIP. An old proposed implementation is now in the rejected alternatives. - Yuriy On Sun, Jul 5, 2020 at 12:03 AM John Roesler wrote: > Hi Yuriy, > > I agree, we can keep them separate. I just wanted to make you aware of it. > > Thanks for the PR, it looks the way I

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-07-04 Thread John Roesler
Hi Yuriy, I agree, we can keep them separate. I just wanted to make you aware of it. Thanks for the PR, it looks the way I expected. I just read over the KIP document again. I think it needs to be updated to the current proposal, and then we’ll be able to start the vote. Thanks, John On

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-06-30 Thread Yuriy Badalyantc
Hi everybody! Looks like a discussion about KIP-513 could take a while. I think we should move forward with KIP-616 without waiting for KIP-513. I created a new pull request for KIP-616: https://github.com/apache/kafka/pull/8955. It contains a new

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-06-02 Thread Yuriy Badalyantc
Hi, John Thanks for pointing that out. I expressed my thoughts about KIP-513 and its connection to KIP-616 in the KIP-513 mail list. - Yuriy On Sun, May 31, 2020 at 1:26 AM John Roesler wrote: > Hi Yuriy, > > I was just looking back at KIP-513, and I’m wondering if there’s any > overlap we

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-30 Thread John Roesler
Hi Yuriy, I was just looking back at KIP-513, and I’m wondering if there’s any overlap we should consider here, or if they are just orthogonal. Thanks, -John On Thu, May 28, 2020, at 21:36, Yuriy Badalyantc wrote: > At the current moment, I think John's plan is better than the original plan >

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-28 Thread Yuriy Badalyantc
At the current moment, I think John's plan is better than the original plan described in the KIP. I think we should create a new `Serdes` in another package. The old one will be deprecated. - Yuriy On Fri, May 29, 2020 at 8:58 AM John Roesler wrote: > Thanks, Matthias, > > If we go with the

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-28 Thread John Roesler
Thanks, Matthias, If we go with the approach Yuriy and I agreed on, to deprecate and replace the whole class and not just a few of the methods, then the timeline is less of a concern. Under that plan, Yuriy can just write the new class exactly the way he wants and people can cleanly swap over

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-28 Thread Matthias J. Sax
I am not a Scale person, so I cannot really contribute much. However for the deprecation period, if we get the change into 2.7, it might be ok to remove the deprecated classed in 3.0. It would only be one minor release in between what is a little bit short (we usually prefer at least two minor

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-28 Thread John Roesler
Hi Yuriy, Sounds good to me! I had a feeling we were bringing different context to the discussion; thanks for sticking with the conversation until we got it hashed out. I'm glad you prefer Serde*s*, since having multiple different classes with the same name leads to all kinds of trouble.

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-28 Thread Yuriy Badalyantc
Ok, I understood you, John. I wasn't sure about kafka deprecation policy and thought that the full cycle could be done with 2.7 version. Waiting for 3.0 is too much, I agree with it. So, I think creating one more `Serdes` in another package is our way. I suggest one of the following: 1.

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-27 Thread John Roesler
Hi Yuriy, Thanks for the clarification. I guess my concern is twofold: 1. We typically leave deprecated methods in place for at least a major release cycle before removing them, so it would seem abrupt to have a deprecation period of only one minor release. If we follow the same pattern here,

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-27 Thread Yuriy Badalyantc
Hi John, I'm stick with the `org.apache.kafka.streams.scala.Serdes` because it's sort of conventional in the scala community. If you have a typeclass `Foo`, you probably will search `Foo` related stuff in the `Foo` or maybe `Foos` (plural). All other places are far less discoverable for the

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-27 Thread John Roesler
Hi Yuriy, Thanks for the reply. I guess I've been out of the Scala game for a while; all this summoner business is totally new to me. I think I followed the rationale you provided, but I still don't see why you can't implement your whole plan in a new class. What is special about the existing

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-19 Thread Yuriy Badalyantc
Hi John, Your suggestion looks interesting. I think it's technically doable. But I'm not sure that this is the better solution. I will try to explain. From the scala developers' perspective, `Serde` looks really like a typeclass. Typical typeclass in pure scala will look like this: ``` trait

Re: [DISCUSS] KIP-616: Rename implicit Serdes instances in kafka-streams-scala

2020-05-18 Thread John Roesler
Hi Yuriy, Thanks so much for the KIP! I didn’t anticipate the problem you laid out in the KIP, but I find it very plausible. Thanks for pushing back on the “convention” and raising the issue, and also volunteering a solution! I’m wondering if we can “fix” it in one shot by just deprecating