Thanks, Matthias, and thanks again for raising the concern.
-John
On Mon, Jul 29, 2019 at 4:58 PM Matthias J. Sax
wrote:
> Thanks for the details!
>
> Also talked to Guozhang about a potential upgrade path. This KIP seems
> not to put us into an bad position to provide a clean upgrade path if we
Thanks for the details!
Also talked to Guozhang about a potential upgrade path. This KIP seems
not to put us into an bad position to provide a clean upgrade path if we
change the `ProcessorContext` in the future.
Thus, I think we can move forward.
-Matthias
On 7/24/19 3:32 PM, John Roesler wro
Hey again Matthias,
I think it might help to frame the evaluation of the Context question if we
have a "spitball" proposal for what change we might want to make to the
context.
Currently, the ProcessorContext is referenced in the following public
interfaces:
org.apache.kafka.streams.errors.Deser
Hey Matthias,
I agree, it's worth double-checking to make sure that the upgrade path
would be smooth. There's no point in putting ourselves in an awkward jam.
I'll look into it and report back.
Regarding the global store logic, I confirmed that the "state update
processor" shouldn't be forwarding
If we don't fix the `ProcessorContext` now, how would an upgrade path
look like?
We woudl deprecate existing `init()` and add a new `init()`, and during
runtime need to call both? This sound rather error prone to me and might
be confusing to users? Hence, it might be beneficial to fix it right now
Hey Matthias,
Thanks for the review!
I agree about ProcessorContext, it could certainly be split up to improve
compile-time clues about what is or is not permitted (like, do you just
want to be able to see the extra record context vs. forawrding vs.
registering state stores, as you said). But, si
I have concerns about the latest proposal from Guozhang. However, as
John said it's beyond the scope of this KIP and thus, I don't go into
details. I agree thought, that the current "transformer APIs" are not
ideal and could be improved.
An orthogonal though is that we should split the current
`P
Sounds good to me, thanks John!
Guozhang
On Wed, Jul 24, 2019 at 7:40 AM John Roesler wrote:
> Hey Guozhang,
>
> Thanks for the thought! It sounds related to what I was thinking in
> https://issues.apache.org/jira/browse/KAFKA-8396 , but a little "extra"...
>
> I proposed to eliminate ValueTra
Hey Guozhang,
Thanks for the thought! It sounds related to what I was thinking in
https://issues.apache.org/jira/browse/KAFKA-8396 , but a little "extra"...
I proposed to eliminate ValueTransformer, but I believe you're right; we
could eliminate Transformer also and just use Processor in the tran
Hi John,
Just a wild thought about Transformer: now with the new Processor#init(ProcessorContext), do we still need a
Transformer (and even ValueTransformer / ValueTransformerWithKey)?
What if:
* We just make KStream#transform to get a ProcessorSupplier as well, and
inside `process()` we check t
Hi again, all,
I have started the voting thread. Please cast your votes (or voice
your objections)! The vote will remain open at least 72 hours. Once it
closes, I can send the PR pretty quickly.
Thanks for all you help ironing out the details on this feature.
-John
On Mon, Jul 15, 2019 at 5:09 P
Hey all,
It sounds like there's general agreement now on this KIP, so I updated
the KIP to fit in with Guozhang's overall proposed package structure.
Specifically, the proposed name for the new Processor interface is
"org.apache.kafka.streams.processor.api.Processor".
If there are no objections,
Side remark:
> Now that "flat transform" is a specific
>> part of the API it seems okay to steer folks in that direction (to never
>> use context.process in a transformer), but it should be called out
>> explicitly in javadocs. Currently Transformer (which is used for both
>> transform() and flat
Ah, good catch!
I didn't mean to include the for Transform itself. I must
have just glossed over it when I was writing the KIP.
It would apply to TransformValues, since forwarding is disabled. But
for Transform, it should be bounded to the result type.
The Transformer interface actually present
First of all, +1 on the whole idea, my team has run into (admittedly minor,
but definitely annoying) issues because of the weaker typing. We're heavy
users of the PAPI and have Processors that, while not hundreds of lines
long, are certainly quite hefty and call context.forward() in many places.
Sorry for coming late to the party.
As for the naming I'm in favor of RecordProcessor as well.
I agree that we should not take on doing all of the package movements as
part of this KIP, especially as John has pointed out, it will be an
opportunity to discuss some clean-up on individual classes wh
Hi John,
Yeah I think we should not do all the repackaging as part of this KIP as
well (we can just do the movement of the Processor / ProcessorSupplier),
but I think we need to discuss the end goal here since otherwise we may do
the repackaging of Processor in this KIP, but only later on realizin
Hey Guozhang,
Thanks for the idea! I'm wondering if we could take a middle ground
and take your proposed layout as a "roadmap", while only actually
moving the classes that are already involved in this KIP.
The reason I ask is not just to control the scope of this KIP, but
also, I think that if we
Hello John,
Thanks for your detailed explanation, I've done some quick checks on some
existing examples that heavily used Processor and the results also makes me
worried about my previous statements that "the breakage would not be big".
I agree we should maintain compatibility.
About the naming i
I think `RecordProcessor` is a good name.
-Matthias
On 6/21/19 5:09 PM, John Roesler wrote:
> After kicking the naming around a bit more, it seems like any package
> name change is a bit "weird" because it fragments the package and
> directory structure. If we can come up with a reasonable name
After kicking the naming around a bit more, it seems like any package
name change is a bit "weird" because it fragments the package and
directory structure. If we can come up with a reasonable name for the
interface after all, it seems like the better choice.
The real challenge is that the existin
Hi all,
I've updated the KIP with the feedback so far.
The naming question is still the biggest (only?) outstanding issue. It
would be good to hear some more thoughts on it.
As we stand now, there's one vote for changing the package name to
something like 'typedprocessor', one for changing the i
Thanks for the feedback, Guozhang and Matthias,
Regarding motivation: I'll update the wiki. Briefly:
* Any processor can benefit. Imagine a pure user of the ProcessorAPI
who has very complex processing logic. I have seen several processor
implementation that are hundreds of lines long and call
`co
Just want to second what Sophie said about the stores. The type of a
used stores is completely independent of input/output types.
This related to change `addGlobalStore()` method. Why do you want to pin
the types? In fact, people request the ability to filter() and maybe
even map() the data before
Hi John,
Thanks for KIP! I've a few comments below:
1. So far the "Motivation" section is very general, and the only concrete
example that I have in mind is `TransformValues#punctuate`. Do we have any
other concrete issues that drive this KIP? If not then I feel better to
narrow the scope of this
Hi all,
In response to the feedback so far, I changed the package name from
`processor2` to `processor.generic`.
Thanks,
-John
On Mon, Jun 17, 2019 at 4:49 PM John Roesler wrote:
>
> Thanks for the feedback, Sophie!
>
> I actually felt a little uneasy when I wrote that remark, because it's
> no
Thanks for the feedback, Sophie!
I actually felt a little uneasy when I wrote that remark, because it's
not restricted at all in the API, it's just available to you if you
choose to give your stores and context the same parameters. So, I
think your use case is valid, and also perfectly permissable
Hey John, thanks for writing this up! I like the proposal but there's one
point that I think may be too restrictive:
"A processor that happens to use a typed store is actually emitting the
same types that it is storing."
I can imagine someone could want to leverage this new type safety without
al
Hi all,
I'd like to propose KIP-478 (https://cwiki.apache.org/confluence/x/2SkLBw).
This proposal would add output type bounds to the Processor interface
in Kafka Streams, which enables static checking of a number of useful
properties:
* A processor B that consumes the output of processor A is ac
29 matches
Mail list logo