Thanks Almog, I read https://github.com/apache/kafka/pull/15061/files
and I think the additional API is the right way of fixing it.
Guozhang
On Fri, Dec 22, 2023 at 9:14 AM Almog Gavra wrote:
>
> Hello Everyone! I updated the KIP once more as a result of a bug
> investigation - I added
Hello Everyone! I updated the KIP once more as a result of a bug
investigation - I added DslWindowParams#isTimestamped to the public API as
a result of https://issues.apache.org/jira/browse/KAFKA-16046. Please let
me know if there's any concerns with this addition.
On Thu, Dec 14, 2023 at 5:40 PM
Sorry for the late response to the late reply, hah! I didn't give any
thought about how we would want to integrate this custom store supplier
with querying of the stores. My initial intuition suggests that we'd
probably want a separate API for that, or just recommend people to query
their external
Hey Almog,
Sorry for the late reply.
Re: 2) above, maybe I'm just overthinking it. What I had in mind is
that when we have, say, a remote store impl customized by the users.
Besides being used inside the KS app itself, the user may try to
access the store instance outside the KS app as well? If
Hello Guozhang,
Thanks for the feedback! For 1 there are tests verifying this and I did so
manually as well, it does not reveal anything about the store types -- just
the names, so I think we're good there. I've put an example at the bottom
of this reply for people following the conversation.
Hello Almog,
I left a comment in the PR before I got to read the newest updates
from this thread. My 2c:
1. I liked the idea of delaying the instantiation of StoreBuiler from
suppliers after the Topology is created. It has been a bit annoying
for many other features we were trying back then. The
Hello Everyone - one more minor change to the KIP that came up during
implementation (reflected in the KIP itself). I will be adding the method
below to TopologyConfig. This allows us to determine whether or not the
DslStoreSuppliers was explicitly passed in via either
Thanks. Will take a look into the PR.
I don't have any objection to the goal; contrary! It's very annoying
what we have right now, and if we can improve it, I am totally in favor
of it.
-Matthias
On 11/3/23 8:47 AM, Almog Gavra wrote:
Good question :) I have a PR for it already here:
Good question :) I have a PR for it already here:
https://github.com/apache/kafka/pull/14659. The concept is to wrap the
suppliers with an interface that allows for delayed creation of the
StoreBuilder instead of creating the StoreBuilder from the suppliers right
away. Happy to get on a quick call
Almog,
can you explain how you intent to implement this change? It's not clear
to me, how we could do this?
When we call `StreasmBuilder.build()` it will give us a already fully
wired `Topology`, including all store suppliers needed. I don't see a
clean way how we could change the store
Hello everyone - I updated the KIP to also include the following
modification:
Both the new dsl.store.suppliers.class and the old default.dsl.store will
now respect the configurations when passed in via KafkaStreams#new(Topology,
StreamsConfig) (and other related constructors) instead of only
OK! I think I got everything, but I'll give the KIP another read with fresh
eyes later. Just a reminder that the voting is open, so go out and exercise
your civic duty! ;)
- Almog
On Fri, Jul 28, 2023 at 10:38 AM Almog Gavra wrote:
> Thanks Guozhang & Sophie:
>
> A2. Will clarify in the KIP
>
Thanks Guozhang & Sophie:
A2. Will clarify in the KIP
A3. Will change back to the deprecated version!
A5. Seems like I'm outnumbered... DslStoreSuppliers it is.
Will update the KIP today.
- Almog
On Thu, Jul 27, 2023 at 12:42 PM Guozhang Wang
wrote:
> Yes, that sounds right to me. Thanks
Yes, that sounds right to me. Thanks Sophie.
On Thu, Jul 27, 2023 at 12:35 PM Sophie Blee-Goldman
wrote:
>
> A2: Guozhang, just to close the book on the ListValue store thing, I fully
> agree it seems like overreach
> to expose/force this on users, especially if it's fully internal today. But
>
A2: Guozhang, just to close the book on the ListValue store thing, I fully
agree it seems like overreach
to expose/force this on users, especially if it's fully internal today. But
just to make sure we're on the same
page here, you're still ok with this KIP fixing the API gap that exists
today, in
Hi all,
Like Almog's secretary as well! Just following up on that index:
A2: I'm also happy without introducing versioned KV in this KIP as I
would envision it to be introduced as new params into the
KeyValuePluginParams in the future. And just to clarify on Sophie's
previous comment, I think
Hi,
A5. I have to admit that
"If we envision extending this beyond just StoreSupplier types, it could
be a good option."
is scaring me a bit.
I am wondering what would be an example for such an extension?
In general, I would propose to limit the scope of a config. In this case
the config
Thanks for the feedback Bruno -- sounds like we're getting close to a final
consensus here.
It sounds like the two main (only?) semi-unresolved questions that still
have differing
opinions floating around are whether to deprecate the old config, and what
to name the new config
+ interface.
Thanks for the comments Bruno!
A3. Oops... I think I didn't do a great job updating the KIP to reflect
Guozhang's suggestion. This seems like the last point of contention, where
we have two options:
1. Deprecate the config entirely and replace IN_MEMORY/ROCKSDB with
implementations of the
Hi,
Sorry for being late to the party!
A1: I agree with Sophie, Guozhang, and Almog not to block the KIP on
gaps in the implementation.
A2: I am happy with not considering anything special w.r.t. versioned
state stores in this KIP.
A3: Here I agree with Sophie to deprecate the old config.
I have updated the KIP with the points as discussed above. @Guozhang, the
suggested configuration makes it a little more awkward around the
Materialized.as and Materialized.withStoreType APIs than it was when we
were totally deprecating the old configuration. Let me know what you think.
I will
Glad you like my KIP-secretary skills ;)
A2. I'm definitely happy to take your suggestion here and not do anything
special w.r.t. Versioned stores, I think it makes sense especially if we
consider them implementation details of a specific store type.
At EOD I'll update the KIP with all of these
Awesome summary (seriously) -- would you kindly offer your organizational
skills to every ongoing KIP from henceforth? We need you :P
A few answers/comments:
A2: I think there is a 3rd sub-option here, which is to leave
versioned-ness out of this KIP entirely, return only the non-versioned
Lots of thoughts! Happy to see the thriving discussion on this post - lots
going on so I'm trying to enumerate them to keep things organized (prefix
"A" for "Almog" so we can use numbers in responses for other things ;P).
A1. Question around closing implementation gaps (e.g. no rocks based
Guozhang:
On your 2nd point:
> "impl types" (in hindsight it may not be a good name) for rocksdb /
memory / custom, and we used "store types" for kv / windowed / sessioned
etc,
First off, thanks so much for this clarification -- using "store type" here
was definitely making me uncomfortable as
Matthias:
I'm not sure I agree with (or maybe don't follow) this take:
>
> we need all kind of `StoreTypeSpec` implementations,
> and it might also imply that we need follow up KIPs for new feature
> (like in-memory versioned store) that might not need a KIP otherwise.
>
I see this feature as
Thanks everyone for the great discussions so far! I first saw the JIRA
and left some quick thoughts without being aware of the
already-written KIP (kudos to Almog, very great one) and the DISCUSS
thread here. And I happily find some of my initial thoughts align with
the KIP already :)
Would like
Thanks for all the input. My intention was not to block the KIP, but
just to take a step back and try get a holistic picture and discussion,
to explore if there are good/viable alternative designs. As said
originally, I really like to close this gap, and was always aware that
the current
I agree with everything Almog said above, and will just add on to two
points:
1. Regarding whether to block this KIP on the completion of any or all
future implementations of in-memory version stores (or persist suppression
buffers), I feel that would be unfair to this feature which is completely
Thanks for all the feedback folk! Responses inline.
> Basically, I'm suggesting two things: first, call out in some way
(perhaps the StoreTypeSpec javadocs) that each StoreTypeSpec is considered
a public contract in itself and should outline any semantic guarantees it
does, or does not, make.
Thanks for the KIP. Overall I like the idea to close this gap.
However, I am wondering if we should close others gaps first? In
particular, IIRC, we have a few cases for which we only have a RocksDB
implementation for a store, and thus, adding an in-memory version for
these stores first, to
Sophie—
Thanks for chiming in here. +1 to the idea of specifying the ordering
guarantees that we make in the StorageTypeSpec javadocs.
Quick question then. Is the javadoc that says:
> Order is not guaranteed as bytes lexicographical ordering might not
represent key order.
no longer correct,
Hey Almog, first off, thanks for the KIP! I (and others) raised concerns
over how restrictive the default.dsl.store config would be if not
extendable to custom store types, especially given that this seems to be
the primary userbase of such a feature. At the time we didn't really have
any better
Hi All,
I would like to propose a KIP to expand support for default store types
(KIP-591) to encompass custom store implementations:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-954%3A+expand+default+DSL+store+configuration+to+custom+types
Looking forward to your feedback!
Cheers,
34 matches
Mail list logo