Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2019-11-24 Thread Matthias J. Sax
>>>>>>>>>>>>> >>>>>>>>>>>>>>>>>> I think I'm convinced to not do KAFKA-3907 in this KIP. We >>>>>>>>>>>>>&

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2018-03-13 Thread Matthias J. Sax
Warren, thanks for following up this KIP. And sorry for the "messy" discussion thread. Adding this feature is a little tricky. We still hope to get it into 1.2 release, but atm there is not much progress. However, for your use case, you can replace .map() with .transform() that allows you to acce

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2018-03-13 Thread Warren, Brad
Hi devs, It's a bit difficult to put all of the pieces together regarding the status and API changes around the KIPs dealing with exposing the record metadata in the Processor and DSL APIs. This is a feature that my team here at American Airlines is keenly interested in and I'd like to provide

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-12-18 Thread Matthias J. Sax
t;>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>> clear >>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> if it is a good trade-off than asking users to call >> proc

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-12-18 Thread Bill Bejeck
t;>>>>>>>> > >>>>>>>>>>>>>>> I removed the 'commit()' feature, as we discussed. It > >>>>>>>>>>>>>>>> simplified > >>>>>>>>>>>>>>>> the > >>>>>>>

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-12-15 Thread Guozhang Wang
t;>>>> wrote: >>>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>>> Thanks. I understand what you are saying, but I don't >>>>>>>>>>>>>>>>> agree that >>>>>>>&g

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-12-07 Thread Jan Filipiak
l. Guozhang On Mon, Oct 23, 2017 at 1:40 PM, Matthias J. Sax < matth...@confluent.io wrote: Fair point. This is a long discussion and I totally forgot that we discussed this. Seems I changed my opinion about including KAFKA-3907... Happy to hear what others think. -Matth

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-12-05 Thread Bill Bejeck
gt;> information and not Record level info. The only thing we > >>>>>>>>>>>>>>>> need ij > >>>>>>>>>>>>>>>> ProcessorContext is commit() method. > >>>>>>>>>>>>>>>> > >>>>>&

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-12-04 Thread Matthias J. Sax
ter so users can use >>>>>>>>>>>>>>>> >>>>>>>>>>>>>>>> ProcessorContext.commit() >>>>>>>>>>>>>>> ). >>>>>>>>>>>>>>>> As a result, I thought to “propagate” c

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-29 Thread Jan Filipiak
hat we discussed this. Seems I changed my opinion about including KAFKA-3907... Happy to hear what others think. -Matthias On 10/23/17 1:20 PM, Jeyhun Karimov wrote: Hi Matthias, It is probably my bad, the discussion was a bit long in this thread. I proposed the related issue in the rela

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-28 Thread Guozhang Wang
t to request a commit? To >>>>>>>>>>>>>>> me, >>>>>>>>>>>>>>> requesting commits is useful if you manipulated state >>>>>>>>>>>>>>

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-24 Thread Jan Filipiak
On 10/23/17 1:20 PM, Jeyhun Karimov wrote: Hi Matthias, It is probably my bad, the discussion was a bit long in this thread. I proposed the related issue in the related KIP discuss thread [1] and got an approval [2,3]. Maybe I misunderstood. [1] http://search-hadoop.com/m/Kafka/uyzND19Asmg1GKKXT

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-23 Thread Matthias J. Sax
t;>>>>>>>> Hi, >>>>>>>>>>>>>> >>>>>>>>>>>>>> Thanks for your suggestions. >>>>>>>>>>>>>> >>>>>>>>>>>>>&

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-23 Thread Jan Filipiak
, Jeyhun Karimov wrote: Hi Matthias, It is probably my bad, the discussion was a bit long in this thread. I proposed the related issue in the related KIP discuss thread [1] and got an approval [2,3]. Maybe I misunderstood. [1] http://search-hadoop.com/m/Kafka/uyzND19Asmg1GKKXT1?subj=

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-22 Thread Damian Guy
hich is provided in Rich > interfaces. > >>>>>>>>>>> So > >>>>>>>>>>> > >>>>>>>>>> if > >>>>>>> > >>>>>>>> user > >>>>&

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-21 Thread Guozhang Wang
;>>>>>>> >>>>>>>>>>> do so. Internally, `RecordContext.commit()` calls >>>>>>>>>>> `ProcessorContext.commit()` (see the last code snippet in >>>>>>>>>>> >>>>>>>

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-20 Thread Jan Filipiak
bit long in this thread. I proposed the related issue in the related KIP discuss thread [1] and got an approval [2,3]. Maybe I misunderstood. [1] http://search-hadoop.com/m/Kafka/uyzND19Asmg1GKKXT1?subj= Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams [2] http://search-hadoop.com/m

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-20 Thread Matthias J. Sax
return context().recordContext().partition(); >>>>>>>>> } >>>>>>>>> }; >>>>>>>>> >>>>>>>>> >>>>>>>>> So, we cannot deprecate `ProcessorContext.commit()` in this case >>

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-06 Thread Jeyhun Karimov
t`. The main intuition is to "transfer" > >>>>>>> `ProcessorContext.commit()` call to Rich interfaces, to support > >>>>>>> user-specific committing. > >>>>>>> To do so, we introduce `commit()` method in `RecordC

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-06 Thread Matthias J. Sax
call ProcessorContext.commit() inside. (see the above code snippet) >>>>>>> So, in Rich interfaces, we are not dealing with >>>> `ProcessorRecordContext` >>>>>>> at all, and we leave all its methods as it is. >>>>>>> In this KI

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-01 Thread Guozhang Wang
would need to be updated as well. > > > >>> > > > >>> > > > >>> - I think above explanation covers this as well. > > > >>> > > > >>> > > > >>> I want to gain some speed to this KIP, as it

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-11-01 Thread Damian Guy
, as it has gone though many > > changes > > >>> based on user/developer needs, both in > > >> documentation-/implementation-wise. > > >>> > > >>> > > >>> Cheers, > > >>> Jeyhun > > >>> > > &g

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-31 Thread Jeyhun Karimov
s KIP.. > >>>> > >>>> Thinking a bit more, I'm now inclined to go with what we agreed > before, > >> to > >>>> add the commit() call to `RecordContext`. A few minor tweaks on its > >>>> implementation: > >>>

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-27 Thread Matthias J. Sax
;> implementation: >>>> >>>> 1. Maybe we can deprecate the `commit()` in ProcessorContext, to enforce >>>> user to consolidate this call as >>>> "processorContext.recordContext().commit()". And internal implementation >>>>

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-27 Thread Jeyhun Karimov
t; >> > >> 3. In the wiki page, the statement that "However, call to a commit() > >> method, > >> is valid only within RecordContext interface (at least for now), we > throw > >> an exception in ProcessorRecordContext.commit()." and the code snip

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-27 Thread Matthias J. Sax
Oct 23, 2017 at 1:40 PM, Matthias J. Sax >> wrote: >> >>> Fair point. This is a long discussion and I totally forgot that we >>> discussed this. >>> >>> Seems I changed my opinion about including KAFKA-3907... >>> >>> Happy to

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-26 Thread Jeyhun Karimov
; > > -Matthias > > > > On 10/23/17 1:20 PM, Jeyhun Karimov wrote: > > > Hi Matthias, > > > > > > It is probably my bad, the discussion was a bit long in this thread. I > > > proposed the related issue in the related KIP discuss thread [1] and >

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-23 Thread Guozhang Wang
approval [2,3]. > > Maybe I misunderstood. > > > > [1] > > http://search-hadoop.com/m/Kafka/uyzND19Asmg1GKKXT1?subj= > Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams > > [2] > > http://search-hadoop.com/m/Kafka/uyzND1kpct22GKKXT1?subj= > Re+DISCUSS+KI

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-23 Thread Matthias J. Sax
was a bit long in this thread. I > proposed the related issue in the related KIP discuss thread [1] and got an > approval [2,3]. > Maybe I misunderstood. > > [1] > http://search-hadoop.com/m/Kafka/uyzND19Asmg1GKKXT1?subj=Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+St

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-23 Thread Jeyhun Karimov
+functions+to+Streams [2] http://search-hadoop.com/m/Kafka/uyzND1kpct22GKKXT1?subj=Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams [3] http://search-hadoop.com/m/Kafka/uyzND1G6TGIGKKXT1?subj=Re+DISCUSS+KIP+159+Introducing+Rich+functions+to+Streams On Mon, Oct 23, 2017 at 8:44 PM Matthias J

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-23 Thread Matthias J. Sax
Interesting. I thought that https://issues.apache.org/jira/browse/KAFKA-4125 is the main motivation for this KIP :) I also think, that we should not expose the full ProcessorContext at DSL level. Thus, overall I am not even sure if we should fix KAFKA-3907 at all. Manual commits are something DS

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-18 Thread Jeyhun Karimov
Hi, The main intuition is to solve [1], which is part of this KIP. I agree with you that this might not seem semantically correct as we are not committing record state. Alternatively, we can remove commit() from RecordContext and add ProcessorContext (which has commit() method) as an extra argumen

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-16 Thread Guozhang Wang
Regarding #6 above, I'm still not clear why we would need `commit()` in both ProcessorContext and RecordContext, could you elaborate a bit more? To me `commit()` is really a processor context not a record context logically: when you call that function, it means we would commit the state of the who

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-16 Thread Jeyhun Karimov
Hi, Thanks for the feedback. 0. RichInitializer definition seems missing. - Fixed. I'd suggest moving the key parameter in the RichValueXX and RichReducer > after the value parameters, as well as in the templates; e.g. > public interface RichValueJoiner { > VR apply(final V1 value1, fi

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-10-10 Thread Guozhang Wang
Jeyhun, Thanks for the updated KIP, here are my comments. 0. RichInitializer definition seems missing. 1. I'd suggest moving the key parameter in the RichValueXX and RichReducer after the value parameters, as well as in the templates; e.g. public interface RichValueJoiner { VR apply(final V

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-22 Thread Ted Yu
recordContext = new RecordContext() { // recordContext initialization is added in this KIP This code snippet seems to be standard - would it make sense to pull it into a (sample) RecordContext implementation ? Cheers On Fri, Sep 22, 2017 at 12:14 PM, Jeyhun Karimov wrote: > Hi

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-22 Thread Jeyhun Karimov
Hi Ted, Thanks for your comments. I added a couple of comments in KIP to clarify some points. bq. provides a hybrd solution > Typo in hybrid. - My bad. Thanks for the correction. It would be nice if you can name some Value operator as examples. > - I added the corresponding interface names

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-22 Thread Ted Yu
bq. provides a hybrd solution Typo in hybrid. bq. accessing read-only keys within XXXValues operators It would be nice if you can name some Value operator as examples. KTable aggregate(final Initializer initializer, final Aggregator adder, The adder doesn't need t

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-22 Thread Jeyhun Karimov
Dear community, I updated the related KIP [1]. Please feel free to comment. Cheers, Jeyhun [1] https://cwiki.apache.org/confluence/display/KAFKA/KIP-159%3A+Introducing+Rich+functions+to+Streams On Fri, Sep 22, 2017 at 12:20 AM Jeyhun Karimov wrote: > Hi Damian, > > Thanks for the update. I

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-21 Thread Jeyhun Karimov
Hi Damian, Thanks for the update. I working on it and will provide an update soon. Cheers, Jeyhun On Thu, Sep 21, 2017 at 4:50 PM Damian Guy wrote: > Hi Jeyhun, > > All KIP-182 API PRs have now been merged. So you can consider it as stable. > Thanks, > Damian > > On Thu, 21 Sep 2017 at 15:23 J

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-21 Thread Damian Guy
Hi Jeyhun, All KIP-182 API PRs have now been merged. So you can consider it as stable. Thanks, Damian On Thu, 21 Sep 2017 at 15:23 Jeyhun Karimov wrote: > Hi all, > > Thanks a lot for your comments. For the single interface (RichXXX and > XXXWithKey) solution, I have already submitted a PR but

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-21 Thread Jeyhun Karimov
Hi all, Thanks a lot for your comments. For the single interface (RichXXX and XXXWithKey) solution, I have already submitted a PR but probably it is outdated (when the KIP first proposed), I need to revisit that one. @Guozhang, from our (offline) discussion, I understood that we may not make it m

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-21 Thread Damian Guy
+1 On Thu, 21 Sep 2017 at 13:46 Guozhang Wang wrote: > +1 for me as well for collapsing. > > Jeyhun, could you update the wiki accordingly to show what's the final > updates post KIP-182 that needs to be done in KIP-159 including KIP-149? > The child page I made is just a suggestion, but you wou

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-21 Thread Guozhang Wang
+1 for me as well for collapsing. Jeyhun, could you update the wiki accordingly to show what's the final updates post KIP-182 that needs to be done in KIP-159 including KIP-149? The child page I made is just a suggestion, but you would still need to update your proposal for people to comment and v

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-21 Thread Guozhang Wang
+1 On Thu, Sep 14, 2017 at 10:37 PM, Ted Yu wrote: > +1 > > One interface is cleaner. > > On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck wrote: > > > +1 for me on collapsing the Rich and ValueWithKey interfaces > into 1 > > interface. > > > > Thanks, > > Bill > > > > On Wed, Sep 13, 2017

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-14 Thread Ted Yu
+1 One interface is cleaner. On Thu, Sep 14, 2017 at 7:26 AM, Bill Bejeck wrote: > +1 for me on collapsing the Rich and ValueWithKey interfaces into 1 > interface. > > Thanks, > Bill > > On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov > wrote: > > > Hi Damian, > > > > Thanks for your

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-14 Thread Bill Bejeck
+1 for me on collapsing the Rich and ValueWithKey interfaces into 1 interface. Thanks, Bill On Wed, Sep 13, 2017 at 11:31 AM, Jeyhun Karimov wrote: > Hi Damian, > > Thanks for your feedback. Actually, this (what you propose) was the first > idea of KIP-149. Then we decided to divide it

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-13 Thread Jeyhun Karimov
Hi Damian, Thanks for your feedback. Actually, this (what you propose) was the first idea of KIP-149. Then we decided to divide it into two KIPs. I also expressed my opinion that keeping the two interfaces (Rich and withKey) separate would add more overloads. So, email discussion resulted that thi

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-13 Thread Damian Guy
Hi Jeyhun, Thanks for sending out the update. I guess i was thinking more along the lines of option 2 where we collapse the Rich and ValueWithKey etc interfaces into 1 interface that has all of the arguments. I think we then only need to add one additional overload for each operator? Than

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-09-13 Thread Jeyhun Karimov
Dear all, I would like to resume the discussion on KIP-159. I (and Guozhang) think that releasing KIP-149 and KIP-159 in the same release would make sense to avoid a release with "partial" public APIs. There is a KIP [1] proposed by Guozhang (and approved by me) to unify both KIPs. Please feel fre

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-20 Thread Jeyhun Karimov
Hi Matthias, Damian, all, Thanks for your comments and sorry for super-late update. Sure, the DSL refactoring is not blocking for this KIP. I made some changes to KIP document based on my prototype. Please feel free to comment. Cheers, Jeyhun On Fri, Jul 7, 2017 at 9:35 PM Matthias J. Sax wro

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-07 Thread Matthias J. Sax
I would not block this KIP with regard to DSL refactoring. IMHO, we can just finish this one and the DSL refactoring will help later on to reduce the number of overloads. -Matthias On 7/7/17 5:28 AM, Jeyhun Karimov wrote: > I am following the related thread in the mailing list and looking forward

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-07 Thread Jeyhun Karimov
I am following the related thread in the mailing list and looking forward for one-shot solution for overloads issue. Cheers, Jeyhun On Fri, Jul 7, 2017 at 10:32 AM Damian Guy wrote: > Hi Jeyhun, > > About overrides, what other alternatives do we have? For > > backwards-compatibility we have to

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-07 Thread Damian Guy
Hi Jeyhun, About overrides, what other alternatives do we have? For > backwards-compatibility we have to add extra methods to the existing ones. > > It wasn't clear to me in the KIP if these are new methods or replacing existing ones. Also, we are currently discussing options for replacing the ove

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-06 Thread Jeyhun Karimov
Hi Damian, Thanks for comments. About overrides, what other alternatives do we have? For backwards-compatibility we have to add extra methods to the existing ones. About ProcessorContext vs RecordContext, you are right. I think I need to implement a prototype to understand the full picture as som

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-05 Thread Damian Guy
HI Jeyhun, Is the intention that these methods are new overloads on the KStream, KTable, etc? It is worth noting that a ProcessorContext is not a RecordContext. A RecordContext, as it stands, only exists during the processing of a single record. Whereas the ProcessorContext exists for the lifetim

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-07-04 Thread Jeyhun Karimov
Hi Matthias, Actually my intend was to provide to RichInitializer and later on we could provide the context of the record as you also mentioned. I remove that not to confuse the users. Regarding the RecordContext and ProcessorContext interfaces, I just realized the InternalProcessorContext class.

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-29 Thread Matthias J. Sax
One more thing: I don't think `RichInitializer` does make sense. As we don't have any input record, there is also no context. We could of course provide the context of the record that triggers the init call, but this seems to be semantically questionable. Also, the context for this first record wi

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-29 Thread Matthias J. Sax
Thanks for updating the KIP. I have one concern with regard to backward compatibility. You suggest to use RecrodContext as base interface for ProcessorContext. This will break compatibility. I think, we should just have two independent interfaces. Our own ProcessorContextImpl class would implemen

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-27 Thread Jeyhun Karimov
Hi all, I updated the KIP w.r.t. discussion and comments. Basically I eliminated overloads for particular method if they are more than 3. As we can see there are a lot of overloads (and more will come with KIP-149 :) ) So, is it wise to wait the result of constructive DSL thread or extend KIP to a

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-14 Thread Matthias J. Sax
Including KAFKA-3907 sounds reasonable to me. -Matthias On 6/14/17 2:29 PM, Guozhang Wang wrote: > LGTM. Thanks! > > > Guozhang > > On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov > wrote: > >> Thanks for the comment Matthias. After all the discussion (thanks to all >> participants), I think

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-14 Thread Guozhang Wang
LGTM. Thanks! Guozhang On Tue, Jun 13, 2017 at 2:20 PM, Jeyhun Karimov wrote: > Thanks for the comment Matthias. After all the discussion (thanks to all > participants), I think this (single method that passes in a RecordContext > object) is the best alternative. > Just a side note: I think KA

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-13 Thread Jeyhun Karimov
Thanks for the comment Matthias. After all the discussion (thanks to all participants), I think this (single method that passes in a RecordContext object) is the best alternative. Just a side note: I think KAFKA-3907 [1] can also be integrated into the KIP by adding related method inside RecordCont

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-13 Thread Matthias J. Sax
Hi, I would like to push this discussion further. It seems we got nice alternatives (thanks for the summary Jeyhun!). With respect to RichFunctions and allowing them to be stateful, I have my doubt as expressed already. From my understanding, the idea was to give access to record metadata informa

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-06 Thread Guozhang Wang
Thanks for the comprehensive summary! Personally I'd prefer the option of passing RecordContext as an additional parameter into he overloaded function. But I'm also open to other arguments if there are sth. that I have overlooked. Guozhang On Mon, Jun 5, 2017 at 3:19 PM, Jeyhun Karimov wrote:

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-05 Thread Jeyhun Karimov
Hi, Thanks for your comments Matthias and Guozhang. Below I mention the quick summary of the main alternatives we looked at to introduce the Rich functions (I will refer to it as Rich functions until we find better/another name). Initially the proposed alternatives was not backwards-compatible, s

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-04 Thread Matthias J. Sax
Yes, we did consider this, and there is no consensus yet what the best alternative is. @Jeyhun: the email thread got pretty long. Maybe you can give a quick summary of the current state of the discussion? -Matthias On 6/4/17 6:04 PM, Guozhang Wang wrote: > Thanks for the explanation Jeyhun and

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-04 Thread Guozhang Wang
Thanks for the explanation Jeyhun and Matthias. I have just read through both KIP-149 and KIP-159 and am wondering if you guys have considered a slight different approach for rich function, that is to add the `RecordContext` into the apply functions as an additional parameter. For example: --

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-01 Thread Matthias J. Sax
I agree with Jeyhun. As already mention, the overall API improvement ideas are overlapping and/or contradicting each other. For this reason, not all ideas can be accomplished and some Jira might just be closed as "won't fix". For this reason, we try to do those KIP discussion with are large scope

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-01 Thread Jeyhun Karimov
Hi Guozhang, It subsumes partially. Initially the idea was to support RichFunctions as a separate interface. Throughout the discussion, however, we considered maybe overloading the related methods (with RecodContext param) is better approach than providing a separate RichFunction interface. Cheer

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-06-01 Thread Guozhang Wang
Does this KIP subsume this ticket as well? https://issues.apache.org/jira/browse/KAFKA-4125 On Sat, May 20, 2017 at 9:05 AM, Jeyhun Karimov wrote: > Dear community, > > As we discussed in KIP-149 [DISCUSS] thread [1], I would like to initiate > KIP for rich functions (interfaces) [2]. > I would

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-28 Thread Jeyhun Karimov
After your response on KIP-149 related with ValueTransformerSupplier, everything you mentioned now makes complete sense. Thanks for clarification. Just a note: We will have additional (to KIP-149) overloaded methods: for each withKey and withoutKey methods (ValueMapper and ValueMapperWithKey) we w

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-28 Thread Matthias J. Sax
Thanks for you comments Jeyhun, I agree about the disadvantages. Only the punctuation part is something I don't buy. IMHO, RichFunctions should not allow to register and use punctuation. If you need punctuation, you should use #transform() or similar. Note, that we plan to provide `RecordContext`

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-27 Thread Jeyhun Karimov
Hi, Thanks for your comments. I will refer the overall approach as rich functions until we find a better name. I think there are some pros and cons of the approach you described. Pros is that it is simple, has clear boundaries, avoids misunderstanding of term "function". So you propose sth like:

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-24 Thread Matthias J. Sax
I confess, the term is borrowed from Flink :) Personally, I never thought about it, but I tend to agree with Michal. I also want to clarify, that the main purpose is the ability to access record metadata. Thus, it might even be sufficient to only have "init". An alternative would of course be, to

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-24 Thread Jeyhun Karimov
Hi Michal, Thanks for your comments. I see your point and I agree with it. However, I don't have a better idea for naming. I checked MR source code. There it is used JobConfigurable and Closable, two different interfaces. Maybe we can rename RichFunction as Configurable? Cheers, Jeyhun On Tue,

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-23 Thread Michal Borowiecki
Hi Jeyhun, I understand your argument about "Rich" in RichFunctions. Perhaps I'm just being too puritan here, but let me ask this anyway: What is it that makes something a function? To me a function is something that takes zero or more arguments and possibly returns a value and while it may

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-23 Thread Jeyhun Karimov
Hi Michal, Thanks for your comments. To me at least it feels strange that something is called a function yet > doesn't follow the functional interface definition of having just one > abstract method. I suppose init and close could be made default methods > with empty bodies once Java 7 support i

Re: [DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-22 Thread Michal Borowiecki
Hi Jeyhun, I'd like to understand better the premise of RichFunctions and why |init(Some params)|,|close() |are said to be needed. To me at least it feels strange that something is called a function yet doesn't follow the functional interface definition of having just one abstract method. I

[DISCUSS]: KIP-159: Introducing Rich functions to Streams

2017-05-20 Thread Jeyhun Karimov
Dear community, As we discussed in KIP-149 [DISCUSS] thread [1], I would like to initiate KIP for rich functions (interfaces) [2]. I would like to get your comments. [1] http://search-hadoop.com/m/Kafka/uyzND1PMjdk2CslH12?subj=Re+DISCUSS+KIP+149+Enabling+key+access+in+ValueTransformer+ValueMappe