Awesome. Thanks to you both - KIP updated appropriately. On 11 May 2018 at 11:52, Matthias J. Sax <matth...@confluent.io> 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' 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 today. But thinking about it a bit more I think we do not need to fix > in > > this KIP, but rather in a different PR, to consider the following: > > > > 1) ProcessorContext#getStateStore(): should we return the internal > store, > > even if user provided the right internal store name? > > > > 2) For stores used in materialization, should we allow users to write to > > them? > > > > > > Guozhang > > > > > > On Fri, May 11, 2018 at 11:17 AM, Andy Coates <a...@confluent.io> wrote: > > > >> 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 code. I'm not sure there are any other places in > the > >> API that take both a Materialized instance and a set of state store > names, > >> so maybe this is a special/first case? > >> > >> I guess my main question would be what are the implications if a user > did > >> mutate the underlying statestate within the transformer itself? Or to > put > >> it another way, would allowing access to the state store result it weird > >> and wonderful bugs, and become a common source of bugs and another > 'thing' > >> developers need to know not to do, or would it 'just work'? If its the > >> former, then I think that's a strong case to have the code explicitly > stop > >> users doing dumb stuff. > >> > >> On 11 May 2018 at 11:00, Matthias J. Sax <matth...@confluent.io> wrote: > >> > >>> 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 processor and mess around with the KTable state. Of > >>> course, they should not do this, but it's possible after all and I > think > >>> it's a user error if they do. > >>> > >>> Thus, if we really want to restrict the access to internal > >>> created/manages stores, we should do it consistently -- but it's > unclear > >>> to me how to do this: note that Processor API and DSL should be > >>> separated (they are still not as clearly as they should, but we put > some > >>> effort into this the last year). Also, accessing the `context` is a > PAPI > >>> level operation. If we want to restrict the access to stores there, we > >>> need to introduce two "categories" of stores -- PAPI stores and DSL > >>> stores. This would be a step backwards IMHO and mangle DSL and PAPI > >>> together in an undesired way. > >>> > >>> Instead, we should clearly communicate to the users, that if they use > >>> `Materialized` they should never write to this store as it's managed by > >>> the library (reading is ok). > >>> > >>> WDYT? > >>> > >>> -Matthias > >>> > >>> On 5/11/18 10:50 AM, Guozhang Wang wrote: > >>>> 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 Materialized, the store name may not be specified by the user > >> and > >>>> hence it will be created internally; what would happen if users call > >>>> `getStateStore()` with the correct internal store name? If not the > >>>> semantics is a bit complex, if yes we are breaking the protocol to not > >>>> expose internal store. > >>>> > >>>> > >>>> Guozhang > >>>> > >>>> On Fri, May 11, 2018 at 10:42 AM, Andy Coates <a...@confluent.io> > >> wrote: > >>>> > >>>>> 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 <a...@confluent.io> wrote: > >>>>> > >>>>>> 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 > >>>>>> to getStateStore where the store name matches the materialized > result > >>> of > >>>>>> the call. > >>>>>> > >>>>>> I'll sort out the PR next. > >>>>>> > >>>>>> On 11 May 2018 at 10:26, Damian Guy <damian....@gmail.com> wrote: > >>>>>> > >>>>>>> I'm a +1 for Guozhang's suggestion > >>>>>>> > >>>>>>> On Fri, 11 May 2018 at 10:20 Andy Coates <a...@confluent.io> > wrote: > >>>>>>> > >>>>>>>> Makes sense to me. What do others think? > >>>>>>>> > >>>>>>>> On 11 May 2018 at 10:13, Guozhang Wang <wangg...@gmail.com> > 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: > >>>>>>>>> > >>>>>>>>> <VR> KTable<K, VR> transformValues(final > >>>>>>>> ValueTransformerWithKeySupplier<? > >>>>>>>>> super K, ? super V, ? extends VR> valueTransformerSupplier, > >>>>>>>>> final String... > >> stateStoreNames); > >>>>>>>>> > >>>>>>>>> <VR> KTable<K, VR> transformValues(final > >>>>>>>> ValueTransformerWithKeySupplier<? > >>>>>>>>> super K, ? super V, ? extends VR> valueTransformerSupplier, > >>>>>>>>> final Materialized<K, VR, > >>>>>>>>> KeyValueStore<Bytes, byte[]>> materialized, > >>>>>>>>> final String... > >> stateStoreNames); > >>>>>>>>> > >>>>>>>>> The reason is that, in KIP-149 we've added the overloaded > >> functions > >>>>>>>>> `withKey`, which should be covering the case without key already > >>>>>>> because > >>>>>>>> if > >>>>>>>>> users do not really need the key, they can just take it as a > dummy > >>>>>>>>> parameter. We did not deprecate the old ones since some of them > >> have > >>>>>>> just > >>>>>>>>> been added one version back. But if we agree that by the end of > >> the > >>>>>>> day > >>>>>>>> we > >>>>>>>>> would only maintain the overloaded value functions "with key" > >> only, > >>>>>>> then > >>>>>>>> we > >>>>>>>>> should not add the ones without keys any more in new KIPs. > >>>>>>>>> > >>>>>>>>> WDYT? > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> Guozhang > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> On Fri, May 11, 2018 at 9:42 AM, Andy Coates <a...@confluent.io> > >>>>>>> wrote: > >>>>>>>>> > >>>>>>>>>> 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(), etc. > >>>>>>>>>> > >>>>>>>>>> On 8 May 2018 at 17:59, Guozhang Wang <wangg...@gmail.com> > >> wrote: > >>>>>>>>>> > >>>>>>>>>>> To follow on Matthias and Damian's comments here: > >>>>>>>>>>> > >>>>>>>>>>> If we are going to add the overload functions as > >>>>>>>>>>> > >>>>>>>>>>> ``` > >>>>>>>>>>> <VR> KTable<K, VR> transformValues(final > >>>>>>> ValueTransformerSupplier<? > >>>>>>>>> super > >>>>>>>>>>> V, > >>>>>>>>>>> ? extends VR> valueTransformerSupplier, > >>>>>>>>>>> final String... > >>>>>>> stateStoreNames, > >>>>>>>>>>> final Materialized<K, > >>>>>>>>>>> VR, KeyValueStore<Bytes, byte[]> materialized); > >>>>>>>>>>> > >>>>>>>>>>> <VR> KTable<K, VR> transformValues(final > >>>>>>>> ValueTransformerWithKeySupplie > >>>>>>>>>> r<? > >>>>>>>>>>> super K, ? super V, ? extends VR> valueTransformerSupplier, > >>>>>>>>>>> final String... > >>>>>>> stateStoreNames, > >>>>>>>>>>> final Materialized<K, > >>>>>>>>>>> VR, KeyValueStore<Bytes, byte[]> materialized); > >>>>>>>>>>> ``` > >>>>>>>>>>> > >>>>>>>>>>> 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 materialized > >>>>> store > >>>>>>>>> should > >>>>>>>>>>> be managed by the Streams library itself, but we should > probably > >>>>>>> help > >>>>>>>>>> users > >>>>>>>>>>> to understand if they have some stores used for the same > purpose > >>>>>>>>> (storing > >>>>>>>>>>> the value that are going to be sent to the downstream changelog > >>>>>>>> stream > >>>>>>>>> of > >>>>>>>>>>> KTable), they should save that store and not creating by > >>>>>>> themselves > >>>>>>>> as > >>>>>>>>> it > >>>>>>>>>>> will be auto created by the Streams library. > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> Guozhang > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> On Tue, May 8, 2018 at 7:45 AM, Damian Guy < > >>>>> damian....@gmail.com> > >>>>>>>>> wrote: > >>>>>>>>>>> > >>>>>>>>>>>> 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 < > >>>>>>> matth...@confluent.io > >>>>>>>>> > >>>>>>>>>>> wrote: > >>>>>>>>>>>> > >>>>>>>>>>>>> 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 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 > >>>>>>>>>>>>> added > >>>>>>>>>>>>>> functions: > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> 1) For the resulted KTable, how should its > >>>>>>>> "queryableStoreName()" > >>>>>>>>>> be > >>>>>>>>>>>>>> returning? > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> 2) More specifically, how do we decide if the resulted > >>>>>>> KTable > >>>>>>>> is > >>>>>>>>> to > >>>>>>>>>>> be > >>>>>>>>>>>>>> materialized or not? E.g. if there is no store names > >>>>>>> provided > >>>>>>>>> then > >>>>>>>>>> it > >>>>>>>>>>>> is > >>>>>>>>>>>>>> likely that the resulted KTable is not materialized, or at > >>>>>>>> least > >>>>>>>>>> not > >>>>>>>>>>>>>> logically materialized and not be queryable. What if there > >>>>>>> is > >>>>>>>> at > >>>>>>>>>>> least > >>>>>>>>>>>>> one > >>>>>>>>>>>>>> state store provided? Will any of them be provided as the > >>>>>>>>>>> materialized > >>>>>>>>>>>>>> store, or should we still add a Materialized parameter for > >>>>>>> this > >>>>>>>>>>>> purpose? > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> 3) For its internal implementations, how should the > >>>>>>> key/value > >>>>>>>>>> serde, > >>>>>>>>>>>>>> sendOldValues flag etc be inherited from its parent > >>>>>>> processor > >>>>>>>>> node? > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> Guozhang > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> On Wed, May 2, 2018 at 12:43 PM, Andy Coates < > >>>>>>>> a...@confluent.io> > >>>>>>>>>>>> wrote: > >>>>>>>>>>>>>> > >>>>>>>>>>>>>>> Hi everyone, > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> I would like to start a discussion for KIP 292. I would > >>>>>>>>> appreciate > >>>>>>>>>>> it > >>>>>>>>>>>> if > >>>>>>>>>>>>>>> you could review and provide feedback. > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> KIP: KIP-292: Add transformValues() method to KTable > >>>>>>>>>>>>>>> <https://cwiki.apache.org/confluence/display/KAFKA/KIP- > >>>>>>>>>>>>>>> 292%3A+Add+transformValues%28%29+method+to+KTable> > >>>>>>>>>>>>>>> Jira: KAFKA-6849 <https://issues.apache.org/ > >>>>>>>>>> jira/browse/KAFKA-6849> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> PR: #4959 <https://github.com/apache/kafka/pull/4959> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> Thanks, > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>>> Andy > >>>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>>> > >>>>>>>>>>>>> > >>>>>>>>>>>>> > >>>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> > >>>>>>>>>>> -- > >>>>>>>>>>> -- Guozhang > >>>>>>>>>>> > >>>>>>>>>> > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> > >>>>>>>>> -- > >>>>>>>>> -- Guozhang > >>>>>>>>> > >>>>>>>> > >>>>>>> > >>>>>> > >>>>>> > >>>>> > >>>> > >>>> > >>>> > >>> > >>> > >> > > > > > > > >