Jeyhun, Matthias:

Thanks for the explanation, I overlooked the repartition argument
previously.

1) Based on that argument I'm convinced of having ValueMapperWithKey /
ValueJoinerWithKey / ValueTransformerWithKey; though I'm still not
convinced with ReducerWithKey and InitializerWithKey since for the former
it can be covered with `aggregate` completely and with latter I have seen
little use cases with it.

2) Another comment is on public interface ValueTransformer<V, VR> extends
ValueTransformerCommon<VR>:

I think changing the interface to extend from a new interface is not binary
compatible though source compatible, i.e. users still need to recompile
their code though no need to make code changes. We may need to mention that
in the upgrade path if we want to keep it that way.

Guozhang

On Mon, Jun 5, 2017 at 2:28 PM, Jeyhun Karimov <je.kari...@gmail.com> wrote:

> Hi,
>
>
> Sorry for late reply. Just to make everybody in sync, the current version
> of KIP supports lambdas. "withKey" (ValueMapperWithKey) interfaces are
> independent, meaning they do not extend from "withoutKey" (ValueMapper)
> interfaces.
>
>
> I agree with Guozhang, and I am personally a bit reluctant to increase
> overloaded methods in public APIs but it seems this is only way to solve
> all related jira issues.
> However, the most overloaded methods will be with ValueJoiner type, which
> will be with ValueJoinerWithKey with new overloaded methods. Other
> interfaces require mostly 1 extra overload.
>
>
> >> I would suggest not doing it if user pop it up, but rather suggesting
> them
> >> to use `map`
>
> I agree with Matthias as the core idea of this KIP was to collect all
> related jira issues and propose one-shot solution for all. Afterwards, we
> broke its scope into 2 KIPs (149 and 159).
>
> Cheers,
> Jeyhun
>
>
>
> On Mon, Jun 5, 2017 at 7:55 AM Matthias J. Sax <matth...@confluent.io>
> wrote:
>
> > I guess I missunderstood you. Your are right that overloading the method
> > would also work. As both interfaces will be independent from each other,
> > there should be no problem.
> >
> > The initial proposal was to use
> >
> > > interface ValueMapperWithKey extends ValueMapper
> >
> > and this would break Lambda. We totally missed, that we don't need new
> > methods but only only overlaods. Great you point this out! I was not
> > quite happy with the newly added methods.
> >
> > >> I would suggest not doing it if user pop it up, but rather suggesting
> > them
> > >> to use `map`
> >
> > But this might introduce unnecessary re-partitioning for downstream
> > operators. I don't thinks that a good suggestion. But if we only add new
> > overloads for mapValue(), flatMapValue() etc, I don't see a big issue
> > with "overstaffing" the API (what is btw a very valid concern).
> >
> >
> > -Matthias
> >
> >
> > On 6/4/17 10:37 PM, Guozhang Wang wrote:
> > > On Sun, Jun 4, 2017 at 8:41 PM, Matthias J. Sax <matth...@confluent.io
> >
> > > wrote:
> > >
> > >> We started with this proposal but it does not allow for Lambdas (in
> case
> > >> you want key access). Do you think preserving Lambdas is not important
> > >> enough for this case -- I agree that there is some price to be paid to
> > >> keep Lambdas.
> > >>
> > >
> > > Not sure if I understand this comment.. My main point is not on
> changing
> > > the proposal but just reduce it scope to `ValueJoinerWithKey` only; and
> > > even if we want to keep them all, I'd suggest we just implement the
> added
> > > APIs by using the `KeyValueMapper` for `ValueMapperWithKeys`, etc,
> which
> > > seems simpler to me. Does that affect lambda expression usage?
> > >
> > >>
> > >> About API consistency: I can't remember a concrete user request to
> have
> > >> key access in all operators. But I am pretty sure, if we only add it
> for
> > >> some, this request will pop up quickly. IMHO, it's better to do it all
> > >> in one shot. (But I am not strict about it if most people think we
> > >> should limit it to what was requested by users.)
> > >>
> > >>
> > > I would suggest not doing it if user pop it up, but rather suggesting
> > them
> > > to use `map` etc directly but set the key unchanged rather than
> > providing a
> > > new operator for it. To me some syntax sugars like this seems of less
> > > valuable than others (like print / writeAsText / foreach that are all
> > > depending on peek), and keeping adding them will just make the DSL too
> > > “overstaffed”.
> > >
> > >
> > >>
> > >> -Matthias
> > >>
> > >> On 6/4/17 6:23 PM, Guozhang Wang wrote:
> > >>> With KIP-159, the added "valueMapperWithKey" and
> > >> "valueTransformerWithKey"
> > >>> along seem to be just adding a few syntax-sugars? E.g. we can simply
> > >>> implement:
> > >>>
> > >>> mapValues(ValueMapperWithKey mapperWithKey)
> > >>>
> > >>> as
> > >>>
> > >>> map((k, v) -> (k, mapperWithKey(k, v))
> > >>>
> > >>> ----------------------
> > >>>
> > >>> I'm not sure how many users would really need such syntax sugars, and
> > >> even
> > >>> they do, we can support them easily as the above implementations;
> > >>>
> > >>> Similarly for "ReducerWithKey", it can be implemented as
> `Aggregator<K,
> > >> V,
> > >>> V>`, and people who needs key access can just use `aggregate`
> directly.
> > >>>
> > >>> The function which I think is really of valuable is
> > `ValueJoinerWithKey`,
> > >>> and that seems to be the original request from users while others are
> > >>> coming from "API consistency" concerns. Personally I'd prefer only
> keep
> > >> the
> > >>> last one (`InitializerWithKey` might also have some value, but I have
> > not
> > >>> seen people widely requesting it in their DSL usage yet; if there is
> a
> > >>> common request we can keep it in this KIP as well). WDYT?
> > >>>
> > >>> Guozhang
> > >>>
> > >>>
> > >>> On Sun, May 28, 2017 at 10:16 AM, Jeyhun Karimov <
> je.kari...@gmail.com
> > >
> > >>> wrote:
> > >>>
> > >>>> Thanks for clarification Matthias, now everything is clear.
> > >>>>
> > >>>> On Sun, May 28, 2017 at 6:21 PM Matthias J. Sax <
> > matth...@confluent.io>
> > >>>> wrote:
> > >>>>
> > >>>>> I don't think we can drop ValueTransformerSupplier. If you don't
> have
> > >> an
> > >>>>> supplier, you only get a single instance of your function. But for
> a
> > >>>>> stateful transformation, we need multiple instances (one for each
> > task)
> > >>>>> of ValueTransformer.
> > >>>>>
> > >>>>> We don't need suppliers for functions like "ValueMapper" etc
> because
> > >>>>> those are stateless and thus, we can reuse a single instance over
> > >>>>> multiple tasks -- but we can't do this for ValueTransformer (and
> > >>>> similar).
> > >>>>>
> > >>>>> Btw: This reminds me about KIP-159: with regard to the RichFunction
> > we
> > >>>>> might need a supplier pattern, too. (I'll comment on the other
> > thread,
> > >>>>> too.)
> > >>>>>
> > >>>>>
> > >>>>> -Matthias
> > >>>>>
> > >>>>> On 5/28/17 5:45 AM, Jeyhun Karimov wrote:
> > >>>>>> Hi,
> > >>>>>>
> > >>>>>> I updated KIP.
> > >>>>>> Just to avoid misunderstanding, I meant deprecating
> > >>>>> ValueTransformerSupplier
> > >>>>>> and I am ok with ValueTransformer.
> > >>>>>> So instead of using ValueTransformerSupplier can't we directly use
> > >>>>>> ValueTransformer
> > >>>>>> or ValueTransformerWithKey?
> > >>>>>>
> > >>>>>> Btw, in current design all features of ValueTransformer will be
> > >>>> available
> > >>>>>> in  ValueTransformerWithKey interface.
> > >>>>>>
> > >>>>>> Cheers,
> > >>>>>> Jeyhun
> > >>>>>>
> > >>>>>> On Sun, May 28, 2017 at 6:15 AM Matthias J. Sax <
> > >> matth...@confluent.io
> > >>>>>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Thanks Jeyhun.
> > >>>>>>>
> > >>>>>>> About ValueTransformer: I don't think we can remove it. Note,
> > >>>>>>> ValueTransformer allows to attach a state and also allows to
> > register
> > >>>>>>> punctuations. Both those features will not be available via
> > withKey()
> > >>>>>>> interfaces.
> > >>>>>>>
> > >>>>>>> -Matthias
> > >>>>>>>
> > >>>>>>>
> > >>>>>>> On 5/27/17 1:25 PM, Jeyhun Karimov wrote:
> > >>>>>>>> Hi Matthias,
> > >>>>>>>>
> > >>>>>>>> Thanks for your comments.
> > >>>>>>>>
> > >>>>>>>> I tested the deep copy approach. It has significant overhead.
> > >>>>> Especially
> > >>>>>>>> for "light" and stateless operators it slows down the topology
> > >>>>>>>> significantly (> 20% ). I think "warning"  users about
> > not-changing
> > >>>> the
> > >>>>>>> key
> > >>>>>>>> is better warning them about possible performance loss.
> > >>>>>>>>
> > >>>>>>>> About the interfaces, additionally I considered adding
> > >>>>>>> InitializerWithKey,
> > >>>>>>>> AggregatorWithKey and ValueTransformerWithKey. I think they are
> > >>>>> included
> > >>>>>>> in
> > >>>>>>>> PR but not in KIP. I will also include them in KIP, sorry my
> bad.
> > >>>>>>>> Including ReducerWithKey definitely makes sense. Thanks.
> > >>>>>>>>
> > >>>>>>>> One thing I want to mention is that, maybe we should deprecate
> > >>>> methods
> > >>>>>>> with
> > >>>>>>>> argument type ValueTransformerSupplier
> > >> (KStream.transformValues(...))
> > >>>>> and
> > >>>>>>>> and as a whole the ValueTransformerSupplier interface.
> > >>>>>>>> We can use ValueTransformer/ValueTransformerWithKey type
> instead
> > >>>>> without
> > >>>>>>>> additional supplier layer.
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> Cheers,
> > >>>>>>>> Jeyhun
> > >>>>>>>>
> > >>>>>>>>
> > >>>>>>>> On Thu, May 25, 2017 at 1:07 AM Matthias J. Sax <
> > >>>> matth...@confluent.io
> > >>>>>>
> > >>>>>>>> wrote:
> > >>>>>>>>
> > >>>>>>>>> One more question:
> > >>>>>>>>>
> > >>>>>>>>> Should we add any of
> > >>>>>>>>>  - InitizialierWithKey
> > >>>>>>>>>  - ReducerWithKey
> > >>>>>>>>>  - ValueTransformerWithKey
> > >>>>>>>>>
> > >>>>>>>>> To get consistent/complete API, it might be a good idea. Any
> > >>>> thoughts?
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> -Matthias
> > >>>>>>>>>
> > >>>>>>>>>
> > >>>>>>>>> On 5/24/17 3:47 PM, Matthias J. Sax wrote:
> > >>>>>>>>>> Jeyhun,
> > >>>>>>>>>>
> > >>>>>>>>>> I was just wondering if you did look into the key-deep-copy
> idea
> > >> we
> > >>>>>>>>>> discussed. I am curious to see what the impact might be.
> > >>>>>>>>>>
> > >>>>>>>>>>
> > >>>>>>>>>> -Matthias
> > >>>>>>>>>>
> > >>>>>>>>>> On 5/20/17 2:03 AM, Jeyhun Karimov wrote:
> > >>>>>>>>>>> Hi,
> > >>>>>>>>>>>
> > >>>>>>>>>>> Thanks for your comments. I rethink about including rich
> > >> functions
> > >>>>>>> into
> > >>>>>>>>>>> this KIP.
> > >>>>>>>>>>> I think once we include rich functions in this KIP and then
> fix
> > >>>>>>>>>>> ProcessorContext in another KIP and incorporate with existing
> > >> rich
> > >>>>>>>>>>> functions, the code will not be backwards compatible.
> > >>>>>>>>>>>
> > >>>>>>>>>>> I see Damian's and your point more clearly now.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Conclusion: we include only withKey interfaces in this KIP (I
> > >>>>> updated
> > >>>>>>>>> the
> > >>>>>>>>>>> KIP), I will try also initiate another KIP for rich
> functions.
> > >>>>>>>>>>>
> > >>>>>>>>>>> Cheers,
> > >>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>
> > >>>>>>>>>>> On Fri, May 19, 2017 at 10:50 PM Matthias J. Sax <
> > >>>>>>> matth...@confluent.io
> > >>>>>>>>>>
> > >>>>>>>>>>> wrote:
> > >>>>>>>>>>>
> > >>>>>>>>>>>> With the current KIP, using ValueMapper and
> ValueMapperWithKey
> > >>>>>>>>>>>> interfaces, RichFunction seems to be an independent add-on.
> To
> > >>>> fix
> > >>>>>>> the
> > >>>>>>>>>>>> original issue to allow key access, RichFunctions are not
> > >>>> required
> > >>>>>>>>> IMHO.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> I initially put the RichFunction idea on the table, because
> I
> > >> was
> > >>>>>>>>> hoping
> > >>>>>>>>>>>> to get a uniform API. And I think, is was good to consider
> > them
> > >>>> --
> > >>>>>>>>>>>> however, the discussion showed that they are not necessary
> for
> > >>>> key
> > >>>>>>>>>>>> access. Thus, it seems to be better to handle RichFunctions
> in
> > >> an
> > >>>>> own
> > >>>>>>>>>>>> KIP. The ProcessorContext/RecordContext issues seems to be a
> > >> main
> > >>>>>>>>>>>> challenge for this. And introducing RichFunctions with
> > >>>>> parameter-less
> > >>>>>>>>>>>> init() method, seem not to help too much. We would get an
> > >>>>>>>>> "intermediate"
> > >>>>>>>>>>>> API that we want to change anyway later on...
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> As you put already much effort into RichFunction, feel free
> do
> > >>>> push
> > >>>>>>>>> this
> > >>>>>>>>>>>> further and start a new KIP (we can do this even in
> parallel)
> > --
> > >>>> we
> > >>>>>>>>>>>> don't want to slow you down :) But it make the discussion
> and
> > >>>> code
> > >>>>>>>>>>>> review easier, if we separate both IMHO.
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>
> > >>>>>>>>>>>>
> > >>>>>>>>>>>> On 5/19/17 2:25 AM, Jeyhun Karimov wrote:
> > >>>>>>>>>>>>> Hi Damian,
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Thanks for your comments. I think providing to users
> > >> *interface*
> > >>>>>>>>> rather
> > >>>>>>>>>>>>> than *abstract class* should be preferred (Matthias also
> > raised
> > >>>>> this
> > >>>>>>>>>>>> issue
> > >>>>>>>>>>>>> ), anyway I changed the corresponding parts of KIP.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Regarding with passing additional contextual information, I
> > >>>> think
> > >>>>> it
> > >>>>>>>>> is a
> > >>>>>>>>>>>>> tradeoff,
> > >>>>>>>>>>>>> 1) first, we fix the context parameter for *init() *method
> in
> > >>>>>>> another
> > >>>>>>>>> PR
> > >>>>>>>>>>>>> and solve Rich functions afterwards
> > >>>>>>>>>>>>> 2) first, we fix the requested issues on jira ([1-3]) with
> > >>>>> providing
> > >>>>>>>>>>>>> (not-complete) Rich functions and integrate the context
> > >>>> parameters
> > >>>>>>> to
> > >>>>>>>>>>>> this
> > >>>>>>>>>>>>> afterwards (like improvement)
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> To me, the second approach seems more incremental. However
> > you
> > >>>> are
> > >>>>>>>>> right,
> > >>>>>>>>>>>>> the names might confuse the users.
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> [1] https://issues.apache.org/jira/browse/KAFKA-4218
> > >>>>>>>>>>>>> [2] https://issues.apache.org/jira/browse/KAFKA-4726
> > >>>>>>>>>>>>> [3] https://issues.apache.org/jira/browse/KAFKA-3745
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>> On Fri, May 19, 2017 at 10:42 AM Damian Guy <
> > >>>> damian....@gmail.com
> > >>>>>>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> I see you've removed the `ProcessorContext` from the
> > >>>> RichFunction
> > >>>>>>>>> which
> > >>>>>>>>>>>> is
> > >>>>>>>>>>>>>> good, but why is it a `RichFunction`? I'd have expected it
> > to
> > >>>>> pass
> > >>>>>>>>> some
> > >>>>>>>>>>>>>> additional contextual information, i.e., the
> `RecordContext`
> > >>>> that
> > >>>>>>>>>>>> contains
> > >>>>>>>>>>>>>> just the topic, partition, timestamp, offset.  I'm ok with
> > it
> > >>>> not
> > >>>>>>>>>>>> passing
> > >>>>>>>>>>>>>> this contextual information, but is the naming incorrect?
> > I'm
> > >>>> not
> > >>>>>>>>> sure,
> > >>>>>>>>>>>>>> tbh. I'm wondering if we should drop `RichFunctions` until
> > we
> > >>>> can
> > >>>>>>> do
> > >>>>>>>>> it
> > >>>>>>>>>>>>>> properly with the correct context?
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Also, i don't like the abstract classes: RichValueMapper,
> > >>>>>>>>>>>> RichValueJoiner,
> > >>>>>>>>>>>>>> RichInitializer etc. Why can't they not just be
> interfaces?
> > >>>>>>>>> Generally we
> > >>>>>>>>>>>>>> should provide users with Intefaces to code against, not
> > >>>> classes.
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> Thanks,
> > >>>>>>>>>>>>>> Damian
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>> On Fri, 19 May 2017 at 00:50 Jeyhun Karimov <
> > >>>>> je.kari...@gmail.com>
> > >>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Thanks. I initiated the PR as well, to get a better
> > overview.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> The only reason that I used abstract class instead of
> > >>>> interface
> > >>>>>>> for
> > >>>>>>>>>>>> Rich
> > >>>>>>>>>>>>>>> functions is that in future if we will have some
> > >>>>>>>>> AbstractRichFunction
> > >>>>>>>>>>>>>>> abstract classes,
> > >>>>>>>>>>>>>>> we can easily extend:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> public abstract class RichValueMapper<K, V, VR>
> implements
> > >>>>>>>>>>>>>>> ValueMapperWithKey<K, V, VR>, RichFunction *extends
> > >>>>>>>>>>>>>> AbstractRichFunction*{
> > >>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>  With interfaces we are only limited to interfaces for
> > >>>>>>> inheritance.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> However, I think we can easily change it (from abstract
> > class
> > >>>> ->
> > >>>>>>>>>>>>>> interface)
> > >>>>>>>>>>>>>>> if you think interface is a better fit.
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> On Fri, May 19, 2017 at 1:00 AM Matthias J. Sax <
> > >>>>>>>>> matth...@confluent.io
> > >>>>>>>>>>>>>
> > >>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Thanks for the update and explanations. The KIP is quite
> > >>>>> improved
> > >>>>>>>>> now
> > >>>>>>>>>>>>>> --
> > >>>>>>>>>>>>>>>> great job!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> One more question: Why are RichValueMapper etc abstract
> > >>>> classes
> > >>>>>>> and
> > >>>>>>>>>>>> not
> > >>>>>>>>>>>>>>>> interfaces?
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> Overall, I like the KIP a lot!
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>> On 5/16/17 7:03 AM, Jeyhun Karimov wrote:
> > >>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Thanks for your comments.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> I think supporting Lambdas for `withKey` and
> > >>>>>>>>> `AbstractRichFunction`
> > >>>>>>>>>>>>>>>>>> don't go together, as Lambdas are only supported for
> > >>>>> interfaces
> > >>>>>>>>>>>>>> AFAIK.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Maybe I misunderstood your comment.
> > >>>>>>>>>>>>>>>>> *withKey* and and *withOnlyValue* are interfaces. So
> they
> > >>>>> don't
> > >>>>>>>>> have
> > >>>>>>>>>>>>>>>> direct
> > >>>>>>>>>>>>>>>>> relation with *AbstractRichFunction*.
> > >>>>>>>>>>>>>>>>> *withKey* and and *withOnlyValue* interfaces have only
> > one
> > >>>>>>>>> method ,
> > >>>>>>>>>>>>>> so
> > >>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>> can use lambdas.
> > >>>>>>>>>>>>>>>>> Where does the *AbstractRichFunction* comes to play?
> > Inside
> > >>>>> Rich
> > >>>>>>>>>>>>>>>> functions.
> > >>>>>>>>>>>>>>>>> And we use Rich functions in 2 places:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> 1. User doesn't use rich functions. Just regular
> > *withKey*
> > >>>> and
> > >>>>>>> and
> > >>>>>>>>>>>>>>>>> *withOnlyValue* interfaces(both support lambdas) . We
> get
> > >>>>> those
> > >>>>>>>>>>>>>>>> interfaces
> > >>>>>>>>>>>>>>>>> and wrap into Rich function while building the
> topology,
> > >> and
> > >>>>>>> send
> > >>>>>>>>> to
> > >>>>>>>>>>>>>>>>> Processor.
> > >>>>>>>>>>>>>>>>> 2. User does use rich functions (Rich functions
> implement
> > >>>>>>>>> *withKey*
> > >>>>>>>>>>>>>>>>> interface). As a result no lamdas here by definition.
> In
> > >>>> this
> > >>>>>>>>> case,
> > >>>>>>>>>>>>>>> while
> > >>>>>>>>>>>>>>>>> building the topology we do a type check if the object
> > type
> > >>>> is
> > >>>>>>>>>>>>>>>>> *withKey* or *RichFunction*.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> So *AbstractRichFunction* is just syntactic sugar for
> > Rich
> > >>>>>>>>> functions
> > >>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>> does not affect using lambdas.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Thus, if we want to support Lambdas for `withKey`, we
> > need
> > >>>> to
> > >>>>>>>>> have a
> > >>>>>>>>>>>>>>>>>> interface approach like this
> > >>>>>>>>>>>>>>>>>>   - RichFunction -> only adding init() and close()
> > >>>>>>>>>>>>>>>>>>   - ValueMapper
> > >>>>>>>>>>>>>>>>>>   - ValueMapperWithKey
> > >>>>>>>>>>>>>>>>>>   - RichValueMapper extends ValueMapperWithKey,
> > >>>> RichFunction
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> As I said above, currently we support lambdas for
> > *withKey*
> > >>>>>>>>>>>>>> interfaces
> > >>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>> well.  However, I agree with your idea and I will
> remove
> > >> the
> > >>>>>>>>>>>>>>>>> AbstractRichFunction from the design.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> As an alternative, we could argue, that it is
> sufficient
> > to
> > >>>>>>>>> support
> > >>>>>>>>>>>>>>>>>> Lambdas for the "plain" API only, but not for any
> > >> "extended
> > >>>>>>> API".
> > >>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>> this, RichFunction could add key+init+close and
> > >>>>>>>>> AbstractRichFunction
> > >>>>>>>>>>>>>>>>>> would allow to only care about getting the key.
> > >>>>>>>>>>>>>>>>>> Not sure, which one is better. I don't like the idea
> of
> > >>>> more
> > >>>>>>>>>>>>>>> overloaded
> > >>>>>>>>>>>>>>>>>> methods to get Lambdas for `withKey` interfaces too
> much
> > >>>>>>> because
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>> already so many overlaods. On the other hand, I do see
> > >>>> value
> > >>>>> in
> > >>>>>>>>>>>>>>>>>> supporting Lambdas for `withKey`.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Just to clarify, with current design we have only one
> > extra
> > >>>>>>>>>>>>>> overloaded
> > >>>>>>>>>>>>>>>>> method per *withOnlyValue* interface:  which is
> *withKey*
> > >>>>>>> version
> > >>>>>>>>> of
> > >>>>>>>>>>>>>>>>> particular interface.
> > >>>>>>>>>>>>>>>>> We don't need extra overload for Rich function as Rich
> > >>>>> function
> > >>>>>>>>>>>>>>>> implements
> > >>>>>>>>>>>>>>>>> *withKey* interface as a result they have same type. We
> > >>>>>>>>> differentiate
> > >>>>>>>>>>>>>>>> them
> > >>>>>>>>>>>>>>>>> while building the topology.
> > >>>>>>>>>>>>>>>>> We supported lambdas for *withKey* APIs because of the
> > >>>>> comment:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> @Jeyhun: I did not put any thought into this, but can
> we
> > >>>> have
> > >>>>> a
> > >>>>>>>>>>>>>> design
> > >>>>>>>>>>>>>>>>>> that allows for both? Also, with regard to lambdas, it
> > >>>> might
> > >>>>>>> make
> > >>>>>>>>>>>>>>> sense
> > >>>>>>>>>>>>>>>>>> to allow for both `V -> newV` and `(K, V) -> newV` ?
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> However, I don't think that this complicates the
> overall
> > >>>>> design
> > >>>>>>>>>>>>>>>>> significantly.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Depending on what we want to support, it might make
> sense
> > >> to
> > >>>>>>>>>>>>>>>>>> include/exclude RichFunctions from this KIP -- and
> thus,
> > >>>> this
> > >>>>>>>>> also
> > >>>>>>>>>>>>>>>>>> determines if we should have a "ProcessorContext KIP"
> > >>>> before
> > >>>>>>>>> driving
> > >>>>>>>>>>>>>>>>>> this KIP further.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Based on our discussion I think we should keep Rich
> > >>>> functions
> > >>>>>>> as I
> > >>>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>> think that they bring extra layer of overhead to
> library.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Any comments are appreciated.
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>> On Tue, May 16, 2017 at 12:10 AM Matthias J. Sax <
> > >>>>>>>>>>>>>>> matth...@confluent.io>
> > >>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Jeyhun,
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> thanks for the update.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> I think supporting Lambdas for `withKey` and
> > >>>>>>>>> `AbstractRichFunction`
> > >>>>>>>>>>>>>>>>>> don't go together, as Lambdas are only supported for
> > >>>>> interfaces
> > >>>>>>>>>>>>>> AFAIK.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Thus, if we want to support Lambdas for `withKey`, we
> > need
> > >>>> to
> > >>>>>>>>> have a
> > >>>>>>>>>>>>>>>>>> interface approach like this
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>   - RichFunction -> only adding init() and close()
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>   - ValueMapper
> > >>>>>>>>>>>>>>>>>>   - ValueMapperWithKey
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>   - RichValueMapper extends ValueMapperWithKey,
> > >>>> RichFunction
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> For this approach, AbstractRichFunction does not make
> > >> sense
> > >>>>>>>>> anymore,
> > >>>>>>>>>>>>>>> as
> > >>>>>>>>>>>>>>>>>> the only purpose of `RichFunction` is to allow the
> > >>>>>>>>> implementation of
> > >>>>>>>>>>>>>>>>>> init() and close() -- if you don't want those, you
> would
> > >>>>>>>>> implement a
> > >>>>>>>>>>>>>>>>>> different interface (ie, ValueMapperWithKey)
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> As an alternative, we could argue, that it is
> sufficient
> > >> to
> > >>>>>>>>> support
> > >>>>>>>>>>>>>>>>>> Lambdas for the "plain" API only, but not for any
> > >> "extended
> > >>>>>>> API".
> > >>>>>>>>>>>>>> For
> > >>>>>>>>>>>>>>>>>> this, RichFunction could add key+init+close and
> > >>>>>>>>> AbstractRichFunction
> > >>>>>>>>>>>>>>>>>> would allow to only care about getting the key.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Not sure, which one is better. I don't like the idea
> of
> > >>>> more
> > >>>>>>>>>>>>>>> overloaded
> > >>>>>>>>>>>>>>>>>> methods to get Lambdas for `withKey` interfaces too
> much
> > >>>>>>> because
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>> have
> > >>>>>>>>>>>>>>>>>> already so many overlaods. On the other hand, I do see
> > >>>> value
> > >>>>> in
> > >>>>>>>>>>>>>>>>>> supporting Lambdas for `withKey`.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Depending on what we want to support, it might make
> > sense
> > >>>> to
> > >>>>>>>>>>>>>>>>>> include/exclude RichFunctions from this KIP -- and
> thus,
> > >>>> this
> > >>>>>>>>> also
> > >>>>>>>>>>>>>>>>>> determines if we should have a "ProcessorContext KIP"
> > >>>> before
> > >>>>>>>>> driving
> > >>>>>>>>>>>>>>>>>> this KIP further.
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> Thoughts?
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>> On 5/15/17 11:01 AM, Jeyhun Karimov wrote:
> > >>>>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Sorry for super late response. Thanks for your
> > comments.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I am not an expert on Lambdas. Can you elaborate a
> > little
> > >>>>>>> bit? I
> > >>>>>>>>>>>>>>> cannot
> > >>>>>>>>>>>>>>>>>>>> follow the explanation in the KIP to see what the
> > >> problem
> > >>>>> is.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - From [1] says "A functional interface is an
> interface
> > >>>> that
> > >>>>>>> has
> > >>>>>>>>>>>>>> just
> > >>>>>>>>>>>>>>>> one
> > >>>>>>>>>>>>>>>>>>> abstract method, and thus represents a single
> function
> > >>>>>>>>> contract".
> > >>>>>>>>>>>>>>>>>>> So basically once we extend some interface from
> another
> > >>>> (in
> > >>>>>>> our
> > >>>>>>>>>>>>>> case,
> > >>>>>>>>>>>>>>>>>>> ValueMapperWithKey from ValueMapper) we cannot use
> > >> lambdas
> > >>>>> in
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>>>> extended
> > >>>>>>>>>>>>>>>>>>> interface.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Further comments:
> > >>>>>>>>>>>>>>>>>>>>  - The KIP get a little hard to read -- can you
> maybe
> > >>>>>>> reformat
> > >>>>>>>>> the
> > >>>>>>>>>>>>>>>> wiki
> > >>>>>>>>>>>>>>>>>>>> page a little bit? I think using `CodeBlock` would
> > help.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - I will work on the KIP.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>  - What about KStream-KTable joins? You don't have
> > >>>> overlaods
> > >>>>>>>>> added
> > >>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>> them. Why? (Even if I still hope that we don't need
> to
> > >>>> add
> > >>>>>>> any
> > >>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>> overloads)
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> - Actually there are more than one Processor and
> public
> > >>>> APIs
> > >>>>>>> to
> > >>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>> changed (KStream-KTable
> > >>>>>>>>>>>>>>>>>>> joins is one case). However all of them has similar
> > >>>>> structure:
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>>>>> overload
> > >>>>>>>>>>>>>>>>>>> the *method* with  *methodWithKey*,
> > >>>>>>>>>>>>>>>>>>> wrap it into the Rich function, send to processor and
> > >>>> inside
> > >>>>>>> the
> > >>>>>>>>>>>>>>>>>> processor
> > >>>>>>>>>>>>>>>>>>> call *init* and *close* methods of the Rich function.
> > >>>>>>>>>>>>>>>>>>> As I wrote in KIP, I wanted to demonstrate the
> overall
> > >>>> idea
> > >>>>>>> with
> > >>>>>>>>>>>>>> only
> > >>>>>>>>>>>>>>>>>>> *ValueMapper* as the same can be applied to all
> > changes.
> > >>>>>>>>>>>>>>>>>>> Anyway I will update the KIP.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>  - Why do we need `AbstractRichFunction`?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Instead of overriding the *init(ProcessorContext p)*
> > and*
> > >>>>>>>>> close()*
> > >>>>>>>>>>>>>>>>>> methods
> > >>>>>>>>>>>>>>>>>>> in every Rich function with empty body like:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> @Override
> > >>>>>>>>>>>>>>>>>>> void init(ProcessorContext context) {}
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> @Override
> > >>>>>>>>>>>>>>>>>>> void close () {}
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I thought that we can override them once in
> > >>>>>>>>> *AbstractRichFunction*
> > >>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>> extent new Rich functions from
> *AbstractRichFunction*.
> > >>>>>>>>>>>>>>>>>>> Basically this can eliminate code copy-paste and ease
> > the
> > >>>>>>>>>>>>>>> maintenance.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>  - What about interfaces Initializer, ForeachAction,
> > >>>> Merger,
> > >>>>>>>>>>>>>>> Predicate,
> > >>>>>>>>>>>>>>>>>>>> Reducer? I don't want to say we should/need to add
> to
> > >>>> all,
> > >>>>>>> but
> > >>>>>>>>> we
> > >>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>> discuss all of them and add where it does make sense
> > >>>> (e.g.,
> > >>>>>>>>>>>>>>>>>>>> RichForachAction does make sense IMHO)
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Definitely agree. As I said, the same technique
> applies
> > >> to
> > >>>>> all
> > >>>>>>>>> this
> > >>>>>>>>>>>>>>>>>>> interfaces and I didn't want to explode the KIP, just
> > >>>> wanted
> > >>>>>>> to
> > >>>>>>>>>>>>>> give
> > >>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>> overall intuition.
> > >>>>>>>>>>>>>>>>>>> However, I will update the KIP as I said.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Btw: I like the hierarchy `ValueXX` --
> `ValueXXWithKey`
> > >> --
> > >>>>>>>>>>>>>>>> `RichValueXX`
> > >>>>>>>>>>>>>>>>>>>> in general -- but why can't we do all this with
> > >>>> interfaces
> > >>>>>>>>> only?
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Sure we can. However the main intuition is we should
> > not
> > >>>>> force
> > >>>>>>>>>>>>>> users
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>> implement *init(ProcessorContext)* and *close()*
> > >> functions
> > >>>>>>> every
> > >>>>>>>>>>>>>> time
> > >>>>>>>>>>>>>>>>>> they
> > >>>>>>>>>>>>>>>>>>> use Rich functions.
> > >>>>>>>>>>>>>>>>>>> If one needs, she can override the respective
> methods.
> > >>>>>>> However,
> > >>>>>>>>> I
> > >>>>>>>>>>>>>> am
> > >>>>>>>>>>>>>>>> open
> > >>>>>>>>>>>>>>>>>>> for discussion.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> I'd rather not see the use of  `ProcessorContext`
> > spread
> > >>>> any
> > >>>>>>>>>>>>>> further
> > >>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>> it currently is. So maybe we need another KIP that
> is
> > >>>> done
> > >>>>>>>>> before
> > >>>>>>>>>>>>>>>> this?
> > >>>>>>>>>>>>>>>>>>>> Otherwise i think the scope of this KIP is becoming
> > too
> > >>>>>>> large.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> That is good point. I wanted to make
> > >>>>> *init(ProcessorContext)*
> > >>>>>>>>>>>>>> method
> > >>>>>>>>>>>>>>>>>>> persistent among the library (which use
> > ProcessorContext
> > >>>> as
> > >>>>> an
> > >>>>>>>>>>>>>>> input),
> > >>>>>>>>>>>>>>>>>>> therefore I put *ProcessorContext* as an input.
> > >>>>>>>>>>>>>>>>>>> So the important question is that (as @dguy and
> @mjsax
> > >>>>>>>>> mentioned)
> > >>>>>>>>>>>>>>>> whether
> > >>>>>>>>>>>>>>>>>>> continue this KIP without providing users an access
> to
> > >>>>>>>>>>>>>>>> *ProcessorContext*
> > >>>>>>>>>>>>>>>>>>> (change *init (ProcessorContext)* to * init()* ) or
> > >>>>>>>>>>>>>>>>>>> initiate another KIP before this.
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> [1]
> > >>>>>>>>>>>>>>>>>>>
> > >>>>> http://cr.openjdk.java.net/~mr/se/8/java-se-8-pfd-spec/java-
> > >>>>>>>>>>>>>>>>>> se-8-jls-pfd-diffs.pdf
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>> On Mon, May 15, 2017 at 7:15 PM, Damian Guy <
> > >>>>>>>>> damian....@gmail.com>
> > >>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> I'd rather not see the use of  `ProcessorContext`
> > spread
> > >>>>> any
> > >>>>>>>>>>>>>> further
> > >>>>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>> it currently is. So maybe we need another KIP that
> is
> > >>>> done
> > >>>>>>>>> before
> > >>>>>>>>>>>>>>>> this?
> > >>>>>>>>>>>>>>>>>>>> Otherwise i think the scope of this KIP is becoming
> > too
> > >>>>>>> large.
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> On Mon, 15 May 2017 at 18:06 Matthias J. Sax <
> > >>>>>>>>>>>>>> matth...@confluent.io
> > >>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> I agree that that `ProcessorContext` interface is
> too
> > >>>>> broad
> > >>>>>>> in
> > >>>>>>>>>>>>>>>> general
> > >>>>>>>>>>>>>>>>>>>>> -- this is even true for transform/process, and
> it's
> > >>>> also
> > >>>>>>>>>>>>>> reflected
> > >>>>>>>>>>>>>>>> in
> > >>>>>>>>>>>>>>>>>>>>> the API improvement list we want to do.
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/
> > >>>>>>>>>>>>>>>>>>>> Kafka+Streams+Discussions
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> So I am wondering, if you question the
> `RichFunction`
> > >>>>>>>>> approach in
> > >>>>>>>>>>>>>>>>>>>>> general? Or if you suggest to either extend the
> scope
> > >> of
> > >>>>>>> this
> > >>>>>>>>> KIP
> > >>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> include this---or maybe better, do another KIP for
> it
> > >>>> and
> > >>>>>>>>> delay
> > >>>>>>>>>>>>>>> this
> > >>>>>>>>>>>>>>>>>> KIP
> > >>>>>>>>>>>>>>>>>>>>> until the other one is done?
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> On 5/15/17 2:35 AM, Damian Guy wrote:
> > >>>>>>>>>>>>>>>>>>>>>> Thanks for the KIP.
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> I'm not convinced on the `RichFunction` approach.
> Do
> > >> we
> > >>>>>>>>> really
> > >>>>>>>>>>>>>>> want
> > >>>>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>>>>>>> give
> > >>>>>>>>>>>>>>>>>>>>>> every DSL method access to the `ProcessorContext`
> ?
> > It
> > >>>>> has
> > >>>>>>> a
> > >>>>>>>>>>>>>> bunch
> > >>>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>>>> methods on it that seem in-appropriate for some of
> > the
> > >>>>> DSL
> > >>>>>>>>>>>>>>> methods,
> > >>>>>>>>>>>>>>>>>>>> i.e,
> > >>>>>>>>>>>>>>>>>>>>>> `register`, `getStateStore`, `forward`, `schedule`
> > >> etc.
> > >>>>> It
> > >>>>>>> is
> > >>>>>>>>>>>>>> far
> > >>>>>>>>>>>>>>>> too
> > >>>>>>>>>>>>>>>>>>>>>> broad. I think it would be better to have a
> narrower
> > >>>>>>>>> interface
> > >>>>>>>>>>>>>>> like
> > >>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>> `RecordContext`  - remembering it is easier to add
> > >>>>>>>>>>>>>>>> methods/interfaces
> > >>>>>>>>>>>>>>>>>>>>> later
> > >>>>>>>>>>>>>>>>>>>>>> than to remove them
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>> On Sat, 13 May 2017 at 22:26 Matthias J. Sax <
> > >>>>>>>>>>>>>>> matth...@confluent.io
> > >>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Jeyhun,
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> I am not an expert on Lambdas. Can you elaborate
> a
> > >>>>> little
> > >>>>>>>>> bit?
> > >>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>> cannot
> > >>>>>>>>>>>>>>>>>>>>>>> follow the explanation in the KIP to see what the
> > >>>>> problem
> > >>>>>>>>> is.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> For updating the KIP title I don't know -- guess
> > it's
> > >>>> up
> > >>>>>>> to
> > >>>>>>>>>>>>>> you.
> > >>>>>>>>>>>>>>>>>>>> Maybe a
> > >>>>>>>>>>>>>>>>>>>>>>> committer can comment on this?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Further comments:
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>  - The KIP get a little hard to read -- can you
> > maybe
> > >>>>>>>>> reformat
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> wiki
> > >>>>>>>>>>>>>>>>>>>>>>> page a little bit? I think using `CodeBlock`
> would
> > >>>> help.
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>  - What about KStream-KTable joins? You don't
> have
> > >>>>>>> overlaods
> > >>>>>>>>>>>>>>> added
> > >>>>>>>>>>>>>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>> them. Why? (Even if I still hope that we don't
> need
> > >> to
> > >>>>> add
> > >>>>>>>>> any
> > >>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>> overloads)
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>  - Why do we need `AbstractRichFunction`?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>  - What about interfaces Initializer,
> > ForeachAction,
> > >>>>>>> Merger,
> > >>>>>>>>>>>>>>>>>>>> Predicate,
> > >>>>>>>>>>>>>>>>>>>>>>> Reducer? I don't want to say we should/need to
> add
> > to
> > >>>>> all,
> > >>>>>>>>> but
> > >>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>>>> discuss all of them and add where it does make
> > sense
> > >>>>>>> (e.g.,
> > >>>>>>>>>>>>>>>>>>>>>>> RichForachAction does make sense IMHO)
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> Btw: I like the hierarchy `ValueXX` --
> > >>>> `ValueXXWithKey`
> > >>>>> --
> > >>>>>>>>>>>>>>>>>>>> `RichValueXX`
> > >>>>>>>>>>>>>>>>>>>>>>> in general -- but why can't we do all this with
> > >>>>> interfaces
> > >>>>>>>>>>>>>> only?
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>> On 5/11/17 7:00 AM, Jeyhun Karimov wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Thanks for your comments. I think we cannot
> extend
> > >>>> the
> > >>>>>>> two
> > >>>>>>>>>>>>>>>>>> interfaces
> > >>>>>>>>>>>>>>>>>>>>> if
> > >>>>>>>>>>>>>>>>>>>>>>> we
> > >>>>>>>>>>>>>>>>>>>>>>>> want to keep lambdas. I updated the KIP [1].
> > Maybe I
> > >>>>>>> should
> > >>>>>>>>>>>>>>> change
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>> title, because now we are not limiting the KIP
> to
> > >>>> only
> > >>>>>>>>>>>>>>>> ValueMapper,
> > >>>>>>>>>>>>>>>>>>>>>>>> ValueTransformer and ValueJoiner.
> > >>>>>>>>>>>>>>>>>>>>>>>> Please feel free to comment.
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> [1]
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>> https://cwiki.apache.org/
> > confluence/display/KAFKA/KIP-
> > >>>>>>>>>>>>>>>>>>>> 149%3A+Enabling+key+access+in+ValueTransformer%2C+
> > >>>>>>>>>>>>>>>>>>>> ValueMapper%2C+and+ValueJoiner
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>> On Tue, May 9, 2017 at 7:36 PM Matthias J. Sax <
> > >>>>>>>>>>>>>>>>>>>> matth...@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> If `ValueMapperWithKey` extends `ValueMapper`
> we
> > >>>> don't
> > >>>>>>>>> need
> > >>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>> new
> > >>>>>>>>>>>>>>>>>>>>>>>>> overlaod.
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> And yes, we need to do one check -- but this
> > >> happens
> > >>>>>>> when
> > >>>>>>>>>>>>>>>> building
> > >>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> topology. At runtime (I mean after
> > >>>> KafkaStream#start()
> > >>>>>>> we
> > >>>>>>>>>>>>>> don't
> > >>>>>>>>>>>>>>>>>> need
> > >>>>>>>>>>>>>>>>>>>>> any
> > >>>>>>>>>>>>>>>>>>>>>>>>> check as we will always use
> `ValueMapperWithKey`)
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>> On 5/9/17 2:55 AM, Jeyhun Karimov wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for feedback.
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Then we need to overload method
> > >>>>>>>>>>>>>>>>>>>>>>>>>>   <VR> KStream<K, VR> mapValues(ValueMapper<?
> > >> super
> > >>>>> V,
> > >>>>>>> ?
> > >>>>>>>>>>>>>>> extends
> > >>>>>>>>>>>>>>>>>>>> VR>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> mapper);
> > >>>>>>>>>>>>>>>>>>>>>>>>>> with
> > >>>>>>>>>>>>>>>>>>>>>>>>>>   <VR> KStream<K, VR>
> > >>>> mapValues(ValueMapperWithKey<?
> > >>>>>>>>> super
> > >>>>>>>>>>>>>> V,
> > >>>>>>>>>>>>>>> ?
> > >>>>>>>>>>>>>>>>>>>>> extends
> > >>>>>>>>>>>>>>>>>>>>>>>>> VR>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> mapper);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> and in runtime (inside processor) we still
> have
> > to
> > >>>>>>> check
> > >>>>>>>>> it
> > >>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>> ValueMapper
> > >>>>>>>>>>>>>>>>>>>>>>>>>> or ValueMapperWithKey before wrapping it into
> > the
> > >>>>> rich
> > >>>>>>>>>>>>>>> function.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Please correct me if I am wrong.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>> On Tue, May 9, 2017 at 10:56 AM Michal
> > Borowiecki
> > >> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>> michal.borowie...@openbet.com> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> +1 :)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> On 08/05/17 23:52, Matthias J. Sax wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> I was reading the updated KIP and I am
> > >> wondering,
> > >>>>> if
> > >>>>>>> we
> > >>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>> do
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> design a little different.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Instead of distinguishing between a
> > RichFunction
> > >>>>> and
> > >>>>>>>>>>>>>>>>>>>>> non-RichFunction
> > >>>>>>>>>>>>>>>>>>>>>>>>> at
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> runtime level, we would use RichFunctions
> all
> > >> the
> > >>>>>>> time.
> > >>>>>>>>>>>>>>> Thus,
> > >>>>>>>>>>>>>>>> on
> > >>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>> DSL
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> entry level, if a user provides a
> > >>>> non-RichFunction,
> > >>>>>>> we
> > >>>>>>>>>>>>>> wrap
> > >>>>>>>>>>>>>>> it
> > >>>>>>>>>>>>>>>>>>>> by a
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> RichFunction that is fully implemented by
> > >>>> Streams.
> > >>>>>>> This
> > >>>>>>>>>>>>>>>>>>>>> RichFunction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> would just forward the call omitting the
> key:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Just to sketch the idea (incomplete code
> > >>>> snippet):
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> public StreamsRichValueMapper implements
> > >>>>>>>>>>>>>> RichValueMapper()
> > >>>>>>>>>>>>>>> {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>    private ValueMapper userProvidedMapper;
> //
> > >>>> set
> > >>>>> by
> > >>>>>>>>>>>>>>>>>> constructor
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>    public VR apply(final K key, final V1
> > >> value1,
> > >>>>>>>>> final V2
> > >>>>>>>>>>>>>>>>>>>> value2)
> > >>>>>>>>>>>>>>>>>>>>> {
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>        return userProvidedMapper(value1,
> > >>>> value2);
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>    }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> }
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>  From a performance point of view, I am not
> > sure
> > >>>> if
> > >>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> "if(isRichFunction)" including casts etc
> would
> > >>>> have
> > >>>>>>>>> more
> > >>>>>>>>>>>>>>>>>> overhead
> > >>>>>>>>>>>>>>>>>>>>>>> than
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> this approach (we would do more nested
> method
> > >>>> call
> > >>>>>>> for
> > >>>>>>>>>>>>>>>>>>>>>>> non-RichFunction
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> which should be more common than
> > RichFunctions).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> This approach should not effect lambdas (or
> > do I
> > >>>>> miss
> > >>>>>>>>>>>>>>>>>> something?)
> > >>>>>>>>>>>>>>>>>>>>> and
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> might be cleaner, as we could have one more
> > top
> > >>>>> level
> > >>>>>>>>>>>>>>>> interface
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> `RichFunction` with methods `init()` and
> > >>>> `close()`
> > >>>>>>> and
> > >>>>>>>>>>>>>> also
> > >>>>>>>>>>>>>>>>>>>>>>> interfaces
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> for `RichValueMapper` etc. (thus, no
> abstract
> > >>>>> classes
> > >>>>>>>>>>>>>>>> required).
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> Any thoughts?
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> -Matthias
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>> On 5/6/17 5:29 PM, Jeyhun Karimov wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hi,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Thanks for comments. I extended PR and KIP
> to
> > >>>>>>> include
> > >>>>>>>>>>>>>> rich
> > >>>>>>>>>>>>>>>>>>>>>>> functions.
> > >>>>>>>>>>>>>>>>>>>>>>>>> I
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> will still have to evaluate the cost of
> deep
> > >>>>> copying
> > >>>>>>>>> of
> > >>>>>>>>>>>>>>> keys.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Cheers,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> Jeyhun
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 5, 2017 at 8:02 PM Mathieu
> > Fenniak
> > >> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> mathieu.fenn...@replicon.com>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Hey Matthias,
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> My opinion would be that documenting the
> > >>>>>>>>> immutability of
> > >>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>> key
> > >>>>>>>>>>>>>>>>>>>>> is
> > >>>>>>>>>>>>>>>>>>>>>>>>> the
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> best approach available.  Better than
> > >> requiring
> > >>>>> the
> > >>>>>>>>> key
> > >>>>>>>>>>>>>> to
> > >>>>>>>>>>>>>>>> be
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> serializable
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> (as with Jeyhun's last pass at the PR), no
> > >>>>>>>>> performance
> > >>>>>>>>>>>>>>> risk.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> It'd be different if Java had immutable
> type
> > >>>>>>>>> constraints
> > >>>>>>>>>>>>>>> of
> > >>>>>>>>>>>>>>>>>>>> some
> > >>>>>>>>>>>>>>>>>>>>>>>>> kind.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> :-)
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Mathieu
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> On Fri, May 5, 2017 at 11:31 AM, Matthias
> J.
> > >>>> Sax
> > >>>>> <
> > >>>>>>>>>>>>>>>>>>>>>>>>>>> matth...@confluent.io>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>> wrote:
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> Agreed about RichFunction. If we follow
> > this
> > >>>>> path,
> > >>>>>>>>> it
> > >>>>>>>>>>>>>>>> should
> > >>>>>>>>>>>>>>>>>>>>> cover
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>>>> all(?) DSL interfaces.
> > >>>>>>>>>>>>>>>>>>>>>>>>>>>>
>



-- 
-- Guozhang

Reply via email to