Thanks, Matthias. KIP is updated with the APIs.
On Tue, 15 Mar 2022 at 23:28, Matthias J. Sax wrote:
> Thanks.
>
> It would be good to add the concrete interfaces of the new classed to
> the KIP, ie,
>
> - FixedKeyProcessorSupplier
> - FixedKeyProcessor
> - FixedKeyProcessorContext
> -
Thanks.
It would be good to add the concrete interfaces of the new classed to
the KIP, ie,
- FixedKeyProcessorSupplier
- FixedKeyProcessor
- FixedKeyProcessorContext
- FixedKeyRecord
-Matthias
On 3/10/22 3:15 PM, Jorge Esteban Quilcate Otoya wrote:
Thanks all!
I agree with Matthias
Thanks all!
I agree with Matthias and Jon on going forward with the new
`FixedKeyRecord` approach.
The KIP has been updated accordingly.
Feel free to add your vote or amend on the vote thread if needed.
Cheers,
On Mon, 7 Mar 2022 at 21:57, Matthias J. Sax wrote:
> I think it's ok that we
I think it's ok that we cannot prevent users from mutating a given
read-only object. We have similar issues "all over the place" in the
API, because it's just how Java works unfortunately (eg,
`ValueMapperWithKey` and similar interfaces).
The point being is, that the API clearly expresses
Thanks John! I feel a bit ashamed of just thinking loud here without trying
out prototypes myself :P
I think the FixedKeyProcessor/Record looks very good -- like you said,
since we are making a new set of APIs then why don't we reconsider more
bolderly -- but at the same time I'd also like to
Thanks John! This is great!
I guess I was aware the pulling the `ValueRecord` string might move more
parts than I mentioned. Happy to see you being supportive of the idea.
-Matthias
On 3/7/22 10:31 AM, Jorge Esteban Quilcate Otoya wrote:
Thanks, John!
This looks very promising.
I will
Thanks, John!
This looks very promising.
I will familiarize this approach and update the KIP accordingly. From what
I can see so far, this should cover most of the open issues in this
proposal.
PS.
> Just as a reminder, the current approach with transformers
> is NOT enforced at compile time.
Hello all,
It seems like we're making good progress on this discussion.
If I'm keeping track correctly, if we can resolve this
question about how to handle processValues(), then we should
be able to finalize the vote, right?
I share Matthias's preference for having a type-safe API.
Just as a
Matthias, thanks for your feedback.
I can see the following alternatives to deal with `processValues()`:
1. Runtime key validation (current proposal)
2. Using Void type. Guozhang already points out some important
considerations about allocating `Record` twice.
3. Adding a new ValueRecord,
John, thanks for verifying source compatibility. My impression was that
it should be source compatible, I was just not 100% sure.
The question about `processValues()` is really a hard one. Guozhang's
point is very good one. Maybe we need to be pragmatic and accept the
runtime check (even if I
Thanks, Guozhang.
> Compared with reference checks and runtime exceptions for those who
> mistakenly change the key, I think that enforcing everyone to `setValue`
> may incur more costs..
This is a fair point. I agree that this may incur in more costs than key
checking.
Will hold for more
Hello folks,
Regarding the outstanding question, I'm actually a bit leaning towards the
second option since that `withKey()` itself always creates a new Record
object. This has a few implications:
* That we would have to discard the previous Record object to be GC'ed with
the new object --- note
Hi all,
Appreciate very much all the great feedback received so far.
> After applying that interface change, I don't see any syntax
errors in our tests (which use those methods), and the
StreamBuilderTest still passes for me.
This is awesome John, thank you for your efforts here.
> Jorge, do
Oh, one more thing Jorge,
I think the other outstanding question for you is whether
the output key type for processValues should be K or Void. I
get the impression that all of us don't feel too strongly
about it, so I think the ball is in your court to consider
everyone's points and make a call
Hello all,
I'll chime in again in the interest of trying to do a better
job of keeping KIPs moving forward...
Matthias raised some very good questions about whether the
change is really source compatible. I just checked out the
code and make the interface change that Jorge specified in
the KIP:
For this KIP, I also see the value. I was just trying to make a step
back and ask if it's a good short term solution. If we believe it is, I
am fine with it.
(I am more worried about the header's KIP...)
Btw: I am still wondering if we can change existing `process()` as
proposed in the KIP?
Hi folks,
I agree with John that this KIP by itself could be a good improvement, and
I feel it aligns well with the eventual DSL 2.0 proposal so we do not need
to hold it until later.
Regarding the last point (i.e. whether we should do enforcement with a new
interface), here's my 2c: in the past
Hello all,
While I sympathize with Matthias’s desire to wipe the slate clean and redesign
the dsl with full knowledge of everything we’ve learned in the past few years,
that would also be a pretty intense project on its own. It seems better to
leave that project for someone who is motivated to
On Fri, 18 Feb 2022 at 02:16, Matthias J. Sax wrote:
> > It probably deserves its own thread to start discussing ideas.
>
> Yes. My question was: if we think it's time to do a DSL 2.0, should we
> drop this KIP and just fix via DSL 2.0 instead?
>
>
Good question. Would love to hear what others
It probably deserves its own thread to start discussing ideas.
Yes. My question was: if we think it's time to do a DSL 2.0, should we
drop this KIP and just fix via DSL 2.0 instead?
You're right. I'm not proposing the method signature.
What signature do you propose? I don't see an update
Regarding the last question Matthias had, I wonder if it's similar to my
first email's point 2) above? I think the rationale is that, since
reference checks are relatively very cheap, it is worthwhile to pay this
extra runtime checks and in return to have a single consolidated
ProcessorSupplier
Thank you Matthias, this is great feedback.
Adding my comments below.
On Wed, 16 Feb 2022 at 00:42, Matthias J. Sax wrote:
> Thanks for the KIP.
>
> In alignment to my reply to KIP-634, I am wondering if we are heading
> into the right direction, or if we should consider to re-design the DSL
>
Thanks for the KIP.
In alignment to my reply to KIP-634, I am wondering if we are heading
into the right direction, or if we should consider to re-design the DSL
from scratch?
Even if we don't do a DSL 2.0 right now, I have some concerns about this
KIP:
(1) I am not sure if the propose
My apologies, this feedback was intended for KIP-634.
-John
On Tue, Feb 15, 2022, at 13:15, John Roesler wrote:
> Thanks for the update, Jorge,
>
> I've just looked over the KIP again. Just one more small
> concern:
>
> 5) We can't just change the type of Record#headers() to a
> new fully
Thanks for the update, Jorge,
I've just looked over the KIP again. Just one more small
concern:
5) We can't just change the type of Record#headers() to a
new fully qualified type. That would be a source-
incompatible breaking change for users.
Out options are:
* Deprecate the existing method
Thanks Paul, this is very helpful. Glad to know this will be potentially
helpful for your team moving forward.
Also, thanks everyone for your feedback so far. If there's no additional
questions about the current state of this KIP, I'll be starting a vote
thread soon.
Happy to keep discussing the
No specific comments, but I just wanted to mention I like the direction of
the KIP. My team is a big user of "transform" methods because of the
ability to chain them, and I have always found the terminology challenging
to explain alongside "process". It felt like one concept with two names.
So
Got it. Thanks John, this make sense.
I've updated the KIP to include the deprecation of:
- KStream#transform
- KStream#transformValues
- KStream#flatTransform
- KStream#flatTransformValues
On Fri, 11 Feb 2022 at 15:16, John Roesler wrote:
> Thanks, Jorge!
>
> I think it’ll be
Thanks, Jorge!
I think it’ll be better to keep this KIP focused on KStream methods only. I
suspect that the KTable methods may be more complicated than just that proposed
replacement, but it’ll also be easier to consider that question in isolation.
The nice thing about just deprecating the
Thanks, John.
> 4) I agree that we shouldn't deprecate the Transformer*
classes, but do you think we should deprecate the
KStream#transform* methods? I'm curious if there's any
remaining reason to have those methods, or if your KIP
completely obviates them.
Good catch.
I considered that
Thanks for the update, Jorge!
I just read over the KIP again, and I'm in support. One more
question came up for me, though:
4) I agree that we shouldn't deprecate the Transformer*
classes, but do you think we should deprecate the
KStream#transform* methods? I'm curious if there's any
remaining
Thank you both for your feedback!
I have added the following note on punctuation:
```
NOTE: The key validation can be defined when processing the message.
Though, with punctuations it won't be possible to define the key for
validation before forwarding, therefore it won't be possible to forward
I'm +1 on John's point 3) for punctuations.
And I think if people are on the same page that a reference equality check
per record is not a huge overhead, I think doing that enforcement is better
than documentations and hand-wavy undefined behaviors.
Guozhang
On Wed, Feb 9, 2022 at 11:27 AM
Thanks for the KIP Jorge,
I'm in support of your proposal.
1)
I do agree with Guozhang's point (1). I think the cleanest
approach. I think it's cleaner and better to keep the
enforcement internal to the framework than to introduce a
public API or context wrapper for processors to use
explicitly.
Thanks Guozhang.
> Does `ValueProcessorContext` have to be a public API? It seems to me
that this can be completely abstracted away from user interfaces as an
internal class
Totally agree. No intention to add these as public APIs. Will update the
KIP to reflect this.
> in the past the rationale
Hello Jorge,
Thanks for bringing this KIP! I think this is a nice idea to consider using
a single overloaded function name for #process, just a couple quick
questions after reading the proposal:
1) Does `ValueProcessorContext` have to be a public API? It seems to me
that this can be completely
Hi Dev team,
I'd like to start a new discussion thread on Kafka Streams KIP-820:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-820%3A+Extend+KStream+process+with+new+Processor+API
This KIP is aimed to extend the current `KStream#process` API to return
output values that could be chained
37 matches
Mail list logo