Thanks Paul!
On 8/15/19 7:28 PM, Paul Whalen wrote:
> I updated the KIP (and PR) to relax the restriction on connecting state
> stores via either means; it definitely makes sense to me at this point.
> I'd love to hear if there are any other concerns or broad objections to the
> KIP.
>
> Paul
>
I updated the KIP (and PR) to relax the restriction on connecting state
stores via either means; it definitely makes sense to me at this point.
I'd love to hear if there are any other concerns or broad objections to the
KIP.
Paul
On Thu, Aug 8, 2019 at 10:12 PM Paul Whalen wrote:
> Matthias,
>
Matthias,
You did summarize my thinking correctly, thanks for writing it out. I
think the disconnect on opinion is due to a couple things influenced by my
habits while writing streams code:
1) I don't see state stores that are "individually owned" versus "shared"
as that much different at all, a
I am not sure if I full understand, hence, I try to rephrase:
> I can't think of an example that would require both ways, or would
> even be more readable using both ways.
Example:
There are two processor A and B, and one store S that both need to
access and one store S_b that only B needs to ac
My thinking on restricting the API to enforce only one way of connecting
stores would make it more simple to use and end up with more readable
code. I can't think of an example that would require both ways, or would
even be more readable using both ways. Shared stores can be passed from
the outsi
Sorry for the long silence on this KIP Paul! I guess the 2.3 release
distracted us somewhat.
Overall, I am +1.
With regard to John's point about owned vs shared state stores, I think
it describe a valid use case, and throwing an exception if people want
to mix both features might be too restricti
Hey, all,
Sorry I'm late to the party. I meant to read into this KIP before, but
didn't get around to it. I was just reminded when Paul mentioned it in
a different thread. Please feel free to bump a discussion any time it
stalls!
I've just read through the whole discussion so far, and, to echo th
It wasn't much of a lift changing option B to work for option C, so I
closed that PR and made a new one, which should be identical to the KIP
right now: https://github.com/apache/kafka/pull/6824. There are a few
todos still which I will hold off until the KIP is accepted.
I created a voting threa
Per Matthias's suggestion from a while ago, I actually implemented a good
amount of option B to get a sense of the user experience and documentation
requirements. For a few reasons mentioned below, I think it's not my
favorite option, and I prefer option C. But since I did the work and it
can hel
I think the discussion mixed approaches a little bit, hence, let me
rephrase my understanding:
A) add new method with default implementation to `ProcessorSupplier`:
For this case, we don't add a new interface, but only add a new method
to `ProcessorSupplier` -- to keep backward compatibility, we
Hi Paul,
I will try to express myself a bit clearer.
Ad 1)
My assumption is that if `StateStoreConnector#stateStores()` returns `null`
Kafka Streams will throw an NPE because on purpose no null check is
performed before the loop that calls `StreamsBuilder#addStateStore()`. When
the user finally u
I definitely don't mind anyone jumping, Bruno, thanks for the comments!
1) I'm not totally sure I'm clear on your point, but I think we're on the
same page - if we're adding a method to the XSupplier interfaces (by making
them inherit from a super interface StateStoreConnector) then we definitely
Hi,
@Paul: Thank you for the KIP!
I hope you do not mind that I jump in.
I have the following comments:
1) `null` vs empty list in the default implementation
IIUC, returning `null` in the default implementation should basically
signal that the method `stateStores` was not overridden. Why then p
Thank Paul!
I agree with all of that. If we think that the general design is good,
refactoring a PR if we want to pick a different name should not be too
much additional work (hopefully). Thus, if you want to open a WIP PR and
we use it to nail the open details, it might help to find a good conclu
Great thoughts Matthias, thanks! I think we're all agreed that naming and
documentation/education are the biggest hurdles for this KIP, and in light
of that, I think it makes sense for me to just take a stab at a full
fledged PR with documentation to convince us that it's possible to do it
with eno
Hi, sorry for the long pause. Just trying to catch up here.
I think it save to allow `addStateStore()` to be idempotent for the same
`StoreBuilder` object. In fact, the `name` is "hard coded" and thus it's
not really possible to use the same `StoreBuilder` object to create
different stores.
I als
Ivan and Guozhang,
Thanks for the thoughts! Ivan's use case is definitely interesting. The
way I see it, if we can achieve the main goal of the KIP (allowing
Processor/TransformerSuppliers to encapsulate their usage of state stores),
we will enable this kind of thing in "user space" very easily.
Hi all!
I was about to write another KIP, but found out that KIP-401 addresses
exactly the problem I faced. So let me jump into your discussion and ask
you to assess another idea.
I fully agree with the KIP-401's motivation part. E. g in my project I
had to invent a wrapper class that hides
Hello Paul,
Thanks for the uploaded PR and the detailed description! I've made a pass
on it and left some comments.
Overall I think I agree with you that passing in the storebuilder directly
that store name is more convienent as it does not require another
`addStore` call, but we just need to spe
I'd like to resurrect this discussion with a cursory, proof-of-concept
implementation of the KIP which combines many of our ideas:
https://github.com/apache/kafka/pull/6496. I tried to keep the diff as
small as possible for now, just using it to convey the main ideas. But
I'll separately address
Matthias / Paul,
The concern I had about introducing `StoreBuilderSupplier` is simply
because it is another XXSupplier to the public API, so I'd like to ask if
we really have to add it :)
The difference between encapsulating the store name and encapsulating the
full state store builder is that, i
Thanks for the great thoughts Matthias and Guozhang!
If I'm not mistaken, Guozhang's suggestion is what my second alternative on
the KIP is ("Have the added method on the Supplier interfaces only return
store names, not builders"). I do think it would be a worthwhile usability
improvement on its
Guozhang,
>> Regarding the last option to catch "store exist already" exception and
>> fallback to connect stores, I'm a bit concerned it may be hiding actual
>> user bugs.
I agree with this concern. From my original email:
> The only disadvantage I see, might be
> potential bugs about sharing s
Matthias,
Thanks for your feedbacks.
Regarding the last option to catch "store exist already" exception and
fallback to connect stores, I'm a bit concerned it may be hiding actual
user bugs.
Thinking about Paul's proposal and your suggestion again, I'd like to
propose another alternative somewhe
Just a meta comment: do we really need to deprecate existing
`transform()` etc methods?
The last argument is a vararg, and thus, just keeping the existing API
for this part seems to work too, allowing to implement both patterns?
Also, instead of adding a default method, we could also add a new
in
Ah yes of course, this was an oversight, I completely ignored the multiple
processors sharing the same state store when writing up the KIP. Which is
funny, because I've actually done this (different processors sharing state
stores) a fair amount myself, and I've settled on a pattern where I group
Hello Paul,
Thanks for the great writeup (very detailed and crystal motivation
sections!).
This is quite an interesting idea and I do like the API cleanness you
proposed. The original motivation of letting StreamsTopology to add state
stores though, is to allow different processors to share the s
Here's KIP-401 for discussion, a minor Kafka Streams API change that I
think could greatly increase the usability of the low-level processor API.
I have some code written but will wait to see if there is buy in before
going all out and creating a pull request. It seems like most of the work
would
28 matches
Mail list logo