Awesome. Thanks to you both - KIP updated appropriately.
On 11 May 2018 at 11:52, Matthias J. Sax wrote:
> Sounds good to me to do it in a different KIP (if at all---not convinced
> yet we need this restriction).
>
>
> -Matthias
>
>
>
> On 5/11/18 11:36 AM, Guozhang Wang wrote:
> > Matthias' poi
Sounds good to me to do it in a different KIP (if at all---not convinced
yet we need this restriction).
-Matthias
On 5/11/18 11:36 AM, Guozhang Wang wrote:
> Matthias' point is valid, that today we are not effectively preventing
> users to programmatically access any materialized state store a
Matthias' point is valid, that today we are not effectively preventing
users to programmatically access any materialized state store and mutate it
on the fly anyways. This is a caveat that we should improve since as Andy
asked, the implications of mutating a materialized store is "undefined" as
of
I'm no expert, so happy to go with what ever is decided.
Implementing it so that a call to getStateStore(queryableStoreName) throws
an exception is trivial enough to do, though I can see your point that this
might be better done in a more consistent / single place, rather than
peppered through the
I don't think it worth to complicate things and restrict the access to
the store that is created by the library via `Materialized` parameter.
The reason is, that we don't prevent this from happening anyway atm --
users can always create an arbitrary processor and connect a KTable
state with the pro
Yes I was indeed thinking about now allowing `getStateStore()` to with the name
specified in Materialized. I understand it is a bit too restrictive, but I
cannot think of a elegant way to work around the following:
1) to programmatically enforce that the restore store is read-only.
2) With Materi
Thinking that through more - I guess if the user wanted the output to also
be fed back in as the input on future computations, they wouldn't use the
transformValues() overload with Materialized, but rather create a
materialized store and pass it in via stateStoreNames.
On 11 May 2018 at 10:42, And
Just a thought - but on the subject of disallowing access to the
materialized state store from within the transformer's init method... might
this not be overly restrictive? Could there be valid uses where read-only
access would be useful / valid.
On 11 May 2018 at 10:35, Andy Coates wrote:
> OK,
OK, KIP updated:
- added overloads taking `Materialized`
- dropped overloads taking `ValueTransformerSupplier` in favour of the
`withKey` variants.
- added more info around the limitations of the ProcessorContext passed in
to the transformer's init calls, i.e. no forward calls allowed or calls t
I'm a +1 for Guozhang's suggestion
On Fri, 11 May 2018 at 10:20 Andy Coates wrote:
> Makes sense to me. What do others think?
>
> On 11 May 2018 at 10:13, Guozhang Wang wrote:
>
> > Hi folks,
> >
> > While looking into the overloaded functions, I'm wondering if we can save
> > the transformers
Makes sense to me. What do others think?
On 11 May 2018 at 10:13, Guozhang Wang wrote:
> Hi folks,
>
> While looking into the overloaded functions, I'm wondering if we can save
> the transformers without key, i.e. only add two overloaded functions:
>
> KTable transformValues(final ValueTransfo
With reference to Guozhang's comment:
Then are we going to still only allow the valueTransofmer.init() /
process() to be able to access N stores, with N stores specified with the
stateStoreNames, but not the one specified in materialized.name()?
Personally I think it should be the case as the mate
Hi folks,
While looking into the overloaded functions, I'm wondering if we can save
the transformers without key, i.e. only add two overloaded functions:
KTable transformValues(final ValueTransformerWithKeySupplier valueTransformerSupplier,
final String... stat
Sorry for my lack of response - I've been out of action with a bad back for
a few days!
I originally had the `Materialized` overloads added to the API. I'll update
the KIP / PR with these again. In terms of semantics, as Matthias suggests,
these should be consistent with filter() and mapValues(),
To follow on Matthias and Damian's comments here:
If we are going to add the overload functions as
```
KTable transformValues(final ValueTransformerSupplier valueTransformerSupplier,
final String... stateStoreNames,
final Mate
Initially i thought materializing a store would be overkill, but from a
consistency point of view it makes sense to add an overload that takes a
`Materialized` and only create the store if that is supplied.
On Sun, 6 May 2018 at 17:52 Matthias J. Sax wrote:
> Andy,
>
> thanks for the KIP. I don'
Andy,
thanks for the KIP. I don't have any further comments.
My 2cents about Guozhang's questions: as I like consistent behavior, I
think transfromValues() should behave the same way as filter() and
mapValues().
-Matthias
On 5/2/18 2:24 PM, Guozhang Wang wrote:
> Hello Andy,
>
> Thanks for th
Hello Andy,
Thanks for the KIP. The motivation and the general proposal looks good to
me. I think in KTable it is indeed valuable to add the functions that does
not change key, such as mapValues, transformValues, and filter.
There are a few meta comments I have about the semantics of the newly ad
18 matches
Mail list logo